Merge branch 'master' of https://code.google.com/p/hyracks
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
index 336c3fc..bfcafe3 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
@@ -62,7 +62,7 @@
resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
}
- public void close() {
+ public void close() throws HyracksDataException {
bufferCache.close();
}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/LSMTreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/LSMTreeOperatorTestHelper.java
index 80b5285..f257b46 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/LSMTreeOperatorTestHelper.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/LSMTreeOperatorTestHelper.java
@@ -21,13 +21,20 @@
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+import edu.uci.ics.hyracks.test.support.TestVirtualBufferCacheProvider;
public class LSMTreeOperatorTestHelper extends TreeOperatorTestHelper {
-
+ protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
+ protected static int DEFAULT_MEM_NUM_PAGES = 1000;
+ protected static double DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
protected final IOManager ioManager;
+ protected final IVirtualBufferCacheProvider virtualBufferCacheProvider;
public LSMTreeOperatorTestHelper(IOManager ioManager) {
this.ioManager = ioManager;
+ this.virtualBufferCacheProvider = new TestVirtualBufferCacheProvider(DEFAULT_MEM_PAGE_SIZE,
+ DEFAULT_MEM_NUM_PAGES);
}
public String getPrimaryIndexName() {
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java
index 932e166..362badb 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java
@@ -23,9 +23,6 @@
protected final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
protected final String sep = System.getProperty("file.separator");
- protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
- protected static int DEFAULT_MEM_NUM_PAGES = 1000;
- protected static double DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
public String getPrimaryIndexName() {
return System.getProperty("java.io.tmpdir") + sep + "primary" + simpleDateFormat.format(new Date());
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
index 808afac..cd3cda7 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
@@ -57,6 +57,7 @@
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexBulkLoadOperatorDescriptor;
@@ -73,6 +74,7 @@
import edu.uci.ics.hyracks.test.support.TestIndexLifecycleManagerProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestVirtualBufferCacheProvider;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
@SuppressWarnings("rawtypes")
@@ -83,6 +85,8 @@
protected static final int MERGE_THRESHOLD = 3;
+ protected IVirtualBufferCacheProvider virtualBufferCacheProvider = new TestVirtualBufferCacheProvider(
+ DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES);
protected IStorageManagerInterface storageManager = new TestStorageManagerInterface();
protected IIndexLifecycleManagerProvider lcManagerProvider = new TestIndexLifecycleManagerProvider();
protected IIndexDataflowHelperFactory btreeDataflowHelperFactory = new BTreeDataflowHelperFactory();
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java
index dfd5495..aa3bccd 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java
@@ -29,7 +29,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
public class PartitionedWordInvertedIndexTest extends AbstractfWordInvertedIndexTest {
@@ -52,9 +52,9 @@
PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) };
invertedIndexDataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
- new ConstantMergePolicyProvider(MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
- SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE,
- DEFAULT_MEM_NUM_PAGES, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
+ virtualBufferCacheProvider, new ConstantMergePolicyProvider(MERGE_THRESHOLD),
+ ThreadCountingOperationTrackerProvider.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
+ NoOpIOOperationCallback.INSTANCE, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
}
@Override
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
index 9e5a47b..d08ad07 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
@@ -27,7 +27,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
public class WordInvertedIndexTest extends AbstractfWordInvertedIndexTest {
@@ -46,10 +46,10 @@
PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) };
- invertedIndexDataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(new ConstantMergePolicyProvider(
- MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
- SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE,
- DEFAULT_MEM_NUM_PAGES, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
+ invertedIndexDataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(virtualBufferCacheProvider,
+ new ConstantMergePolicyProvider(MERGE_THRESHOLD), ThreadCountingOperationTrackerProvider.INSTANCE,
+ SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE,
+ DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
}
@Override
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
index c67cbfd..4a7cf86 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -21,7 +21,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerProvider;
import edu.uci.ics.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
public class LSMBTreeOperatorTestHelper extends LSMTreeOperatorTestHelper {
@@ -33,9 +33,9 @@
}
public IIndexDataflowHelperFactory createDataFlowHelperFactory() {
- return new LSMBTreeDataflowHelperFactory(new ConstantMergePolicyProvider(MERGE_THRESHOLD),
- ThreadCountingOperationTrackerFactory.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
- NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+ return new LSMBTreeDataflowHelperFactory(virtualBufferCacheProvider, new ConstantMergePolicyProvider(
+ MERGE_THRESHOLD), ThreadCountingOperationTrackerProvider.INSTANCE,
+ SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE,
DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
index a892b4e..42c2739 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
@@ -23,7 +23,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
import edu.uci.ics.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
@@ -40,8 +40,8 @@
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
return new LSMRTreeDataflowHelperFactory(valueProviderFactories, rtreePolicyType, btreeComparatorFactories,
- new ConstantMergePolicyProvider(MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
- SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory,
- DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
+ virtualBufferCacheProvider, new ConstantMergePolicyProvider(MERGE_THRESHOLD),
+ ThreadCountingOperationTrackerProvider.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
+ NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
}
}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
index bb31dcd..94e7511 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
@@ -23,7 +23,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterTuplesDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
import edu.uci.ics.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
@@ -40,8 +40,8 @@
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
return new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(valueProviderFactories, rtreePolicyType,
- btreeComparatorFactories, new ConstantMergePolicyProvider(MERGE_THRESHOLD),
- ThreadCountingOperationTrackerFactory.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
+ btreeComparatorFactories, virtualBufferCacheProvider, new ConstantMergePolicyProvider(MERGE_THRESHOLD),
+ ThreadCountingOperationTrackerProvider.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory);
}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IInMemoryFreePageManager.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IInMemoryFreePageManager.java
deleted file mode 100644
index 67935a7..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IInMemoryFreePageManager.java
+++ /dev/null
@@ -1,9 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.common.api;
-
-public interface IInMemoryFreePageManager extends IFreePageManager {
- public int getCapacity();
-
- public void reset();
-
- public boolean isFull();
-}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexLifecycleManager.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexLifecycleManager.java
index c4f43b9..854d34e 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexLifecycleManager.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexLifecycleManager.java
@@ -5,7 +5,7 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public interface IIndexLifecycleManager {
- public IIndex getIndex(long resourceID);
+ public IIndex getIndex(long resourceID) throws HyracksDataException;
public void register(long resourceID, IIndex index) throws HyracksDataException;
@@ -13,7 +13,7 @@
public void open(long resourceID) throws HyracksDataException;
- public void close(long resourceID);
+ public void close(long resourceID) throws HyracksDataException;
public List<IIndex> getOpenIndexes();
}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IVirtualFreePageManager.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IVirtualFreePageManager.java
new file mode 100644
index 0000000..e6819b9
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IVirtualFreePageManager.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+public interface IVirtualFreePageManager extends IFreePageManager {
+ public int getCapacity();
+
+ public void reset();
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index 9bf4a4f..8745e0e 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -118,7 +118,7 @@
public synchronized void activate() throws HyracksDataException {
if (isActivated) {
- return;
+ throw new HyracksDataException("Failed to activate the index since it is already activated.");
}
boolean fileIsMapped = false;
@@ -149,7 +149,7 @@
public synchronized void deactivate() throws HyracksDataException {
if (!isActivated) {
- return;
+ throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
}
bufferCache.closeFile(fileId);
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
index eb2e760..12e5799 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
@@ -17,54 +17,43 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
-import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
public class LSMBTreeDataflowHelper extends AbstractLSMIndexDataflowHelper {
public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
- DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
- ioOpCallbackProvider);
+ IVirtualBufferCache virtualBufferCache, ILSMMergePolicy mergePolicy,
+ ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
+ opTrackerFactory, ioScheduler, ioOpCallbackProvider);
}
public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+ ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ super(opDesc, ctx, partition, virtualBufferCache, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+ ioScheduler, ioOpCallbackProvider);
}
@Override
public ITreeIndex createIndexInstance() throws HyracksDataException {
AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
- ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
- IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
- memNumPages, new TransientFileMapManager());
- IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(memNumPages, metaDataFrameFactory);
- return LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ctx.getIOManager(), file, opDesc
- .getStorageManager().getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx),
- treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc
- .getTreeIndexBloomFilterKeyFields(), bloomFilterFalsePositiveRate, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider,
- opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
+ return LSMBTreeUtils.createLSMTree(virtualBufferCache, ctx.getIOManager(), file, opDesc.getStorageManager()
+ .getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc
+ .getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc
+ .getTreeIndexBloomFilterKeyFields(), bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory
+ .getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider, opDesc.getFileSplitProvider()
+ .getFileSplits()[partition].getIODeviceId());
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
index e706786..cc752b9 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
@@ -21,26 +21,28 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
public class LSMBTreeDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
private static final long serialVersionUID = 1L;
- public LSMBTreeDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+ public LSMBTreeDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
+ ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerFactory,
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
double bloomFilterFalsePositiveRate) {
- super(mergePolicyProvider, opTrackerFactory, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
- memNumPages, bloomFilterFalsePositiveRate);
+ super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerFactory, ioSchedulerProvider,
+ ioOpCallbackProvider, bloomFilterFalsePositiveRate);
}
@Override
public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
- return new LSMBTreeDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
- bloomFilterFalsePositiveRate, mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory,
- ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider);
+ return new LSMBTreeDataflowHelper(opDesc, ctx, partition,
+ virtualBufferCacheProvider.getVirtualBufferCache(ctx), bloomFilterFalsePositiveRate,
+ mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
+ ioOpCallbackProvider);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index 1652f5a..71be9be 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -34,7 +34,6 @@
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
@@ -51,7 +50,6 @@
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
@@ -63,8 +61,9 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
@@ -89,20 +88,19 @@
private final ITreeIndexFrameFactory deleteLeafFrameFactory;
private final IBinaryComparatorFactory[] cmpFactories;
- public LSMBTree(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
- ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
- ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager,
- TreeIndexFactory<BTree> diskBTreeFactory, TreeIndexFactory<BTree> bulkLoadBTreeFactory,
- BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
- IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+ public LSMBTree(IVirtualBufferCache virtualBufferCache, ITreeIndexFrameFactory interiorFrameFactory,
+ ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory,
+ ILSMIndexFileManager fileManager, TreeIndexFactory<BTree> diskBTreeFactory,
+ TreeIndexFactory<BTree> bulkLoadBTreeFactory, BloomFilterFactory bloomFilterFactory,
+ double bloomFilterFalsePositiveRate, IFileMapProvider diskFileMapProvider, int fieldCount,
+ IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- super(memFreePageManager, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
- bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
- mutableComponent = new LSMBTreeMutableComponent(new BTree(memBufferCache,
- ((InMemoryBufferCache) memBufferCache).getFileMapProvider(), memFreePageManager, interiorFrameFactory,
- insertLeafFrameFactory, cmpFactories, fieldCount, new FileReference(new File("membtree"))),
- memFreePageManager);
+ super(virtualBufferCache, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ mutableComponent = new LSMBTreeMutableComponent(new BTree(virtualBufferCache,
+ virtualBufferCache.getFileMapProvider(), new VirtualFreePageManager(virtualBufferCache.getNumPages()),
+ interiorFrameFactory, insertLeafFrameFactory, cmpFactories, fieldCount, new FileReference(new File(
+ fileManager.getBaseDir() + "_virtual"))), virtualBufferCache);
this.insertLeafFrameFactory = insertLeafFrameFactory;
this.deleteLeafFrameFactory = deleteLeafFrameFactory;
this.cmpFactories = cmpFactories;
@@ -124,10 +122,10 @@
@Override
public synchronized void activate() throws HyracksDataException {
if (isActivated) {
- return;
+ throw new HyracksDataException("Failed to activate the index since it is already activated.");
}
- ((InMemoryBufferCache) mutableComponent.getBTree().getBufferCache()).open();
+ ((IVirtualBufferCache) mutableComponent.getBTree().getBufferCache()).open();
mutableComponent.getBTree().create();
mutableComponent.getBTree().activate();
List<ILSMComponent> immutableComponents = componentsRef.get();
@@ -155,7 +153,7 @@
@Override
public synchronized void deactivate(boolean flushOnExit) throws HyracksDataException {
if (!isActivated) {
- return;
+ throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
}
if (flushOnExit) {
@@ -180,7 +178,7 @@
}
mutableComponent.getBTree().deactivate();
mutableComponent.getBTree().destroy();
- ((InMemoryBufferCache) mutableComponent.getBTree().getBufferCache()).close();
+ ((IVirtualBufferCache) mutableComponent.getBTree().getBufferCache()).close();
isActivated = false;
}
@@ -622,8 +620,8 @@
@Override
public long getMemoryAllocationSize() {
- InMemoryBufferCache memBufferCache = (InMemoryBufferCache) mutableComponent.getBTree().getBufferCache();
- return memBufferCache.getNumPages() * memBufferCache.getPageSize();
+ IBufferCache virtualBufferCache = mutableComponent.getBTree().getBufferCache();
+ return virtualBufferCache.getNumPages() * virtualBufferCache.getPageSize();
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMutableComponent.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMutableComponent.java
index d340599..5444d6b 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMutableComponent.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMutableComponent.java
@@ -17,17 +17,17 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMutableLSMComponent;
public class LSMBTreeMutableComponent extends AbstractMutableLSMComponent {
private final BTree btree;
- private final IInMemoryFreePageManager mfpm;
+ private final IVirtualBufferCache vbc;
- public LSMBTreeMutableComponent(BTree btree, IInMemoryFreePageManager mfpm) {
+ public LSMBTreeMutableComponent(BTree btree, IVirtualBufferCache vbc) {
this.btree = btree;
- this.mfpm = mfpm;
+ this.vbc = vbc;
}
public BTree getBTree() {
@@ -36,13 +36,16 @@
@Override
protected boolean isFull() {
- return mfpm.isFull();
+ return vbc.isFull();
}
@Override
protected void reset() throws HyracksDataException {
super.reset();
- btree.clear();
+ btree.deactivate();
+ btree.destroy();
+ btree.create();
+ btree.activate();
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index 5736200..c61a63c 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -122,6 +122,7 @@
@Override
public void reset() throws HyracksDataException, IndexException {
+ try {
if (rangeCursors != null) {
for (int i = 0; i < rangeCursors.length; ++i) {
rangeCursors[i].reset();
@@ -130,6 +131,11 @@
rangeCursors = null;
nextHasBeenCalled = false;
foundTuple = false;
+ } finally {
+ if (lsmHarness != null) {
+ lsmHarness.endSearch(opCtx);
+ }
+ }
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
index d92e93d..7707712 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
@@ -24,7 +24,6 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManagerFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
@@ -33,23 +32,33 @@
import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeCopyTupleWriterFactory;
import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public class LSMBTreeUtils {
- public static LSMBTree createLSMTree(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
- IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
- IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+
+ public static LSMBTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
+ FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+ ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
+ double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ return createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits,
+ cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
+ ioOpCallbackProvider, 0);
+ }
+
+ public static LSMBTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
+ FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+ ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
+ double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) {
LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
cmpFactories.length, false);
@@ -77,10 +86,10 @@
ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, diskFileMapProvider, file,
diskBTreeFactory, ioDeviceId);
- LSMBTree lsmTree = new LSMBTree(memBufferCache, memFreePageManager, interiorFrameFactory,
- insertLeafFrameFactory, deleteLeafFrameFactory, fileNameManager, diskBTreeFactory,
- bulkLoadBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider,
- typeTraits.length, cmpFactories, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ LSMBTree lsmTree = new LSMBTree(virtualBufferCache, interiorFrameFactory, insertLeafFrameFactory,
+ deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
+ bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length, cmpFactories, mergePolicy,
+ opTracker, ioScheduler, ioOpCallbackProvider);
return lsmTree;
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IInMemoryBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IInMemoryBufferCache.java
deleted file mode 100644
index 082ad2f..0000000
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IInMemoryBufferCache.java
+++ /dev/null
@@ -1,7 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.lsm.common.api;
-
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCacheInternal;
-
-public interface IInMemoryBufferCache extends IBufferCacheInternal {
- public void open();
-}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java
index f2c052b..bec8a2e 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java
@@ -19,7 +19,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -59,12 +58,12 @@
*/
public void getOperationalComponents(ILSMIndexOperationContext ctx);
- public IInMemoryFreePageManager getInMemoryFreePageManager();
-
public List<ILSMComponent> getImmutableComponents();
public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException;
public void setFlushStatus(boolean needsFlush);
+ public boolean isFull();
+
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTracker.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTracker.java
index c3f1f3e..273f589 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTracker.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTracker.java
@@ -22,7 +22,7 @@
* then this method does not block and returns false.
* Otherwise, this method returns true, and the operation is considered 'active' in the index.
*/
- public void beforeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+ public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException;
/**
@@ -31,7 +31,7 @@
* After this method has been called, the operation is still considered 'active',
* until the issuer of the operation declares it completed by calling completeOperation().
*/
- public void afterOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+ public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException;
/**
@@ -39,6 +39,6 @@
* The use of this method indicates that the operation is no longer 'active'
* for the purpose of coordinating flushes/merges.
*/
- public void completeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+ public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException;
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java
deleted file mode 100644
index db7ff6c..0000000
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java
+++ /dev/null
@@ -1,7 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.lsm.common.api;
-
-import java.io.Serializable;
-
-public interface ILSMOperationTrackerFactory extends Serializable {
- public ILSMOperationTracker createOperationTracker(ILSMIndex index);
-}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerProvider.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerProvider.java
new file mode 100644
index 0000000..b1ea4ee
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerProvider.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public interface ILSMOperationTrackerProvider extends Serializable {
+ public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IVirtualBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IVirtualBufferCache.java
new file mode 100644
index 0000000..96a7404
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IVirtualBufferCache.java
@@ -0,0 +1,15 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
+
+public interface IVirtualBufferCache extends IBufferCache {
+ public void open() throws HyracksDataException;
+
+ public boolean isFull();
+
+ public void reset();
+
+ public IFileMapManager getFileMapProvider();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java
new file mode 100644
index 0000000..06f8bd9
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public interface IVirtualBufferCacheProvider extends Serializable {
+ public IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
index 6202518..d7e5a22 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
@@ -21,38 +21,35 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
public abstract class AbstractLSMIndexDataflowHelper extends IndexDataflowHelper {
- protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
- protected static int DEFAULT_MEM_NUM_PAGES = 1000;
protected static double DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
- protected final int memPageSize;
- protected final int memNumPages;
protected final double bloomFilterFalsePositiveRate;
+ protected final IVirtualBufferCache virtualBufferCache;
protected final ILSMMergePolicy mergePolicy;
protected final ILSMIOOperationScheduler ioScheduler;
- protected final ILSMOperationTrackerFactory opTrackerFactory;
+ protected final ILSMOperationTrackerProvider opTrackerFactory;
protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
- DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
- ioOpCallbackProvider);
+ IVirtualBufferCache virtualBufferCache, ILSMMergePolicy mergePolicy,
+ ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
+ opTrackerFactory, ioScheduler, ioOpCallbackProvider);
}
public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+ ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
super(opDesc, ctx, partition);
- this.memPageSize = memPageSize;
- this.memNumPages = memNumPages;
+ this.virtualBufferCache = virtualBufferCache;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.mergePolicy = mergePolicy;
this.opTrackerFactory = opTrackerFactory;
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
index bc4271c..1ff1251 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
@@ -19,29 +19,28 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
public abstract class AbstractLSMIndexDataflowHelperFactory implements IIndexDataflowHelperFactory {
protected static final long serialVersionUID = 1L;
+ protected final IVirtualBufferCacheProvider virtualBufferCacheProvider;
protected final ILSMMergePolicyProvider mergePolicyProvider;
- protected final ILSMOperationTrackerFactory opTrackerFactory;
+ protected final ILSMOperationTrackerProvider opTrackerFactory;
protected final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
- protected final int memPageSize;
- protected final int memNumPages;
protected final double bloomFilterFalsePositiveRate;
- public AbstractLSMIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+ public AbstractLSMIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
+ ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerFactory,
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
double bloomFilterFalsePositiveRate) {
+ this.virtualBufferCacheProvider = virtualBufferCacheProvider;
this.mergePolicyProvider = mergePolicyProvider;
this.opTrackerFactory = opTrackerFactory;
this.ioSchedulerProvider = ioSchedulerProvider;
this.ioOpCallbackProvider = ioOpCallbackProvider;
- this.memPageSize = memPageSize;
- this.memNumPages = memNumPages;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
index a3e84ed..fdfcbd4 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -39,7 +39,7 @@
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
accessor.reset(buffer);
ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
- int lastFlushedTupleIndex = 0;
+ int nextFlushTupleIndex = 0;
int tupleCount = accessor.getTupleCount();
for (int i = 0; i < tupleCount; i++) {
try {
@@ -55,32 +55,32 @@
switch (op) {
case INSERT: {
if (!lsmAccessor.tryInsert(tuple)) {
- flushPartialFrame(lastFlushedTupleIndex, i);
- lastFlushedTupleIndex = (i == 0) ? 0 : i - 1;
+ flushPartialFrame(nextFlushTupleIndex, i);
+ nextFlushTupleIndex = i;
lsmAccessor.insert(tuple);
}
break;
}
case DELETE: {
if (!lsmAccessor.tryDelete(tuple)) {
- flushPartialFrame(lastFlushedTupleIndex, i);
- lastFlushedTupleIndex = (i == 0) ? 0 : i - 1;
+ flushPartialFrame(nextFlushTupleIndex, i);
+ nextFlushTupleIndex = i;
lsmAccessor.delete(tuple);
}
break;
}
case UPSERT: {
if (!lsmAccessor.tryUpsert(tuple)) {
- flushPartialFrame(lastFlushedTupleIndex, i);
- lastFlushedTupleIndex = (i == 0) ? 0 : i - 1;
+ flushPartialFrame(nextFlushTupleIndex, i);
+ nextFlushTupleIndex = i;
lsmAccessor.upsert(tuple);
}
break;
}
case UPDATE: {
if (!lsmAccessor.tryUpdate(tuple)) {
- flushPartialFrame(lastFlushedTupleIndex, i);
- lastFlushedTupleIndex = (i == 0) ? 0 : i - 1;
+ flushPartialFrame(nextFlushTupleIndex, i);
+ nextFlushTupleIndex = i;
lsmAccessor.update(tuple);
}
break;
@@ -96,13 +96,13 @@
throw new HyracksDataException(e);
}
}
- if (lastFlushedTupleIndex == 0) {
+ if (nextFlushTupleIndex == 0) {
// No partial flushing was necessary. Forward entire frame.
System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
FrameUtils.flushFrame(writeBuffer, writer);
} else {
// Flush remaining partial frame.
- flushPartialFrame(lastFlushedTupleIndex, tupleCount);
+ flushPartialFrame(nextFlushTupleIndex, tupleCount);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 6297576..0dbb6a7 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -37,8 +37,8 @@
private static final long serialVersionUID = 1L;
- private final int[] fieldPermutation;
- private final IndexOperation op;
+ protected final int[] fieldPermutation;
+ protected final IndexOperation op;
public LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryBufferCache.java
deleted file mode 100644
index c24c473..0000000
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryBufferCache.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.common.freepage;
-
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
-import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
-
-/**
- * In-memory buffer cache that supports two tree indexes.
- * We assume that the tree indexes have 2 fixed pages, one at index 0 (metadata page), and one at index 1 (root page).
- */
-public class DualIndexInMemoryBufferCache extends InMemoryBufferCache {
-
- public DualIndexInMemoryBufferCache(ICacheMemoryAllocator allocator, int pageSize, int numPages) {
- super(allocator, pageSize, numPages, new TransientFileMapManager());
- }
-
- @Override
- public ICachedPage pin(long dpid, boolean newPage) {
- int pageId = BufferedFileHandle.getPageId(dpid);
- int fileId = BufferedFileHandle.getFileId(dpid);
- if (pageId < pages.length) {
- // Common case: Return regular page.
- if (pageId == 0 || pageId == 1) {
- return pages[pageId + 2 * fileId];
- } else {
- return pages[pageId];
- }
- } else {
- // Rare case: Return overflow page, possibly expanding overflow
- // array.
- synchronized (overflowPages) {
- int numNewPages = pageId - pages.length - overflowPages.size() + 1;
- if (numNewPages > 0) {
- ByteBuffer[] buffers = allocator.allocate(pageSize, numNewPages);
- for (int i = 0; i < numNewPages; i++) {
- CachedPage overflowPage = new CachedPage(pages.length + overflowPages.size(), buffers[i]);
- overflowPages.add(overflowPage);
- }
- }
- return overflowPages.get(pageId - pages.length);
- }
- }
- }
-
-}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryFreePageManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryFreePageManager.java
deleted file mode 100644
index 7a2be7f..0000000
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryFreePageManager.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.common.freepage;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-
-/**
- * In-memory free page manager that supports two tree indexes.
- * We assume that the tree indexes have 2 fixed pages, one at index 0 (metadata page), and one at index 1 (root page).
- */
-public class DualIndexInMemoryFreePageManager extends InMemoryFreePageManager {
-
- public DualIndexInMemoryFreePageManager(int capacity, ITreeIndexMetaDataFrameFactory metaDataFrameFactory) {
- super(capacity, metaDataFrameFactory);
- // We start the currentPageId from 3, because the RTree uses
- // the first page as metadata page, and the second page as root page.
- // And the BTree uses the third page as metadata, and the third page as root page
- // (when returning free pages we first increment, then get)
- currentPageId.set(3);
- }
-
- @Override
- public void init(ITreeIndexMetaDataFrame metaFrame, int currentMaxPage) throws HyracksDataException {
- currentPageId.set(3);
- }
-
- public int getCapacity() {
- return capacity - 4;
- }
-
- public void reset() {
- currentPageId.set(3);
- }
-}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryBufferCache.java
deleted file mode 100644
index 66d8ec2..0000000
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryBufferCache.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * Copyright 2009-2012 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.common.freepage;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPageInternal;
-import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-
-public class InMemoryBufferCache implements IInMemoryBufferCache {
- protected final ICacheMemoryAllocator allocator;
- protected final IFileMapManager fileMapManager;
- protected final int pageSize;
- protected final int numPages;
- protected final List<CachedPage> overflowPages = new ArrayList<CachedPage>();
- protected CachedPage[] pages;
-
- public InMemoryBufferCache(ICacheMemoryAllocator allocator, int pageSize, int numPages,
- IFileMapManager fileMapManager) {
- this.allocator = allocator;
- this.fileMapManager = fileMapManager;
- this.pageSize = pageSize;
- this.numPages = numPages;
- }
-
- public void open() {
- pages = new CachedPage[numPages];
- ByteBuffer[] buffers = allocator.allocate(pageSize, numPages);
- for (int i = 0; i < buffers.length; ++i) {
- pages[i] = new CachedPage(i, buffers[i]);
- }
- }
-
- @Override
- public ICachedPage pin(long dpid, boolean newPage) {
- int pageId = BufferedFileHandle.getPageId(dpid);
- if (pageId < pages.length) {
- // Common case: Return regular page.
- return pages[pageId];
- } else {
- // Rare case: Return overflow page, possibly expanding overflow array.
- synchronized (overflowPages) {
- int numNewPages = pageId - pages.length - overflowPages.size() + 1;
- if (numNewPages > 0) {
- ByteBuffer[] buffers = allocator.allocate(pageSize, numNewPages);
- for (int i = 0; i < numNewPages; i++) {
- CachedPage overflowPage = new CachedPage(pages.length + overflowPages.size(), buffers[i]);
- overflowPages.add(overflowPage);
- }
- }
- return overflowPages.get(pageId - pages.length);
- }
- }
- }
-
- @Override
- public ICachedPage tryPin(long dpid) throws HyracksDataException {
- return pin(dpid, false);
- }
-
- @Override
- public int getPageSize() {
- return pageSize;
- }
-
- @Override
- public int getNumPages() {
- return numPages;
- }
-
- @Override
- public ICachedPageInternal getPage(int cpid) {
- return pages[cpid];
- }
-
- public int getNumOverflowPages() {
- return overflowPages.size();
- }
-
- @Override
- public void createFile(FileReference fileRef) throws HyracksDataException {
- synchronized (fileMapManager) {
- fileMapManager.registerFile(fileRef);
- }
- }
-
- @Override
- public void openFile(int fileId) throws HyracksDataException {
- // Do nothing.
- }
-
- @Override
- public void closeFile(int fileId) throws HyracksDataException {
- // Do nothing.
- }
-
- @Override
- public void deleteFile(int fileId, boolean flushDirtyPages) throws HyracksDataException {
- synchronized (fileMapManager) {
- fileMapManager.unregisterFile(fileId);
- }
- }
-
- @Override
- public void unpin(ICachedPage page) throws HyracksDataException {
- // Do Nothing.
- }
-
- @Override
- public void close() {
- for (int i = 0; i < numPages; ++i) {
- pages[i] = null;
- }
- overflowPages.clear();
- }
-
- public class CachedPage implements ICachedPageInternal {
- private final int cpid;
- private final ByteBuffer buffer;
- private final ReadWriteLock latch;
-
- public CachedPage(int cpid, ByteBuffer buffer) {
- this.cpid = cpid;
- this.buffer = buffer;
- latch = new ReentrantReadWriteLock(true);
- }
-
- @Override
- public ByteBuffer getBuffer() {
- return buffer;
- }
-
- @Override
- public Object getReplacementStrategyObject() {
- // Do nothing.
- return null;
- }
-
- @Override
- public boolean pinIfGoodVictim() {
- // Do nothing.
- return false;
- }
-
- @Override
- public int getCachedPageId() {
- return cpid;
- }
-
- @Override
- public void acquireReadLatch() {
- latch.readLock().lock();
- }
-
- @Override
- public void acquireWriteLatch() {
- latch.writeLock().lock();
- }
-
- @Override
- public void releaseReadLatch() {
- latch.readLock().unlock();
- }
-
- @Override
- public void releaseWriteLatch() {
- latch.writeLock().unlock();
- }
- }
-
- @Override
- public void force(int fileId, boolean metadata) throws HyracksDataException {
- }
-
- @Override
- public void flushDirtyPage(ICachedPage page) throws HyracksDataException {
- }
-
- public IFileMapProvider getFileMapProvider() {
- return fileMapManager;
- }
-}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryFreePageManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java
similarity index 84%
rename from hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryFreePageManager.java
rename to hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java
index c601a9b..823656d 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryFreePageManager.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java
@@ -18,22 +18,20 @@
import java.util.concurrent.atomic.AtomicInteger;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IVirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-public class InMemoryFreePageManager implements IInMemoryFreePageManager {
+public class VirtualFreePageManager implements IVirtualFreePageManager {
protected final int capacity;
protected final AtomicInteger currentPageId = new AtomicInteger();
- protected final ITreeIndexMetaDataFrameFactory metaDataFrameFactory;
- public InMemoryFreePageManager(int capacity, ITreeIndexMetaDataFrameFactory metaDataFrameFactory) {
+ public VirtualFreePageManager(int capacity) {
// We start the currentPageId from 1, because the BTree uses
// the first page as metadata page, and the second page as root page.
// (when returning free pages we first increment, then get)
currentPageId.set(1);
this.capacity = capacity;
- this.metaDataFrameFactory = metaDataFrameFactory;
}
@Override
@@ -55,7 +53,7 @@
@Override
public ITreeIndexMetaDataFrameFactory getMetaDataFrameFactory() {
- return metaDataFrameFactory;
+ return NullMetadataFrameFactory.INSTANCE;
}
public int getCapacity() {
@@ -66,10 +64,6 @@
currentPageId.set(1);
}
- public boolean isFull() {
- return currentPageId.get() >= capacity;
- }
-
@Override
public void addFreePage(ITreeIndexMetaDataFrame metaFrame, int freePage) throws HyracksDataException {
}
@@ -109,4 +103,14 @@
public void close() {
// Method doesn't make sense for this free page manager.
}
+
+ private static class NullMetadataFrameFactory implements ITreeIndexMetaDataFrameFactory {
+ private static final NullMetadataFrameFactory INSTANCE = new NullMetadataFrameFactory();
+
+ @Override
+ public ITreeIndexMetaDataFrame createFrame() {
+ return null;
+ }
+
+ }
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractImmutableLSMComponent.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractImmutableLSMComponent.java
index b6fc2f7..053a8a4 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractImmutableLSMComponent.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractImmutableLSMComponent.java
@@ -26,6 +26,7 @@
}
switch (opType) {
+ case FORCE_MODIFICATION:
case MODIFICATION:
case SEARCH:
readerCount++;
@@ -50,6 +51,7 @@
if (failedOperation) {
state = ComponentState.READABLE;
}
+ case FORCE_MODIFICATION:
case MODIFICATION:
case SEARCH:
readerCount--;
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index 6782fbe..f081c37 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -21,7 +21,6 @@
import java.util.concurrent.atomic.AtomicReference;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
@@ -32,7 +31,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexInternal;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
@@ -45,7 +44,7 @@
protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
// In-memory components.
- protected final IInMemoryFreePageManager memFreePageManager;
+ protected final IVirtualBufferCache virtualBufferCache;
// On-disk components.
protected final IBufferCache diskBufferCache;
@@ -58,19 +57,17 @@
private boolean needsFlush = false;
- public AbstractLSMIndex(IInMemoryFreePageManager memFreePageManager, IBufferCache diskBufferCache,
+ public AbstractLSMIndex(IVirtualBufferCache virtualBufferCache, IBufferCache diskBufferCache,
ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider,
- double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- this.memFreePageManager = memFreePageManager;
+ double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ this.virtualBufferCache = virtualBufferCache;
this.diskBufferCache = diskBufferCache;
this.diskFileMapProvider = diskFileMapProvider;
this.fileManager = fileManager;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.ioScheduler = ioScheduler;
this.ioOpCallbackProvider = ioOpCallbackProvider;
- ILSMOperationTracker opTracker = opTrackerFactory.createOperationTracker(this);
lsmHarness = new LSMHarness(this, mergePolicy, opTracker);
isActivated = false;
componentsRef = new AtomicReference<List<ILSMComponent>>();
@@ -158,11 +155,6 @@
}
@Override
- public IInMemoryFreePageManager getInMemoryFreePageManager() {
- return memFreePageManager;
- }
-
- @Override
public List<ILSMComponent> getImmutableComponents() {
return componentsRef.get();
}
@@ -188,6 +180,11 @@
}
@Override
+ public boolean isFull() {
+ return virtualBufferCache.isFull();
+ }
+
+ @Override
public IBufferCache getBufferCache() {
return diskBufferCache;
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMutableLSMComponent.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMutableLSMComponent.java
index 5cde7bc..f881378 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMutableLSMComponent.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMutableLSMComponent.java
@@ -8,7 +8,7 @@
private int readerCount;
private int writerCount;
private ComponentState state;
-
+
private boolean isModified;
private enum ComponentState {
@@ -100,7 +100,6 @@
}
notifyAll();
}
-
public void setIsModified() {
isModified = true;
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 7372b86..e1db6e7 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -49,10 +49,10 @@
}
private void threadExit(ILSMIndexOperationContext opCtx, LSMOperationType opType) throws HyracksDataException {
- if (!lsmIndex.getFlushStatus() && lsmIndex.getInMemoryFreePageManager().isFull()) {
+ if (!lsmIndex.getFlushStatus() && lsmIndex.isFull()) {
lsmIndex.setFlushStatus(true);
}
- opTracker.afterOperation(opType, opCtx.getSearchOperationCallback(), opCtx.getModificationCallback());
+ opTracker.afterOperation(lsmIndex, opType, opCtx.getSearchOperationCallback(), opCtx.getModificationCallback());
}
private boolean getAndEnterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, boolean tryOperation)
@@ -93,7 +93,7 @@
}
}
- opTracker.beforeOperation(opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
+ opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
return true;
}
@@ -162,7 +162,7 @@
@Override
public void noOp(ILSMIndexOperationContext ctx) throws HyracksDataException {
LSMOperationType opType = LSMOperationType.NOOP;
- opTracker.beforeOperation(opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
+ opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
threadExit(ctx, opType);
}
@@ -191,7 +191,7 @@
LOGGER.info(lsmIndex + ": flushing");
}
ILSMComponent newComponent = lsmIndex.flush(operation);
-
+
operation.getCallback().afterOperation(null, newComponent);
lsmIndex.markAsValid(newComponent);
operation.getCallback().afterFinalize(newComponent);
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index 7f08ba4..1eff689 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -68,16 +68,22 @@
outputElement = null;
needPush = false;
- if (outputPriorityQueue != null) {
- outputPriorityQueue.clear();
- }
+ try {
+ if (outputPriorityQueue != null) {
+ outputPriorityQueue.clear();
+ }
- if (rangeCursors != null) {
- for (int i = 0; i < rangeCursors.length; i++) {
- rangeCursors[i].reset();
+ if (rangeCursors != null) {
+ for (int i = 0; i < rangeCursors.length; i++) {
+ rangeCursors[i].reset();
+ }
+ }
+ rangeCursors = null;
+ } finally {
+ if (lsmHarness != null) {
+ lsmHarness.endSearch(opCtx);
}
}
- rangeCursors = null;
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
new file mode 100644
index 0000000..4d9f8c2
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
@@ -0,0 +1,104 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
+
+public class MultitenantVirtualBufferCache implements IVirtualBufferCache {
+
+ private final IVirtualBufferCache vbc;
+ private int openCount;
+
+ public MultitenantVirtualBufferCache(IVirtualBufferCache virtualBufferCache) {
+ this.vbc = virtualBufferCache;
+ openCount = 0;
+ }
+
+ @Override
+ public void createFile(FileReference fileRef) throws HyracksDataException {
+ vbc.createFile(fileRef);
+ }
+
+ @Override
+ public void openFile(int fileId) throws HyracksDataException {
+ vbc.openFile(fileId);
+ }
+
+ @Override
+ public void closeFile(int fileId) throws HyracksDataException {
+ vbc.closeFile(fileId);
+ }
+
+ @Override
+ public void deleteFile(int fileId, boolean flushDirtyPages) throws HyracksDataException {
+ vbc.deleteFile(fileId, flushDirtyPages);
+ }
+
+ @Override
+ public ICachedPage tryPin(long dpid) throws HyracksDataException {
+ return vbc.tryPin(dpid);
+ }
+
+ @Override
+ public ICachedPage pin(long dpid, boolean newPage) throws HyracksDataException {
+ return vbc.pin(dpid, newPage);
+ }
+
+ @Override
+ public void unpin(ICachedPage page) throws HyracksDataException {
+ vbc.unpin(page);
+ }
+
+ @Override
+ public void flushDirtyPage(ICachedPage page) throws HyracksDataException {
+ vbc.flushDirtyPage(page);
+ }
+
+ @Override
+ public void force(int fileId, boolean metadata) throws HyracksDataException {
+ vbc.force(fileId, metadata);
+ }
+
+ @Override
+ public int getPageSize() {
+ return vbc.getPageSize();
+ }
+
+ @Override
+ public int getNumPages() {
+ return vbc.getNumPages();
+ }
+
+ @Override
+ public synchronized void close() throws HyracksDataException {
+ --openCount;
+ if (openCount == 0) {
+ vbc.close();
+ }
+ }
+
+ @Override
+ public synchronized void open() throws HyracksDataException {
+ ++openCount;
+ if (openCount == 1) {
+ vbc.open();
+ }
+ }
+
+ @Override
+ public boolean isFull() {
+ return vbc.isFull();
+ }
+
+ @Override
+ public void reset() {
+ vbc.reset();
+ }
+
+ @Override
+ public IFileMapManager getFileMapProvider() {
+ return vbc.getFileMapProvider();
+ }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
deleted file mode 100644
index 97ec50e..0000000
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
+++ /dev/null
@@ -1,48 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
-import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-
-/**
- * Operation tracker that does nothing.
- * WARNING: This op tracker should only be used for specific testing purposes.
- * It is assumed than an op tracker cooperates with an lsm index to synchronize flushes with
- * regular operations, and this implementation does no such tracking at all.
- */
-public class NoOpOperationTrackerFactory implements ILSMOperationTrackerFactory {
- private static final long serialVersionUID = 1L;
-
- public static NoOpOperationTrackerFactory INSTANCE = new NoOpOperationTrackerFactory();
-
- @Override
- public ILSMOperationTracker createOperationTracker(ILSMIndex index) {
- return new ILSMOperationTracker() {
-
- @Override
- public void completeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
- IModificationOperationCallback modificationCallback) throws HyracksDataException {
- // Do nothing.
- }
-
- @Override
- public void beforeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
- IModificationOperationCallback modificationCallback) throws HyracksDataException {
- }
-
- @Override
- public void afterOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
- IModificationOperationCallback modificationCallback) throws HyracksDataException {
- // Do nothing.
- }
- };
- }
-
- // Enforce singleton.
- private NoOpOperationTrackerFactory() {
- }
-
-};
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerProvider.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerProvider.java
new file mode 100644
index 0000000..9ed3872
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerProvider.java
@@ -0,0 +1,52 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+
+/**
+ * Operation tracker that does nothing.
+ * WARNING: This op tracker should only be used for specific testing purposes.
+ * It is assumed than an op tracker cooperates with an lsm index to synchronize flushes with
+ * regular operations, and this implementation does no such tracking at all.
+ */
+public class NoOpOperationTrackerProvider implements ILSMOperationTrackerProvider {
+ private static final long serialVersionUID = 1L;
+
+ public static NoOpOperationTrackerProvider INSTANCE = new NoOpOperationTrackerProvider();
+
+ @Override
+ public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
+ return new ILSMOperationTracker() {
+
+ @Override
+ public void completeOperation(ILSMIndex index, LSMOperationType opType,
+ ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
+ throws HyracksDataException {
+ // Do nothing.
+ }
+
+ @Override
+ public void beforeOperation(ILSMIndex index, LSMOperationType opType,
+ ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
+ throws HyracksDataException {
+ }
+
+ @Override
+ public void afterOperation(ILSMIndex index, LSMOperationType opType,
+ ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
+ throws HyracksDataException {
+ // Do nothing.
+ }
+ };
+ }
+
+ // Enforce singleton.
+ private NoOpOperationTrackerProvider() {
+ }
+
+};
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerFactory.java
deleted file mode 100644
index 3b4b00f..0000000
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerFactory.java
+++ /dev/null
@@ -1,21 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
-
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-
-public class ThreadCountingOperationTrackerFactory implements ILSMOperationTrackerFactory {
-
- private static final long serialVersionUID = 1L;
-
- public static ThreadCountingOperationTrackerFactory INSTANCE = new ThreadCountingOperationTrackerFactory();
-
- @Override
- public ILSMOperationTracker createOperationTracker(ILSMIndex index) {
- return new ThreadCountingTracker(index);
- }
-
- // Enforce singleton.
- private ThreadCountingOperationTrackerFactory() {
- }
-}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerProvider.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerProvider.java
new file mode 100644
index 0000000..67808d2
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerProvider.java
@@ -0,0 +1,21 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+
+public class ThreadCountingOperationTrackerProvider implements ILSMOperationTrackerProvider {
+
+ private static final long serialVersionUID = 1L;
+
+ public static ThreadCountingOperationTrackerProvider INSTANCE = new ThreadCountingOperationTrackerProvider();
+
+ @Override
+ public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
+ return new ThreadCountingTracker();
+ }
+
+ // Enforce singleton.
+ private ThreadCountingOperationTrackerProvider() {
+ }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
index 5dfa94e..a84ddd1 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
@@ -12,15 +12,13 @@
public class ThreadCountingTracker implements ILSMOperationTracker {
private final AtomicInteger threadRefCount;
- private final ILSMIndex index;
- public ThreadCountingTracker(ILSMIndex index) {
- this.index = index;
+ public ThreadCountingTracker() {
this.threadRefCount = new AtomicInteger();
}
@Override
- public void beforeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+ public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
if (opType == LSMOperationType.MODIFICATION) {
threadRefCount.incrementAndGet();
@@ -28,14 +26,14 @@
}
@Override
- public void afterOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+ public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
// The operation is considered inactive, immediately after leaving the index.
- completeOperation(opType, searchCallback, modificationCallback);
+ completeOperation(index, opType, searchCallback, modificationCallback);
}
@Override
- public void completeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+ public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
// Flush will only be handled by last exiting thread.
if (opType == LSMOperationType.MODIFICATION) {
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
new file mode 100644
index 0000000..8123a89
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
@@ -0,0 +1,330 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+
+public class VirtualBufferCache implements IVirtualBufferCache {
+ private static final int OVERFLOW_PADDING = 8;
+
+ private final ICacheMemoryAllocator allocator;
+ private final IFileMapManager fileMapManager;
+ private final int pageSize;
+ private final int numPages;
+
+ private final CacheBucket[] buckets;
+ private final ArrayList<VirtualPage> pages;
+
+ private int nextFree;
+
+ private boolean open;
+
+ public VirtualBufferCache(ICacheMemoryAllocator allocator, int pageSize, int numPages) {
+ this.allocator = allocator;
+ this.fileMapManager = new TransientFileMapManager();
+ this.pageSize = pageSize;
+ this.numPages = numPages;
+
+ buckets = new CacheBucket[numPages];
+ pages = new ArrayList<VirtualPage>();
+ nextFree = 0;
+ open = false;
+ }
+
+ @Override
+ public void createFile(FileReference fileRef) throws HyracksDataException {
+ synchronized (fileMapManager) {
+ if (fileMapManager.isMapped(fileRef)) {
+ throw new HyracksDataException("File " + fileRef + " is already mapped");
+ }
+ fileMapManager.registerFile(fileRef);
+ }
+ }
+
+ @Override
+ public void openFile(int fileId) throws HyracksDataException {
+ }
+
+ @Override
+ public void closeFile(int fileId) throws HyracksDataException {
+ }
+
+ @Override
+ public void deleteFile(int fileId, boolean flushDirtyPages) throws HyracksDataException {
+ synchronized (fileMapManager) {
+ if (!fileMapManager.isMapped(fileId)) {
+ throw new HyracksDataException("File with id " + fileId + " is not mapped");
+ }
+ fileMapManager.unregisterFile(fileId);
+ }
+
+ for (int i = 0; i < buckets.length; i++) {
+ CacheBucket bucket = buckets[i];
+ bucket.bucketLock.lock();
+ try {
+ VirtualPage prev = null;
+ VirtualPage curr = bucket.cachedPage;
+ while (curr != null) {
+ if (BufferedFileHandle.getFileId(curr.dpid) == fileId) {
+ if (prev == null) {
+ bucket.cachedPage = curr.next;
+ curr.reset();
+ curr = bucket.cachedPage;
+ } else {
+ prev.next = curr.next;
+ curr.reset();
+ curr = prev.next;
+ }
+ } else {
+ prev = curr;
+ curr = curr.next;
+ }
+ }
+ } finally {
+ bucket.bucketLock.unlock();
+ }
+ }
+ defragPageList();
+ }
+
+ private void defragPageList() {
+ synchronized (pages) {
+ int start = 0;
+ int end = nextFree - 1;
+ while (start < end) {
+ VirtualPage lastUsed = pages.get(end);
+ while (end > 0 && lastUsed.dpid == -1) {
+ --end;
+ lastUsed = pages.get(end);
+ }
+
+ if (end == 0) {
+ nextFree = lastUsed.dpid == -1 ? 0 : 1;
+ break;
+ }
+
+ VirtualPage firstUnused = pages.get(start);
+ while (start < end && firstUnused.dpid != -1) {
+ ++start;
+ firstUnused = pages.get(start);
+ }
+
+ if (start >= end) {
+ break;
+ }
+
+ Collections.swap(pages, start, end);
+ nextFree = end;
+ --end;
+ ++start;
+ }
+ }
+ }
+
+ @Override
+ public ICachedPage tryPin(long dpid) throws HyracksDataException {
+ return pin(dpid, false);
+ }
+
+ @Override
+ public ICachedPage pin(long dpid, boolean newPage) throws HyracksDataException {
+ VirtualPage page = null;
+ int hash = hash(dpid);
+ CacheBucket bucket = buckets[hash];
+
+ bucket.bucketLock.lock();
+ try {
+ page = bucket.cachedPage;
+ while (page != null) {
+ if (page.dpid == dpid) {
+ return page;
+ }
+ page = page.next;
+ }
+
+ if (!newPage) {
+ throw new HyracksDataException("Page " + BufferedFileHandle.getPageId(dpid)
+ + " does not exist in file "
+ + fileMapManager.lookupFileName(BufferedFileHandle.getFileId(dpid)));
+ }
+
+ page = getOrAllocPage(dpid);
+ page.next = bucket.cachedPage;
+ bucket.cachedPage = page;
+ } finally {
+ bucket.bucketLock.unlock();
+ }
+
+ return page;
+ }
+
+ private int hash(long dpid) {
+ return (int) (dpid % buckets.length);
+ }
+
+ private VirtualPage getOrAllocPage(long dpid) {
+ VirtualPage page = null;
+ synchronized (pages) {
+ if (nextFree >= pages.size()) {
+ page = new VirtualPage(allocator.allocate(pageSize, 1)[0]);
+ pages.add(page);
+ } else {
+ page = pages.get(nextFree);
+ }
+ ++nextFree;
+ page.dpid = dpid;
+ }
+ return page;
+ }
+
+ @Override
+ public void unpin(ICachedPage page) throws HyracksDataException {
+ }
+
+ @Override
+ public void flushDirtyPage(ICachedPage page) throws HyracksDataException {
+ }
+
+ @Override
+ public void force(int fileId, boolean metadata) throws HyracksDataException {
+ }
+
+ @Override
+ public int getPageSize() {
+ return pageSize;
+ }
+
+ @Override
+ public int getNumPages() {
+ return numPages;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (open) {
+ throw new HyracksDataException("Failed to open virtual buffercache since it is already open.");
+ }
+ pages.trimToSize();
+ pages.ensureCapacity(numPages + OVERFLOW_PADDING);
+ ByteBuffer[] buffers = allocator.allocate(pageSize, numPages);
+ for (int i = 0; i < numPages; i++) {
+ pages.add(new VirtualPage(buffers[i]));
+ buckets[i] = new CacheBucket();
+ }
+ nextFree = 0;
+ open = true;
+ }
+
+ @Override
+ public void reset() {
+ for (int i = 0; i < numPages; i++) {
+ pages.get(i).reset();
+ buckets[i].cachedPage = null;
+ }
+ int excess = pages.size() - numPages;
+ if (excess > 0) {
+ for (int i = numPages + excess - 1; i >= numPages; i--) {
+ pages.remove(i);
+ }
+ }
+ nextFree = 0;
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (!open) {
+ throw new HyracksDataException("Failed to close virtual buffercache since it is already closed.");
+ }
+
+ pages.clear();
+ for (int i = 0; i < numPages; i++) {
+ buckets[i].cachedPage = null;
+ }
+ open = false;
+ }
+
+ public String dumpState() {
+ StringBuilder sb = new StringBuilder();
+ sb.append(String.format("Page size = %d\n", pageSize));
+ sb.append(String.format("Capacity = %d\n", numPages));
+ sb.append(String.format("Allocated pages = %d\n", pages.size()));
+ sb.append(String.format("Next free page = %d\n", nextFree));
+ return sb.toString();
+ }
+
+ @Override
+ public IFileMapManager getFileMapProvider() {
+ return fileMapManager;
+ }
+
+ @Override
+ public boolean isFull() {
+ return nextFree >= numPages;
+ }
+
+ private static class CacheBucket {
+ private final ReentrantLock bucketLock;
+ private VirtualPage cachedPage;
+
+ public CacheBucket() {
+ this.bucketLock = new ReentrantLock();
+ }
+ }
+
+ private class VirtualPage implements ICachedPage {
+ final ByteBuffer buffer;
+ final ReadWriteLock latch;
+ volatile long dpid;
+ VirtualPage next;
+
+ public VirtualPage(ByteBuffer buffer) {
+ this.buffer = buffer;
+ latch = new ReentrantReadWriteLock(true);
+ dpid = -1;
+ next = null;
+ }
+
+ public void reset() {
+ dpid = -1;
+ next = null;
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return buffer;
+ }
+
+ @Override
+ public void acquireReadLatch() {
+ latch.readLock().lock();
+ }
+
+ @Override
+ public void releaseReadLatch() {
+ latch.readLock().unlock();
+ }
+
+ @Override
+ public void acquireWriteLatch() {
+ latch.writeLock().lock();
+ }
+
+ @Override
+ public void releaseWriteLatch() {
+ latch.writeLock().unlock();
+ }
+
+ }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
index c97e10d..9315d06 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
@@ -16,62 +16,51 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public final class LSMInvertedIndexDataflowHelper extends AbstractLSMIndexDataflowHelper {
public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
- DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
- ioOpCallbackProvider);
+ IVirtualBufferCache virtualBufferCache, ILSMMergePolicy mergePolicy,
+ ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
+ opTrackerFactory, ioScheduler, ioOpCallbackProvider);
}
public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+ ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ super(opDesc, ctx, partition, virtualBufferCache, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+ ioScheduler, ioOpCallbackProvider);
}
@Override
public IIndex createIndexInstance() throws HyracksDataException {
IInvertedIndexOperatorDescriptor invIndexOpDesc = (IInvertedIndexOperatorDescriptor) opDesc;
try {
- ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
- IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(),
- memPageSize, memNumPages);
- IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
- metaDataFrameFactory);
IBufferCache diskBufferCache = opDesc.getStorageManager().getBufferCache(ctx);
IFileMapProvider diskFileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
- LSMInvertedIndex invIndex = InvertedIndexUtils.createLSMInvertedIndex(memBufferCache, memFreePageManager,
- diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(), invIndexOpDesc
- .getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(), invIndexOpDesc
- .getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(), diskBufferCache, ctx
- .getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+ LSMInvertedIndex invIndex = InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCache,
+ diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
+ invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
+ invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
+ diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate,
+ mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider,
opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
return invIndex;
} catch (IndexException e) {
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
index 60a0b47..6a8da23 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
@@ -21,27 +21,29 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
public class LSMInvertedIndexDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
private static final long serialVersionUID = 1L;
- public LSMInvertedIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
- ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+ public LSMInvertedIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
+ ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerProvider,
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
double bloomFilterFalsePositiveRate) {
- super(mergePolicyProvider, opTrackerProvider, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
- memNumPages, bloomFilterFalsePositiveRate);
+ super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerProvider, ioSchedulerProvider,
+ ioOpCallbackProvider, bloomFilterFalsePositiveRate);
}
@Override
public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
- return new LSMInvertedIndexDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
- bloomFilterFalsePositiveRate, mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory,
- ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider);
+ return new LSMInvertedIndexDataflowHelper(opDesc, ctx, partition,
+ virtualBufferCacheProvider.getVirtualBufferCache(ctx), bloomFilterFalsePositiveRate,
+ mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
+ ioOpCallbackProvider);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
index 3122aa0..768d4a8 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
@@ -38,8 +38,8 @@
private static final long serialVersionUID = 1L;
- private final int[] fieldPermutation;
- private final IndexOperation op;
+ protected final int[] fieldPermutation;
+ protected final IndexOperation op;
public LSMInvertedIndexInsertUpdateDeleteOperator(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
index 6c5b25f..c8caa48 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
@@ -16,62 +16,51 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.PartitionedLSMInvertedIndex;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public final class PartitionedLSMInvertedIndexDataflowHelper extends AbstractLSMIndexDataflowHelper {
public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
- int partition, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
- DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
- ioOpCallbackProvider);
+ int partition, IVirtualBufferCache virtualBufferCache, ILSMMergePolicy mergePolicy,
+ ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
+ opTrackerFactory, ioScheduler, ioOpCallbackProvider);
}
public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
- int partition, int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+ int partition, IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ super(opDesc, ctx, partition, virtualBufferCache, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+ ioScheduler, ioOpCallbackProvider);
}
@Override
public IIndex createIndexInstance() throws HyracksDataException {
IInvertedIndexOperatorDescriptor invIndexOpDesc = (IInvertedIndexOperatorDescriptor) opDesc;
try {
- ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
- IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(),
- memPageSize, memNumPages);
- IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
- metaDataFrameFactory);
IBufferCache diskBufferCache = opDesc.getStorageManager().getBufferCache(ctx);
IFileMapProvider diskFileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
- PartitionedLSMInvertedIndex invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(memBufferCache,
- memFreePageManager, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(), invIndexOpDesc
- .getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(), invIndexOpDesc
- .getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(), diskBufferCache, ctx
- .getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+ PartitionedLSMInvertedIndex invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(
+ virtualBufferCache, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
+ invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
+ invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
+ diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate,
+ mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider,
opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
return invIndex;
} catch (IndexException e) {
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
index 801462b..c1b55d0 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
@@ -21,26 +21,28 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
public class PartitionedLSMInvertedIndexDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
private static final long serialVersionUID = 1L;
- public PartitionedLSMInvertedIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
- ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+ public PartitionedLSMInvertedIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
+ ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerProvider,
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
double bloomFilterFalsePositiveRate) {
- super(mergePolicyProvider, opTrackerProvider, ioSchedulerProvider, ioOpCallbackProvider, memNumPages,
- memNumPages, bloomFilterFalsePositiveRate);
+ super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerProvider, ioSchedulerProvider,
+ ioOpCallbackProvider, bloomFilterFalsePositiveRate);
}
@Override
public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
- return new PartitionedLSMInvertedIndexDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
- bloomFilterFalsePositiveRate, mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory,
- ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider);
+ return new PartitionedLSMInvertedIndexDataflowHelper(opDesc, ctx, partition,
+ virtualBufferCacheProvider.getVirtualBufferCache(ctx), bloomFilterFalsePositiveRate,
+ mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
+ ioOpCallbackProvider);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index beb4dc9..9343a71 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -34,7 +34,6 @@
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
@@ -42,13 +41,13 @@
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IVirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
@@ -60,8 +59,9 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
@@ -82,7 +82,7 @@
// In-memory components.
protected final LSMInvertedIndexMutableComponent mutableComponent;
- protected final IInMemoryFreePageManager memFreePageManager;
+ protected final IVirtualFreePageManager virtualFreePageManager;
protected final IBinaryTokenizerFactory tokenizerFactory;
// On-disk components.
@@ -95,29 +95,29 @@
protected final ITypeTraits[] tokenTypeTraits;
protected final IBinaryComparatorFactory[] tokenCmpFactories;
- public LSMInvertedIndex(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
- OnDiskInvertedIndexFactory diskInvIndexFactory, BTreeFactory deletedKeysBTreeFactory,
- BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
- ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+ public LSMInvertedIndex(IVirtualBufferCache virtualBufferCache, OnDiskInvertedIndexFactory diskInvIndexFactory,
+ BTreeFactory deletedKeysBTreeFactory, BloomFilterFactory bloomFilterFactory,
+ double bloomFilterFalsePositiveRate, ILSMIndexFileManager fileManager,
+ IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
- throws IndexException {
- super(memFreePageManager, diskInvIndexFactory.getBufferCache(), fileManager, diskFileMapProvider,
- bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
- this.memFreePageManager = memFreePageManager;
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+ super(virtualBufferCache, diskInvIndexFactory.getBufferCache(), fileManager, diskFileMapProvider,
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ this.virtualFreePageManager = new VirtualFreePageManager(virtualBufferCache.getNumPages());
this.tokenizerFactory = tokenizerFactory;
this.invListTypeTraits = invListTypeTraits;
this.invListCmpFactories = invListCmpFactories;
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
// Create in-memory component.
- InMemoryInvertedIndex memInvIndex = createInMemoryInvertedIndex(memBufferCache);
- BTree deleteKeysBTree = BTreeUtils.createBTree(memBufferCache, memFreePageManager,
- ((InMemoryBufferCache) memBufferCache).getFileMapProvider(), invListTypeTraits, invListCmpFactories,
- BTreeLeafFrameType.REGULAR_NSM, new FileReference(new File("membtree")));
- mutableComponent = new LSMInvertedIndexMutableComponent(memInvIndex, deleteKeysBTree, memFreePageManager);
+ InMemoryInvertedIndex memInvIndex = createInMemoryInvertedIndex(virtualBufferCache);
+ BTree deleteKeysBTree = BTreeUtils.createBTree(virtualBufferCache, new VirtualFreePageManager(
+ virtualBufferCache.getNumPages()), ((IVirtualBufferCache) virtualBufferCache).getFileMapProvider(),
+ invListTypeTraits, invListCmpFactories, BTreeLeafFrameType.REGULAR_NSM, new FileReference(new File(
+ fileManager.getBaseDir() + "_virtual_del")));
+ mutableComponent = new LSMInvertedIndexMutableComponent(memInvIndex, deleteKeysBTree, virtualBufferCache);
componentFactory = new LSMInvertedIndexComponentFactory(diskInvIndexFactory, deletedKeysBTreeFactory,
bloomFilterFactory);
}
@@ -136,11 +136,12 @@
@Override
public synchronized void activate() throws HyracksDataException {
if (isActivated) {
- return;
+ throw new HyracksDataException("Failed to activate the index since it is already activated.");
}
+
try {
List<ILSMComponent> immutableComponents = componentsRef.get();
- ((InMemoryBufferCache) mutableComponent.getInvIndex().getBufferCache()).open();
+ ((IVirtualBufferCache) mutableComponent.getInvIndex().getBufferCache()).open();
mutableComponent.getInvIndex().create();
mutableComponent.getInvIndex().activate();
mutableComponent.getDeletedKeysBTree().create();
@@ -189,11 +190,10 @@
@Override
public synchronized void deactivate(boolean flushOnExit) throws HyracksDataException {
if (!isActivated) {
- return;
+ throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
}
isActivated = false;
-
if (flushOnExit) {
BlockingIOOperationCallbackWrapper blockingCallBack = new BlockingIOOperationCallbackWrapper(
ioOpCallbackProvider.getIOOperationCallback(this));
@@ -218,7 +218,7 @@
mutableComponent.getDeletedKeysBTree().deactivate();
mutableComponent.getInvIndex().destroy();
mutableComponent.getDeletedKeysBTree().destroy();
- ((InMemoryBufferCache) mutableComponent.getInvIndex().getBufferCache()).close();
+ ((IVirtualBufferCache) mutableComponent.getInvIndex().getBufferCache()).close();
}
@Override
@@ -619,10 +619,11 @@
}
}
- protected InMemoryInvertedIndex createInMemoryInvertedIndex(IInMemoryBufferCache memBufferCache)
+ protected InMemoryInvertedIndex createInMemoryInvertedIndex(IVirtualBufferCache virtualBufferCache)
throws IndexException {
- return InvertedIndexUtils.createInMemoryBTreeInvertedindex(memBufferCache, memFreePageManager,
- invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
+ return InvertedIndexUtils.createInMemoryBTreeInvertedindex(virtualBufferCache, virtualFreePageManager,
+ invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+ new FileReference(new File(fileManager.getBaseDir() + "_virtual_vocab")));
}
protected LSMInvertedIndexImmutableComponent createDiskInvIndexComponent(ILSMComponentFactory factory,
@@ -679,8 +680,8 @@
@Override
public long getMemoryAllocationSize() {
- InMemoryBufferCache memBufferCache = (InMemoryBufferCache) mutableComponent.getInvIndex().getBufferCache();
- return memBufferCache.getNumPages() * memBufferCache.getPageSize();
+ IBufferCache virtualBufferCache = mutableComponent.getInvIndex().getBufferCache();
+ return virtualBufferCache.getNumPages() * virtualBufferCache.getPageSize();
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMutableComponent.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMutableComponent.java
index 615f1ee..a2bf110 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMutableComponent.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMutableComponent.java
@@ -17,7 +17,7 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMutableLSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
@@ -25,13 +25,12 @@
private final IInvertedIndex invIndex;
private final BTree deletedKeysBTree;
- private final IInMemoryFreePageManager mfpm;
+ private final IVirtualBufferCache vbc;
- public LSMInvertedIndexMutableComponent(IInvertedIndex invIndex, BTree deletedKeysBTree,
- IInMemoryFreePageManager mfpm) {
+ public LSMInvertedIndexMutableComponent(IInvertedIndex invIndex, BTree deletedKeysBTree, IVirtualBufferCache vbc) {
this.invIndex = invIndex;
this.deletedKeysBTree = deletedKeysBTree;
- this.mfpm = mfpm;
+ this.vbc = vbc;
}
public IInvertedIndex getInvIndex() {
@@ -44,13 +43,19 @@
@Override
protected boolean isFull() {
- return mfpm.isFull();
+ return vbc.isFull();
}
@Override
protected void reset() throws HyracksDataException {
super.reset();
- invIndex.clear();
- deletedKeysBTree.clear();
+ invIndex.deactivate();
+ invIndex.destroy();
+ invIndex.create();
+ invIndex.activate();
+ deletedKeysBTree.deactivate();
+ deletedKeysBTree.destroy();
+ deletedKeysBTree.create();
+ deletedKeysBTree.activate();
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
index 36ad51b..260ddc2 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -165,18 +165,29 @@
@Override
public void close() throws HyracksDataException {
- reset();
- accessorIndex = -1;
- harness.endSearch(opCtx);
+ try {
+ reset();
+ accessorIndex = -1;
+ } finally {
+ if (harness != null) {
+ harness.endSearch(opCtx);
+ }
+ }
}
@Override
public void reset() throws HyracksDataException {
- if (currentCursor != null) {
- currentCursor.close();
- currentCursor = null;
+ try {
+ if (currentCursor != null) {
+ currentCursor.close();
+ currentCursor = null;
+ }
+ accessorIndex = 0;
+ } finally {
+ if (harness != null) {
+ harness.endSearch(opCtx);
+ }
}
- accessorIndex = 0;
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index a45f729..2c579ab 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -15,17 +15,19 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls;
+import java.io.File;
+
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
@@ -35,26 +37,25 @@
public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
- public PartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, OnDiskInvertedIndexFactory diskInvIndexFactory,
- BTreeFactory deletedKeysBTreeFactory, BloomFilterFactory bloomFilterFactory,
- double bloomFilterFalsePositiveRate, ILSMIndexFileManager fileManager,
- IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+ public PartitionedLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
+ OnDiskInvertedIndexFactory diskInvIndexFactory, BTreeFactory deletedKeysBTreeFactory,
+ BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
+ ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
- throws IndexException {
- super(memBufferCache, memFreePageManager, diskInvIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+ super(virtualBufferCache, diskInvIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory,
bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler,
+ tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler,
ioOpCallbackProvider);
}
- protected InMemoryInvertedIndex createInMemoryInvertedIndex(IInMemoryBufferCache memBufferCache)
+ protected InMemoryInvertedIndex createInMemoryInvertedIndex(IVirtualBufferCache virtualBufferCache)
throws IndexException {
- return InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(memBufferCache, memFreePageManager,
- invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
+ return InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(virtualBufferCache,
+ virtualFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+ tokenizerFactory, new FileReference(new File(fileManager.getBaseDir() + "_virtual_vocab")));
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index d5a074e..8728240 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -14,8 +14,6 @@
*/
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory;
-import java.io.File;
-
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -36,7 +34,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
@@ -45,7 +43,6 @@
public class InMemoryInvertedIndex implements IInvertedIndex {
protected final BTree btree;
- protected final FileReference memBTreeFile = new FileReference(new File("memBTree"));
protected final ITypeTraits[] tokenTypeTraits;
protected final IBinaryComparatorFactory[] tokenCmpFactories;
protected final ITypeTraits[] invListTypeTraits;
@@ -55,10 +52,10 @@
protected final ITypeTraits[] btreeTypeTraits;
protected final IBinaryComparatorFactory[] btreeCmpFactories;
- public InMemoryInvertedIndex(IBufferCache memBufferCache, IFreePageManager memFreePageManager,
+ public InMemoryInvertedIndex(IBufferCache virtualBufferCache, IFreePageManager virtualFreePageManager,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory) throws BTreeException {
+ IBinaryTokenizerFactory tokenizerFactory, FileReference btreeFileRef) throws BTreeException {
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.invListTypeTraits = invListTypeTraits;
@@ -76,9 +73,9 @@
btreeTypeTraits[tokenTypeTraits.length + i] = invListTypeTraits[i];
btreeCmpFactories[tokenTypeTraits.length + i] = invListCmpFactories[i];
}
- this.btree = BTreeUtils.createBTree(memBufferCache, memFreePageManager,
- ((InMemoryBufferCache) memBufferCache).getFileMapProvider(), btreeTypeTraits, btreeCmpFactories,
- BTreeLeafFrameType.REGULAR_NSM, memBTreeFile);
+ this.btree = BTreeUtils.createBTree(virtualBufferCache, virtualFreePageManager,
+ ((IVirtualBufferCache) virtualBufferCache).getFileMapProvider(), btreeTypeTraits, btreeCmpFactories,
+ BTreeLeafFrameType.REGULAR_NSM, btreeFileRef);
}
@Override
@@ -145,8 +142,8 @@
@Override
public long getMemoryAllocationSize() {
- InMemoryBufferCache memBufferCache = (InMemoryBufferCache) btree.getBufferCache();
- return memBufferCache.getNumPages() * memBufferCache.getPageSize();
+ IBufferCache virtualBufferCache = btree.getBufferCache();
+ return virtualBufferCache.getNumPages() * virtualBufferCache.getPageSize();
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
index 7c3f4e4..b57e976 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -20,6 +20,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
@@ -48,9 +49,9 @@
public PartitionedInMemoryInvertedIndex(IBufferCache memBufferCache, IFreePageManager memFreePageManager,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory) throws BTreeException {
+ IBinaryTokenizerFactory tokenizerFactory, FileReference btreeFileRef) throws BTreeException {
super(memBufferCache, memFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory);
+ tokenCmpFactories, tokenizerFactory, btreeFileRef);
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index afeaf90..5f9f87c 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -161,7 +161,7 @@
@Override
public synchronized void activate() throws HyracksDataException {
if (isOpen) {
- return;
+ throw new HyracksDataException("Failed to activate the index since it is already activated.");
}
btree.activate();
@@ -190,7 +190,7 @@
@Override
public synchronized void deactivate() throws HyracksDataException {
if (!isOpen) {
- return;
+ throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
}
btree.deactivate();
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index 06be7aa..63d44ac 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -27,18 +27,17 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManagerFactory;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
@@ -60,21 +59,21 @@
public class InvertedIndexUtils {
public static InMemoryInvertedIndex createInMemoryBTreeInvertedindex(IBufferCache memBufferCache,
- IFreePageManager memFreePageManager, ITypeTraits[] invListTypeTraits,
+ IFreePageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
- IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory)
- throws BTreeException {
- return new InMemoryInvertedIndex(memBufferCache, memFreePageManager, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
+ IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+ FileReference btreeFileRef) throws BTreeException {
+ return new InMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits,
+ invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, btreeFileRef);
}
public static InMemoryInvertedIndex createPartitionedInMemoryBTreeInvertedindex(IBufferCache memBufferCache,
- IFreePageManager memFreePageManager, ITypeTraits[] invListTypeTraits,
+ IFreePageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
- IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory)
- throws BTreeException {
- return new PartitionedInMemoryInvertedIndex(memBufferCache, memFreePageManager, invListTypeTraits,
- invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
+ IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+ FileReference btreeFileRef) throws BTreeException {
+ return new PartitionedInMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits,
+ invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, btreeFileRef);
}
public static OnDiskInvertedIndex createOnDiskInvertedIndex(IBufferCache bufferCache,
@@ -117,13 +116,24 @@
return deletedKeysBTreeFactory;
}
- public static LSMInvertedIndex createLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
- ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
- ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
- String onDiskDir, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ public static LSMInvertedIndex createLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
+ IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+ IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+ IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+ IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+ return createLSMInvertedIndex(virtualBufferCache, diskFileMapProvider, invListTypeTraits, invListCmpFactories,
+ tokenTypeTraits, tokenCmpFactories, tokenizerFactory, diskBufferCache, ioManager, onDiskDir,
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, 0);
+ }
+
+ public static LSMInvertedIndex createLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
+ IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+ IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+ IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+ IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws IndexException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
@@ -146,20 +156,31 @@
diskFileMapProvider, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
tokenCmpFactories, fileManager);
- LSMInvertedIndex invIndex = new LSMInvertedIndex(memBufferCache, memFreePageManager, invIndexFactory,
- deletedKeysBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager,
- diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
- tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ LSMInvertedIndex invIndex = new LSMInvertedIndex(virtualBufferCache, invIndexFactory, deletedKeysBTreeFactory,
+ bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits,
+ invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
+ ioScheduler, ioOpCallbackProvider);
return invIndex;
}
- public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
- ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
- ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
- String onDiskDir, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
+ IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+ IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+ IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+ IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+ return createPartitionedLSMInvertedIndex(virtualBufferCache, diskFileMapProvider, invListTypeTraits,
+ invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, diskBufferCache, ioManager,
+ onDiskDir, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, 0);
+ }
+
+ public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
+ IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+ IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+ IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+ IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws IndexException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
@@ -182,10 +203,10 @@
diskBufferCache, diskFileMapProvider, invListBuilderFactory, invListTypeTraits, invListCmpFactories,
tokenTypeTraits, tokenCmpFactories, fileManager);
- PartitionedLSMInvertedIndex invIndex = new PartitionedLSMInvertedIndex(memBufferCache, memFreePageManager,
- invIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate,
- fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ PartitionedLSMInvertedIndex invIndex = new PartitionedLSMInvertedIndex(virtualBufferCache, invIndexFactory,
+ deletedKeysBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager,
+ diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+ tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
return invIndex;
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
index 5fcdcea..0385527 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
@@ -22,23 +22,18 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -50,25 +45,25 @@
protected final ILinearizeComparatorFactory linearizeCmpFactory;
public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IBinaryComparatorFactory[] btreeComparatorFactories,
+ IVirtualBufferCache virtualBufferCache, IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+ ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
ILinearizeComparatorFactory linearizeCmpFactory) {
- this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
- DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, btreeComparatorFactories, valueProviderFactories,
- rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+ this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE,
+ btreeComparatorFactories, valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory,
+ ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
}
public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate,
+ IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate,
IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+ ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
ILinearizeComparatorFactory linearizeCmpFactory) {
- super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ super(opDesc, ctx, partition, virtualBufferCache, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+ ioScheduler, ioOpCallbackProvider);
this.btreeComparatorFactories = btreeComparatorFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
@@ -78,22 +73,18 @@
@Override
public ITreeIndex createIndexInstance() throws HyracksDataException {
AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
- ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
- IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
- memNumPages);
- IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
- metaDataFrameFactory);
- return createLSMTree(memBufferCache, memFreePageManager, ctx.getIOManager(), file, opDesc.getStorageManager()
- .getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx),
+ return createLSMTree(virtualBufferCache, ctx.getIOManager(), file,
+ opDesc.getStorageManager().getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx),
treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(),
- btreeComparatorFactories, valueProviderFactories, rtreePolicyType, linearizeCmpFactory, partition);
+ btreeComparatorFactories, opTrackerFactory.getOperationTracker(ctx), valueProviderFactories,
+ rtreePolicyType, linearizeCmpFactory, partition);
}
- protected abstract ITreeIndex createLSMTree(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
- IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
- IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+ protected abstract ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
+ FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+ ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+ IBinaryComparatorFactory[] btreeCmpFactories, ILSMOperationTracker opTracker,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException;
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
index 5ab5513..80a56c6 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
@@ -22,16 +22,16 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -40,38 +40,38 @@
public class LSMRTreeDataflowHelper extends AbstractLSMRTreeDataflowHelper {
public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IBinaryComparatorFactory[] btreeComparatorFactories,
+ IVirtualBufferCache virtualBufferCache, IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+ ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
ILinearizeComparatorFactory linearizeCmpFactory) {
- super(opDesc, ctx, partition, btreeComparatorFactories, valueProviderFactories, rtreePolicyType, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+ super(opDesc, ctx, partition, virtualBufferCache, btreeComparatorFactories, valueProviderFactories,
+ rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
}
public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate,
+ IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate,
IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+ ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
ILinearizeComparatorFactory linearizeCmpFactory) {
- super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, btreeComparatorFactories,
+ super(opDesc, ctx, partition, virtualBufferCache, bloomFilterFalsePositiveRate, btreeComparatorFactories,
valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
ioOpCallbackProvider, linearizeCmpFactory);
}
@Override
- protected ITreeIndex createLSMTree(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
- IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
- IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+ protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
+ FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+ ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+ IBinaryComparatorFactory[] btreeCmpFactories, ILSMOperationTracker opTracker,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException {
try {
- return LSMRTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache,
+ return LSMRTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
- rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
+ rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
ioOpCallbackProvider, linearizeCmpFactory,
opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
} catch (TreeIndexException e) {
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
index 0778e15..befbc3c 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
@@ -24,7 +24,8 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -39,12 +40,12 @@
public LSMRTreeDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
- ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
- ILinearizeComparatorFactory linearizeCmpFactory, int memPageSize, int memNumPages,
+ IVirtualBufferCacheProvider virtualBufferCacheProvider, ILSMMergePolicyProvider mergePolicyProvider,
+ ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory,
double bloomFilterFalsePositiveRate) {
- super(mergePolicyProvider, opTrackerFactory, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
- memNumPages, bloomFilterFalsePositiveRate);
+ super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerFactory, ioSchedulerProvider,
+ ioOpCallbackProvider, bloomFilterFalsePositiveRate);
this.btreeComparatorFactories = btreeComparatorFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
@@ -54,8 +55,9 @@
@Override
public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
- return new LSMRTreeDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
- bloomFilterFalsePositiveRate, btreeComparatorFactories, valueProviderFactories, rtreePolicyType,
+ return new LSMRTreeDataflowHelper(opDesc, ctx, partition,
+ virtualBufferCacheProvider.getVirtualBufferCache(ctx), bloomFilterFalsePositiveRate,
+ btreeComparatorFactories, valueProviderFactories, rtreePolicyType,
mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
ioOpCallbackProvider, linearizeCmpFactory);
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
index d4fac8f..0a309ff 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
@@ -22,16 +22,16 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -39,37 +39,27 @@
public class LSMRTreeWithAntiMatterTuplesDataflowHelper extends AbstractLSMRTreeDataflowHelper {
public LSMRTreeWithAntiMatterTuplesDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
- int partition, IBinaryComparatorFactory[] btreeComparatorFactories,
+ int partition, IVirtualBufferCache virtualBufferCache, IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+ ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
ILinearizeComparatorFactory linearizeCmpFactory) {
- super(opDesc, ctx, partition, btreeComparatorFactories, valueProviderFactories, rtreePolicyType, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
- }
-
- public LSMRTreeWithAntiMatterTuplesDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
- int partition, int memPageSize, int memNumPages, IBinaryComparatorFactory[] btreeComparatorFactories,
- IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
- ILinearizeComparatorFactory linearizeCmpFactory) {
- super(opDesc, ctx, partition, memPageSize, memNumPages, 0, btreeComparatorFactories, valueProviderFactories,
+ super(opDesc, ctx, partition, virtualBufferCache, btreeComparatorFactories, valueProviderFactories,
rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
}
@Override
- protected ITreeIndex createLSMTree(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
- IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
- IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+ protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
+ FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+ ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+ IBinaryComparatorFactory[] btreeCmpFactories, ILSMOperationTracker opTracker,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException {
try {
- return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(memBufferCache, memFreePageManager, ioManager, file,
+ return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCache, ioManager, file,
diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
- valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
- ioOpCallbackProvider, linearizeCmpFactory, startIODeviceIndex);
+ valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+ linearizeCmpFactory, startIODeviceIndex);
} catch (TreeIndexException e) {
throw new HyracksDataException(e);
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
index b27e84f..7456c8c 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
@@ -25,27 +25,30 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
public class LSMRTreeWithAntiMatterTuplesDataflowHelperFactory implements IIndexDataflowHelperFactory {
private static final long serialVersionUID = 1L;
+ private final IVirtualBufferCacheProvider virtualBufferCacheProvider;
private final IBinaryComparatorFactory[] btreeComparatorFactories;
private final IPrimitiveValueProviderFactory[] valueProviderFactories;
private final RTreePolicyType rtreePolicyType;
private final ILSMMergePolicyProvider mergePolicyProvider;
- private final ILSMOperationTrackerFactory opTrackerProvider;
+ private final ILSMOperationTrackerProvider opTrackerProvider;
private final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
private final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
private final ILinearizeComparatorFactory linearizeCmpFactory;
public LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
- ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerFactory opTrackerProvider,
- ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
- ILinearizeComparatorFactory linearizeCmpFactory) {
+ IVirtualBufferCacheProvider virtualBufferCacheProvider, ILSMMergePolicyProvider mergePolicyProvider,
+ ILSMOperationTrackerProvider opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory) {
+ this.virtualBufferCacheProvider = virtualBufferCacheProvider;
this.btreeComparatorFactories = btreeComparatorFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
@@ -59,7 +62,8 @@
@Override
public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
- return new LSMRTreeWithAntiMatterTuplesDataflowHelper(opDesc, ctx, partition, btreeComparatorFactories,
+ return new LSMRTreeWithAntiMatterTuplesDataflowHelper(opDesc, ctx, partition,
+ virtualBufferCacheProvider.getVirtualBufferCache(ctx), btreeComparatorFactories,
valueProviderFactories, rtreePolicyType, mergePolicyProvider.getMergePolicy(ctx), opTrackerProvider,
ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider, linearizeCmpFactory);
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index f62fab1..850bc7c 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -29,16 +29,15 @@
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IVirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
@@ -47,8 +46,9 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
@@ -57,6 +57,7 @@
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree.RTreeAccessor;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITreeIndex {
@@ -67,7 +68,8 @@
// In-memory components.
protected final LSMRTreeMutableComponent mutableComponent;
- protected final IInMemoryBufferCache memBufferCache;
+ protected final IVirtualBufferCache virtualBufferCache;
+ protected final IVirtualFreePageManager virtualFreePageManager;
protected TreeTupleSorter rTreeTupleSorter;
@@ -84,26 +86,27 @@
protected final ITreeIndexFrameFactory rtreeLeafFrameFactory;
protected final ITreeIndexFrameFactory btreeLeafFrameFactory;
- public AbstractLSMRTree(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
- ITreeIndexFrameFactory rtreeInteriorFrameFactory, ITreeIndexFrameFactory rtreeLeafFrameFactory,
- ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
- ILSMIndexFileManager fileManager, TreeIndexFactory<RTree> diskRTreeFactory,
- ILSMComponentFactory componentFactory, IFileMapProvider diskFileMapProvider, int fieldCount,
- IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
- ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
- double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ public AbstractLSMRTree(IVirtualBufferCache virtualBufferCache, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
+ ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
+ ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileManager,
+ TreeIndexFactory<RTree> diskRTreeFactory, ILSMComponentFactory componentFactory,
+ IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] rtreeCmpFactories,
+ IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
+ int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- super(memFreePageManager, diskRTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
- bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
- RTree memRTree = new RTree(memBufferCache, ((InMemoryBufferCache) memBufferCache).getFileMapProvider(),
- memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, rtreeCmpFactories, fieldCount,
- new FileReference(new File("memrtree")));
- BTree memBTree = new BTree(memBufferCache, ((InMemoryBufferCache) memBufferCache).getFileMapProvider(),
- memFreePageManager, btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeCmpFactories, fieldCount,
- new FileReference(new File("membtree")));
- mutableComponent = new LSMRTreeMutableComponent(memRTree, memBTree, memFreePageManager);
- this.memBufferCache = memBufferCache;
+ super(virtualBufferCache, diskRTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ virtualFreePageManager = new VirtualFreePageManager(virtualBufferCache.getNumPages());
+ RTree memRTree = new RTree(virtualBufferCache, ((IVirtualBufferCache) virtualBufferCache).getFileMapProvider(),
+ virtualFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, rtreeCmpFactories,
+ fieldCount, new FileReference(new File(fileManager.getBaseDir() + "_virtual_r")));
+ BTree memBTree = new BTree(virtualBufferCache, ((IVirtualBufferCache) virtualBufferCache).getFileMapProvider(),
+ new VirtualFreePageManager(virtualBufferCache.getNumPages()), btreeInteriorFrameFactory,
+ btreeLeafFrameFactory, btreeCmpFactories, fieldCount, new FileReference(new File(
+ fileManager.getBaseDir() + "_virtual_b")));
+ mutableComponent = new LSMRTreeMutableComponent(memRTree, memBTree, virtualBufferCache);
+ this.virtualBufferCache = virtualBufferCache;
this.rtreeInteriorFrameFactory = rtreeInteriorFrameFactory;
this.rtreeLeafFrameFactory = rtreeLeafFrameFactory;
this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
@@ -131,10 +134,10 @@
@Override
public synchronized void activate() throws HyracksDataException {
if (isActivated) {
- return;
+ throw new HyracksDataException("Failed to activate the index since it is already activated.");
}
- ((InMemoryBufferCache) mutableComponent.getRTree().getBufferCache()).open();
+ ((IVirtualBufferCache) mutableComponent.getRTree().getBufferCache()).open();
mutableComponent.getRTree().create();
mutableComponent.getBTree().create();
mutableComponent.getRTree().activate();
@@ -144,7 +147,7 @@
@Override
public synchronized void deactivate(boolean flushOnExit) throws HyracksDataException {
if (!isActivated) {
- return;
+ throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
}
if (flushOnExit) {
@@ -164,7 +167,7 @@
mutableComponent.getBTree().deactivate();
mutableComponent.getRTree().destroy();
mutableComponent.getBTree().destroy();
- ((InMemoryBufferCache) mutableComponent.getRTree().getBufferCache()).close();
+ ((IVirtualBufferCache) mutableComponent.getRTree().getBufferCache()).close();
}
@Override
@@ -172,9 +175,6 @@
if (isActivated) {
throw new HyracksDataException("Failed to destroy the index since it is activated.");
}
-
- mutableComponent.getRTree().deactivate();
- mutableComponent.getBTree().deactivate();
}
@Override
@@ -334,9 +334,9 @@
NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
return new LSMRTreeOpContext(rtreeAccessor, (IRTreeLeafFrame) rtreeLeafFrameFactory.createFrame(),
- (IRTreeInteriorFrame) rtreeInteriorFrameFactory.createFrame(), memFreePageManager
+ (IRTreeInteriorFrame) rtreeInteriorFrameFactory.createFrame(), virtualFreePageManager
.getMetaDataFrameFactory().createFrame(), 4, btreeAccessor, btreeLeafFrameFactory,
- btreeInteriorFrameFactory, memFreePageManager.getMetaDataFrameFactory().createFrame(),
+ btreeInteriorFrameFactory, virtualFreePageManager.getMetaDataFrameFactory().createFrame(),
rtreeCmpFactories, btreeCmpFactories, modCallback, NoOpOperationCallback.INSTANCE);
}
@@ -360,8 +360,8 @@
@Override
public long getMemoryAllocationSize() {
- InMemoryBufferCache memBufferCache = (InMemoryBufferCache) mutableComponent.getRTree().getBufferCache();
- return memBufferCache.getNumPages() * memBufferCache.getPageSize();
+ IBufferCache virtualBufferCache = mutableComponent.getRTree().getBufferCache();
+ return virtualBufferCache.getNumPages() * virtualBufferCache.getPageSize();
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 367b387..bdec9ce 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -30,7 +30,6 @@
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -44,7 +43,6 @@
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
@@ -55,7 +53,8 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
@@ -67,22 +66,21 @@
public class LSMRTree extends AbstractLSMRTree {
- public LSMRTree(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
- ITreeIndexFrameFactory rtreeInteriorFrameFactory, ITreeIndexFrameFactory rtreeLeafFrameFactory,
- ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
- ILSMIndexFileManager fileNameManager, TreeIndexFactory<RTree> diskRTreeFactory,
- TreeIndexFactory<BTree> diskBTreeFactory, BloomFilterFactory bloomFilterFactory,
- double bloomFilterFalsePositiveRate, IFileMapProvider diskFileMapProvider, int fieldCount,
- IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
- ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- super(memBufferCache, memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
- btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory,
- new LSMRTreeComponentFactory(diskRTreeFactory, diskBTreeFactory, bloomFilterFactory),
- diskFileMapProvider, fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
- linearizerArray, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
- ioOpCallbackProvider);
+ public LSMRTree(IVirtualBufferCache virtualBufferCache, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
+ ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
+ ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileNameManager,
+ TreeIndexFactory<RTree> diskRTreeFactory, TreeIndexFactory<BTree> diskBTreeFactory,
+ BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
+ IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] rtreeCmpFactories,
+ IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
+ int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
+ ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ super(virtualBufferCache, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
+ btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, new LSMRTreeComponentFactory(
+ diskRTreeFactory, diskBTreeFactory, bloomFilterFactory), diskFileMapProvider, fieldCount,
+ rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray,
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
}
/**
@@ -246,13 +244,9 @@
IBinaryComparatorFactory[] linearizerArray = { linearizer };
- if (rTreeTupleSorter == null) {
- rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getRTree().getFileId(), linearizerArray,
- rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(), flushingComponent
- .getRTree().getBufferCache(), comparatorFields);
- } else {
- rTreeTupleSorter.reset();
- }
+ rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getRTree().getFileId(), linearizerArray,
+ rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(), flushingComponent.getRTree()
+ .getBufferCache(), comparatorFields);
// BulkLoad the tuples from the in-memory tree into the new disk
// RTree.
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMutableComponent.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMutableComponent.java
index 2586d58..35fdde5 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMutableComponent.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMutableComponent.java
@@ -17,7 +17,7 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMutableLSMComponent;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
@@ -25,12 +25,12 @@
private final RTree rtree;
private final BTree btree;
- private final IInMemoryFreePageManager mfpm;
+ private final IVirtualBufferCache vbc;
- public LSMRTreeMutableComponent(RTree rtree, BTree btree, IInMemoryFreePageManager mfpm) {
+ public LSMRTreeMutableComponent(RTree rtree, BTree btree, IVirtualBufferCache vbc) {
this.rtree = rtree;
this.btree = btree;
- this.mfpm = mfpm;
+ this.vbc = vbc;
}
public RTree getRTree() {
@@ -43,15 +43,21 @@
@Override
protected boolean isFull() {
- return mfpm.isFull();
+ return vbc.isFull();
}
@Override
protected void reset() throws HyracksDataException {
super.reset();
- rtree.clear();
+ rtree.deactivate();
+ rtree.destroy();
+ rtree.create();
+ rtree.activate();
if (btree != null) {
- btree.clear();
+ btree.deactivate();
+ btree.destroy();
+ btree.create();
+ btree.activate();
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index a3a8ca2..37c7a11 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -24,7 +24,6 @@
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -38,7 +37,6 @@
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
@@ -50,7 +48,8 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
@@ -68,21 +67,20 @@
// because it should have a different tuple writer in it's leaf frames.
private final ILSMComponentFactory bulkLoaComponentFactory;
- public LSMRTreeWithAntiMatterTuples(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
- ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
- ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileManager,
- TreeIndexFactory<RTree> diskRTreeFactory, TreeIndexFactory<RTree> bulkLoadRTreeFactory,
- IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] rtreeCmpFactories,
- IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
- int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ public LSMRTreeWithAntiMatterTuples(IVirtualBufferCache virtualBufferCache,
+ ITreeIndexFrameFactory rtreeInteriorFrameFactory, ITreeIndexFrameFactory rtreeLeafFrameFactory,
+ ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
+ ILSMIndexFileManager fileManager, TreeIndexFactory<RTree> diskRTreeFactory,
+ TreeIndexFactory<RTree> bulkLoadRTreeFactory, IFileMapProvider diskFileMapProvider, int fieldCount,
+ IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+ ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- super(memBufferCache, memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
- btreeInteriorFrameFactory, btreeLeafFrameFactory, fileManager, diskRTreeFactory,
- new LSMRTreeWithAntiMatterTuplesComponentFactory(diskRTreeFactory), diskFileMapProvider, fieldCount,
- rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray, 0, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ super(virtualBufferCache, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
+ btreeLeafFrameFactory, fileManager, diskRTreeFactory, new LSMRTreeWithAntiMatterTuplesComponentFactory(
+ diskRTreeFactory), diskFileMapProvider, fieldCount, rtreeCmpFactories, btreeCmpFactories,
+ linearizer, comparatorFields, linearizerArray, 0, mergePolicy, opTracker, ioScheduler,
+ ioOpCallbackProvider);
bulkLoaComponentFactory = new LSMRTreeWithAntiMatterTuplesComponentFactory(bulkLoadRTreeFactory);
this.bTreeTupleSorter = null;
}
@@ -237,18 +235,13 @@
// Since the LSM-RTree is used as a secondary assumption, the
// primary key will be the last comparator in the BTree comparators
- if (rTreeTupleSorter == null) {
- rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getRTree().getFileId(), linearizerArray,
- rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(), flushingComponent
- .getRTree().getBufferCache(), comparatorFields);
+ rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getRTree().getFileId(), linearizerArray,
+ rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(), flushingComponent.getRTree()
+ .getBufferCache(), comparatorFields);
- bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBTree().getFileId(), linearizerArray,
- btreeLeafFrameFactory.createFrame(), btreeLeafFrameFactory.createFrame(), flushingComponent
- .getBTree().getBufferCache(), comparatorFields);
- } else {
- rTreeTupleSorter.reset();
- bTreeTupleSorter.reset();
- }
+ bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBTree().getFileId(), linearizerArray,
+ btreeLeafFrameFactory.createFrame(), btreeLeafFrameFactory.createFrame(), flushingComponent.getBTree()
+ .getBufferCache(), comparatorFields);
// BulkLoad the tuples from the in-memory tree into the new disk
// RTree.
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index d299bfe..b80026e 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -26,19 +26,18 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManagerFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTree;
@@ -60,13 +59,26 @@
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public class LSMRTreeUtils {
- public static LSMRTree createLSMTree(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
- IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
- IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
- IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ public static LSMRTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
+ FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+ ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+ IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+ RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+ ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory)
+ throws TreeIndexException {
+ return createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits,
+ rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+ linearizeCmpFactory, 0);
+ }
+
+ public static LSMRTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
+ FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+ ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+ IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+ RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+ ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory,
int ioDeviceId) throws TreeIndexException {
LSMTypeAwareTupleWriterFactory rtreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, false);
@@ -103,23 +115,35 @@
ILSMIndexFileManager fileNameManager = new LSMRTreeFileManager(ioManager, diskFileMapProvider, file,
diskRTreeFactory, diskBTreeFactory, ioDeviceId);
- LSMRTree lsmTree = new LSMRTree(memBufferCache, memFreePageManager, rtreeInteriorFrameFactory,
- rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager,
- diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate,
- diskFileMapProvider, typeTraits.length, rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory,
- comparatorFields, linearizerArray, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ LSMRTree lsmTree = new LSMRTree(virtualBufferCache, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
+ btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory,
+ bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length,
+ rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields, linearizerArray,
+ mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
return lsmTree;
}
- public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
+ public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(
+ IVirtualBufferCache virtualBufferCache, IIOManager ioManager, FileReference file,
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
- ILinearizeComparatorFactory linearizerCmpFactory, int ioDeviceId) throws TreeIndexException {
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizerCmpFactory)
+ throws TreeIndexException {
+ return createLSMTreeWithAntiMatterTuples(virtualBufferCache, ioManager, file, diskBufferCache,
+ diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
+ rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, linearizerCmpFactory, 0);
+ }
+ public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(
+ IVirtualBufferCache virtualBufferCache, IIOManager ioManager, FileReference file,
+ IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+ IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizerCmpFactory,
+ int ioDeviceId) throws TreeIndexException {
LSMRTreeTupleWriterFactory rtreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
@@ -156,11 +180,11 @@
ILSMIndexFileManager fileNameManager = new LSMRTreeWithAntiMatterTuplesFileManager(ioManager,
diskFileMapProvider, file, diskRTreeFactory, ioDeviceId);
- LSMRTreeWithAntiMatterTuples lsmTree = new LSMRTreeWithAntiMatterTuples(memBufferCache, memFreePageManager,
+ LSMRTreeWithAntiMatterTuples lsmTree = new LSMRTreeWithAntiMatterTuples(virtualBufferCache,
rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
fileNameManager, diskRTreeFactory, bulkLoadRTreeFactory, diskFileMapProvider, typeTraits.length,
rtreeCmpFactories, btreeCmpFactories, linearizerCmpFactory, comparatorFields, linearizerArray,
- mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
return lsmTree;
}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/DebugBufferCache.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/DebugBufferCache.java
index df8cc26..81f3a26 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/DebugBufferCache.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/DebugBufferCache.java
@@ -99,7 +99,7 @@
}
@Override
- public void close() {
+ public void close() throws HyracksDataException {
bufferCache.close();
}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCache.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCache.java
index 8b86a91..d61b3f1 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCache.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCache.java
@@ -40,5 +40,5 @@
public int getNumPages();
- public void close();
+ public void close() throws HyracksDataException;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexLifecycleTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexLifecycleTest.java
index 2226d94..50bf89a 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexLifecycleTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexLifecycleTest.java
@@ -38,7 +38,6 @@
// Double open is valid
index.activate();
- index.activate();
Assert.assertTrue(isEmptyIndex());
// Insert some stuff
@@ -55,7 +54,6 @@
// Double close is valid
index.deactivate();
- index.deactivate();
// Check that the inserted stuff is still there
index.activate();
@@ -93,4 +91,19 @@
public void invalidSequenceTest4() throws Exception {
index.clear();
}
+
+ @Test(expected = HyracksDataException.class)
+ public void invalidSequenceTest5() throws Exception {
+ index.create();
+ index.activate();
+ index.activate();
+ }
+
+ @Test(expected = HyracksDataException.class)
+ public void invalidSequenceTest6() throws Exception {
+ index.create();
+ index.activate();
+ index.deactivate();
+ index.deactivate();
+ }
}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestVirtualBufferCacheProvider.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestVirtualBufferCacheProvider.java
new file mode 100644
index 0000000..1bf918b
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestVirtualBufferCacheProvider.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.hyracks.test.support;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+
+public class TestVirtualBufferCacheProvider implements IVirtualBufferCacheProvider {
+
+ private static final long serialVersionUID = 1L;
+
+ private final int pageSize;
+ private final int numPages;
+
+ public TestVirtualBufferCacheProvider(int pageSize, int numPages) {
+ this.pageSize = pageSize;
+ this.numPages = numPages;
+ }
+
+ @Override
+ public IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx) {
+ return new VirtualBufferCache(new HeapBufferAllocator(), pageSize, numPages);
+ }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeLifecycleTest.java
index 1445d2c..12de734 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeLifecycleTest.java
@@ -35,8 +35,12 @@
@Override
public void tearDown() throws HyracksDataException {
- testCtx.getIndex().deactivate();
- testCtx.getIndex().destroy();
+ try {
+ testCtx.getIndex().deactivate();
+ } catch (Exception e) {
+ } finally {
+ testCtx.getIndex().destroy();
+ }
harness.tearDown();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
index a7eee57..56c7ecd 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
@@ -51,11 +51,11 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
+ harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
index a648ff6..a168114 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
@@ -51,11 +51,11 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
- harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
+ harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index b5b0b76..081354e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -34,12 +34,11 @@
@Override
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
int[] bloomFilterKeyFields) throws TreeIndexException {
- return LSMBTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), typeTraits, cmpFactories, bloomFilterKeyFields,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
+ cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
+ harness.getIODeviceId());
}
@Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
index 875bbd4..56fc863 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
@@ -51,11 +51,11 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
- harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
+ harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
index 43d38bc..108741c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
@@ -42,19 +42,22 @@
@Override
public void setup() throws Exception {
harness.setUp();
- testCtx = LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ testCtx = LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, fieldSerdes.length, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
+ harness.getIODeviceId());
index = testCtx.getIndex();
}
@Override
public void tearDown() throws Exception {
- index.deactivate();
- index.destroy();
+ try {
+ index.deactivate();
+ } catch (Exception e) {
+ } finally {
+ index.destroy();
+ }
harness.tearDown();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
index 0ea2af2..85c2fa4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
@@ -50,11 +50,11 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
- harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
+ harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 901a31d..65cd9a4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -26,7 +26,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerProvider;
public class LSMBTreeModificationOperationCallbackTest extends AbstractModificationOperationCallbackTest {
private static final int NUM_TUPLES = 11;
@@ -42,12 +42,12 @@
@Override
protected void createIndexInstance() throws Exception {
- index = LSMBTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), SerdeUtils.serdesToTypeTraits(keySerdes),
+ index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ SerdeUtils.serdesToTypeTraits(keySerdes),
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
+ NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
index ac8290e..1cad578 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
@@ -52,11 +52,11 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
- harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
+ harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index 33e1641..8f3014e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -22,7 +22,7 @@
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerProvider;
public class LSMBTreeSearchOperationCallbackTest extends AbstractSearchOperationCallbackTest {
private final LSMBTreeTestHarness harness;
@@ -35,12 +35,12 @@
@Override
protected void createIndexInstance() throws Exception {
- index = LSMBTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), SerdeUtils.serdesToTypeTraits(keySerdes),
+ index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ SerdeUtils.serdesToTypeTraits(keySerdes),
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
+ NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
index cc2db11..2f03f78 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
@@ -51,11 +51,11 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
- harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
+ harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
index d56ff5b..fdcb6f7 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -50,12 +50,11 @@
@Override
protected ITreeIndex createIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
int[] bloomFilterKeyFields) throws TreeIndexException {
- return LSMBTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), typeTraits, cmpFactories, bloomFilterKeyFields,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
+ cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
+ harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java
index 1b453b7..44300c1 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java
@@ -30,18 +30,15 @@
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
public class InMemoryBTreeRunner extends Thread implements IExperimentRunner {
@@ -66,13 +63,11 @@
protected void init(int pageSize, int numPages, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories)
throws HyracksDataException, BTreeException {
- ICacheMemoryAllocator allocator = new HeapBufferAllocator();
- bufferCache = new InMemoryBufferCache(allocator, pageSize, numPages, new TransientFileMapManager());
+ bufferCache = new VirtualBufferCache(new HeapBufferAllocator(), pageSize, numPages);
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- IFreePageManager freePageManager = new InMemoryFreePageManager(bufferCache.getNumPages(), metaFrameFactory);
+ IFreePageManager freePageManager = new VirtualFreePageManager(bufferCache.getNumPages());
btree = new BTree(bufferCache, new TransientFileMapManager(), freePageManager, interiorFrameFactory,
leafFrameFactory, cmpFactories, typeTraits.length, file);
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index 7e54003..5c1ca0f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -26,27 +26,23 @@
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestUtils;
@@ -92,15 +88,12 @@
ioDeviceId = 0;
IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), inMemPageSize,
- inMemNumPages, new TransientFileMapManager());
- IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(inMemNumPages,
- new LIFOMetaDataFrameFactory());
+ IVirtualBufferCache virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), inMemPageSize,
+ inMemNumPages);
this.ioScheduler = SynchronousScheduler.INSTANCE;
- lsmtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, bufferCache, fmp,
- typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
- ThreadCountingOperationTrackerFactory.INSTANCE, ioScheduler, NoOpIOOperationCallback.INSTANCE,
- ioDeviceId);
+ lsmtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, bufferCache, fmp, typeTraits,
+ cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
+ new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallback.INSTANCE, ioDeviceId);
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
index a830dc0..5195074 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -25,14 +25,13 @@
import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -63,11 +62,10 @@
upsertCheckTuple(checkTuple, checkTuples);
}
- public static LSMBTreeTestContext create(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IOManager ioManager, FileReference file,
- IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
- int numKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ public static LSMBTreeTestContext create(IVirtualBufferCache virtualBufferCache, IOManager ioManager,
+ FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+ ISerializerDeserializer[] fieldSerdes, int numKeyFields, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
@@ -75,10 +73,9 @@
for (int i = 0; i < numKeyFields; ++i) {
bloomFilterKeyFields[i] = i;
}
- LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
- diskBufferCache, diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields,
- bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider,
- ioDeviceId);
+ LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
+ diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
+ mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, ioDeviceId);
LSMBTreeTestContext testCtx = new LSMBTreeTestContext(fieldSerdes, lsmTree);
return testCtx;
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
index 066f57b..e84bced 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -29,24 +29,20 @@
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestUtils;
@@ -69,12 +65,11 @@
protected int ioDeviceId;
protected IBufferCache diskBufferCache;
protected IFileMapProvider diskFileMapProvider;
- protected IInMemoryBufferCache memBufferCache;
- protected IInMemoryFreePageManager memFreePageManager;
+ protected IVirtualBufferCache virtualBufferCache;
protected IHyracksTaskContext ctx;
protected ILSMIOOperationScheduler ioScheduler;
protected ILSMMergePolicy mergePolicy;
- protected ILSMOperationTrackerFactory opTrackerFactory;
+ protected ILSMOperationTracker opTracker;
protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
protected final Random rnd = new Random();
@@ -93,7 +88,7 @@
this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_BTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
this.ioScheduler = SynchronousScheduler.INSTANCE;
this.mergePolicy = NoMergePolicy.INSTANCE;
- this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+ this.opTracker = new ThreadCountingTracker();
this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
}
@@ -108,7 +103,7 @@
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.ioScheduler = SynchronousScheduler.INSTANCE;
this.mergePolicy = NoMergePolicy.INSTANCE;
- this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+ this.opTracker = new ThreadCountingTracker();
}
public void setUp() throws HyracksException {
@@ -118,9 +113,7 @@
TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages,
- new TransientFileMapManager());
- memFreePageManager = new InMemoryFreePageManager(memNumPages, new LIFOMetaDataFrameFactory());
+ virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
ioManager = TestStorageManagerComponentHolder.getIOManager();
ioDeviceId = 0;
rnd.setSeed(RANDOM_SEED);
@@ -186,18 +179,14 @@
return diskFileMapProvider;
}
- public IInMemoryBufferCache getMemBufferCache() {
- return memBufferCache;
+ public IVirtualBufferCache getVirtualBufferCache() {
+ return virtualBufferCache;
}
public double getBoomFilterFalsePositiveRate() {
return bloomFilterFalsePositiveRate;
}
- public IInMemoryFreePageManager getMemFreePageManager() {
- return memFreePageManager;
- }
-
public IHyracksTaskContext getHyracksTastContext() {
return ctx;
}
@@ -214,8 +203,8 @@
return ioScheduler;
}
- public ILSMOperationTrackerFactory getOperationTrackerFactory() {
- return opTrackerFactory;
+ public ILSMOperationTracker getOperationTracker() {
+ return opTracker;
}
public ILSMMergePolicy getMergePolicy() {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryBufferCacheTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryBufferCacheTest.java
deleted file mode 100644
index adba93d..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryBufferCacheTest.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.common;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.util.HashSet;
-
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
-import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
-
-public class InMemoryBufferCacheTest {
- private static final int PAGE_SIZE = 256;
- private static final int NUM_PAGES = 100;
- private HashSet<ICachedPage> pinnedPages = new HashSet<ICachedPage>();
-
- @Test
- public void test01() throws Exception {
- InMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), PAGE_SIZE, NUM_PAGES,
- new TransientFileMapManager());
- memBufferCache.open();
- int dummyFileId = 0;
- // Pin all pages, and make sure they return unique ICachedPages.
- // We expect no overflow pages.
- for (int i = 0; i < NUM_PAGES; i++) {
- ICachedPage page = memBufferCache.pin(BufferedFileHandle.getDiskPageId(dummyFileId, i), false);
- if (pinnedPages.contains(page)) {
- fail("Id collision for ICachedPage, caused by id: " + i);
- }
- pinnedPages.add(page);
- assertEquals(0, memBufferCache.getNumOverflowPages());
- }
- // Pin pages above capacity. We expect to be given new overflow pages.
- // Going above capacity should be very rare, but nevertheless succeed.
- for (int i = 0; i < 100; i++) {
- ICachedPage page = memBufferCache.pin(BufferedFileHandle.getDiskPageId(dummyFileId, i + NUM_PAGES), false);
- if (pinnedPages.contains(page)) {
- fail("Id collision for ICachedPage, caused by overflow id: " + i);
- }
- pinnedPages.add(page);
- assertEquals(i + 1, memBufferCache.getNumOverflowPages());
- }
- memBufferCache.close();
- }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryFreePageManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryFreePageManagerTest.java
deleted file mode 100644
index bd09a3f..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryFreePageManagerTest.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.common;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
-
-public class InMemoryFreePageManagerTest {
-
- private final int NUM_PAGES = 100;
-
- private void testInMemoryFreePageManager(InMemoryFreePageManager memFreePageManager) throws HyracksDataException {
- // The first two pages are reserved for the BTree's metadata page and
- // root page.
- // The "actual" capacity is therefore numPages - 2.
- int capacity = memFreePageManager.getCapacity();
- assertEquals(capacity, NUM_PAGES - 2);
- for (int i = 0; i < capacity; i++) {
- int pageId = memFreePageManager.getFreePage(null);
- // The free pages start from page 2;
- assertEquals(i + 2, pageId);
- assertFalse(memFreePageManager.isFull());
- }
- // Start asking for 100 pages above the capacity.
- // Asking for pages above the capacity should be very rare, but
- // nevertheless succeed.
- // We expect isFull() to return true.
- for (int i = 0; i < 100; i++) {
- int pageId = memFreePageManager.getFreePage(null);
- assertEquals(capacity + i + 2, pageId);
- assertTrue(memFreePageManager.isFull());
- }
- }
-
- @Test
- public void test01() throws HyracksDataException {
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- InMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(NUM_PAGES, metaFrameFactory);
- testInMemoryFreePageManager(memFreePageManager);
- // We expect exactly the same behavior after a reset().
- memFreePageManager.reset();
- testInMemoryFreePageManager(memFreePageManager);
- }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/VirtualBufferCacheTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/VirtualBufferCacheTest.java
new file mode 100644
index 0000000..38a001d
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/VirtualBufferCacheTest.java
@@ -0,0 +1,118 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+
+public class VirtualBufferCacheTest {
+ private static final long SEED = 123456789L;
+ private static final int NUM_OVERPIN = 128;
+ private static final int PAGE_SIZE = 256;
+ private static final int NUM_FILES = 10;
+ private static final int NUM_PAGES = 1000;
+
+ private final Random random;
+ private final FileState[] fileStates;
+
+ private VirtualBufferCache vbc;
+
+ public VirtualBufferCacheTest() {
+ fileStates = new FileState[NUM_FILES];
+ for (int i = 0; i < NUM_FILES; i++) {
+ fileStates[i] = new FileState();
+ }
+ random = new Random(SEED);
+ vbc = null;
+ }
+
+ private static class FileState {
+ private int fileId;
+ private FileReference fileRef;
+ private int pinCount;
+ private Set<ICachedPage> pinnedPages;
+
+ public FileState() {
+ fileId = -1;
+ fileRef = null;
+ pinCount = 0;
+ pinnedPages = new HashSet<ICachedPage>();
+ }
+ }
+
+ /**
+ * Pins NUM_PAGES randomly distributed across NUM_FILES and checks that each
+ * set of cached pages pinned on behalf of a file are disjoint from all other sets of
+ * cached pages pinned on behalf of other files.
+ * Additionally, the test perform the same test when pinning over soft cap (NUM_PAGES)
+ * of pages.
+ */
+ @Test
+ public void test01() throws Exception {
+ ICacheMemoryAllocator allocator = new HeapBufferAllocator();
+ vbc = new VirtualBufferCache(allocator, PAGE_SIZE, NUM_PAGES);
+ vbc.open();
+ createFiles();
+
+ kPins(NUM_PAGES);
+ assertTrue(pagesDisjointed());
+
+ kPins(NUM_OVERPIN);
+ assertTrue(pagesDisjointed());
+
+ deleteFiles();
+ vbc.close();
+ }
+
+ private boolean pagesDisjointed() {
+ boolean disjoint = true;
+ for (int i = 0; i < NUM_FILES; i++) {
+ FileState fi = fileStates[i];
+ for (int j = i + 1; j < NUM_FILES; j++) {
+ FileState fj = fileStates[j];
+ disjoint = disjoint && Collections.disjoint(fi.pinnedPages, fj.pinnedPages);
+ }
+ }
+ return disjoint;
+ }
+
+ private void createFiles() throws Exception {
+ for (int i = 0; i < NUM_FILES; i++) {
+ FileState f = fileStates[i];
+ String fName = String.format("f%d", i);
+ f.fileRef = new FileReference(new File(fName));
+ vbc.createFile(f.fileRef);
+ f.fileId = vbc.getFileMapProvider().lookupFileId(f.fileRef);
+ }
+ }
+
+ private void deleteFiles() throws Exception {
+ for (int i = 0; i < NUM_FILES; i++) {
+ vbc.deleteFile(fileStates[i].fileId, false);
+ }
+ }
+
+ private void kPins(int k) throws Exception {
+ int numPinned = 0;
+ while (numPinned < k) {
+ int fsIdx = random.nextInt(NUM_FILES);
+ FileState f = fileStates[fsIdx];
+ ICachedPage p = vbc.pin(BufferedFileHandle.getDiskPageId(f.fileId, f.pinCount), true);
+ f.pinnedPages.add(p);
+ ++f.pinCount;
+ ++numPinned;
+ }
+ }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/VirtualFreePageManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/VirtualFreePageManagerTest.java
new file mode 100644
index 0000000..1dca8b3
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/VirtualFreePageManagerTest.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
+
+public class VirtualFreePageManagerTest {
+
+ private final int NUM_PAGES = 100;
+
+ private void testInMemoryFreePageManager(VirtualFreePageManager virtualFreePageManager) throws HyracksDataException {
+ // The first two pages are reserved for the BTree's metadata page and
+ // root page.
+ // The "actual" capacity is therefore numPages - 2.
+ int capacity = virtualFreePageManager.getCapacity();
+ assertEquals(capacity, NUM_PAGES - 2);
+ for (int i = 0; i < capacity; i++) {
+ int pageId = virtualFreePageManager.getFreePage(null);
+ // The free pages start from page 2;
+ assertEquals(i + 2, pageId);
+ }
+ // Start asking for 100 pages above the capacity.
+ // Asking for pages above the capacity should be very rare, but
+ // nevertheless succeed.
+ // We expect isFull() to return true.
+ for (int i = 0; i < 100; i++) {
+ int pageId = virtualFreePageManager.getFreePage(null);
+ assertEquals(capacity + i + 2, pageId);
+ }
+ }
+
+ @Test
+ public void test01() throws HyracksDataException {
+ VirtualFreePageManager virtualFreePageManager = new VirtualFreePageManager(NUM_PAGES);
+ testInMemoryFreePageManager(virtualFreePageManager);
+ // We expect exactly the same behavior after a reset().
+ virtualFreePageManager.reset();
+ testInMemoryFreePageManager(virtualFreePageManager);
+ }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index bf2d5e8..e1f6232 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -27,20 +27,20 @@
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IVirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -63,12 +63,12 @@
protected int ioDeviceId;
protected IBufferCache diskBufferCache;
protected IFileMapProvider diskFileMapProvider;
- protected IInMemoryBufferCache memBufferCache;
- protected IInMemoryFreePageManager memFreePageManager;
+ protected IVirtualBufferCache virtualBufferCache;
+ protected IVirtualFreePageManager virtualFreePageManager;
protected IHyracksTaskContext ctx;
protected ILSMIOOperationScheduler ioScheduler;
protected ILSMMergePolicy mergePolicy;
- protected ILSMOperationTrackerFactory opTrackerFactory;
+ protected ILSMOperationTracker opTracker;
protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
protected final Random rnd = new Random();
@@ -89,7 +89,7 @@
this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE;
this.ioScheduler = SynchronousScheduler.INSTANCE;
this.mergePolicy = NoMergePolicy.INSTANCE;
- this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+ this.opTracker = new ThreadCountingTracker();
this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
}
@@ -104,7 +104,7 @@
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.ioScheduler = SynchronousScheduler.INSTANCE;
this.mergePolicy = NoMergePolicy.INSTANCE;
- this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+ this.opTracker = new ThreadCountingTracker();
}
public void setUp() throws HyracksException {
@@ -113,9 +113,10 @@
TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
- memBufferCache.open();
- memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages, new LIFOMetaDataFrameFactory());
+ virtualBufferCache = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(),
+ memPageSize, memNumPages));
+ virtualBufferCache.open();
+ virtualFreePageManager = new VirtualFreePageManager(memNumPages);
ioManager = TestStorageManagerComponentHolder.getIOManager();
ioDeviceId = 0;
rnd.setSeed(RANDOM_SEED);
@@ -140,7 +141,7 @@
}
dir.delete();
}
- memBufferCache.close();
+ virtualBufferCache.close();
}
public FileReference getInvListsFileRef() {
@@ -187,16 +188,16 @@
return diskFileMapProvider;
}
- public IInMemoryBufferCache getMemBufferCache() {
- return memBufferCache;
+ public IVirtualBufferCache getVirtualBufferCache() {
+ return virtualBufferCache;
}
public double getBoomFilterFalsePositiveRate() {
return bloomFilterFalsePositiveRate;
}
- public IInMemoryFreePageManager getMemFreePageManager() {
- return memFreePageManager;
+ public IVirtualFreePageManager getVirtualFreePageManager() {
+ return virtualFreePageManager;
}
public IHyracksTaskContext getHyracksTastContext() {
@@ -215,8 +216,8 @@
return ioScheduler;
}
- public ILSMOperationTrackerFactory getOperationTrackerFactory() {
- return opTrackerFactory;
+ public ILSMOperationTracker getOperationTracker() {
+ return opTracker;
}
public ILSMMergePolicy getMergePolicy() {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
index 09432de..a46a2d1 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
@@ -57,17 +57,22 @@
IBinaryComparatorFactory[] invListCmpFactories = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
.of(IntegerPointable.FACTORY) };
IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
- FileReference btreeFile = new FileReference(new File(harness.getInvListsFileRef().getFile().getPath() + "_btree"));
+ FileReference btreeFile = new FileReference(new File(harness.getInvListsFileRef().getFile().getPath()
+ + "_btree"));
index = new OnDiskInvertedIndex(harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), invListBuilder,
- invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, harness.getInvListsFileRef(),
- btreeFile);
+ invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+ harness.getInvListsFileRef(), btreeFile);
}
@Override
public void tearDown() throws Exception {
- index.deactivate();
- index.destroy();
+ try {
+ index.deactivate();
+ } catch (Exception e) {
+ } finally {
+ index.destroy();
+ }
harness.tearDown();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index f955fc9..f28d3b6 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -18,6 +18,7 @@
import java.io.ByteArrayInputStream;
import java.io.DataInput;
import java.io.DataInputStream;
+import java.io.File;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
@@ -27,6 +28,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
@@ -118,15 +120,16 @@
IInvertedIndex invIndex;
switch (invIndexType) {
case INMEMORY: {
- invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory);
+ invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(harness.getVirtualBufferCache(),
+ harness.getVirtualFreePageManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+ tokenCmpFactories, tokenizerFactory, new FileReference(new File(harness.getOnDiskDir())));
break;
}
case PARTITIONED_INMEMORY: {
- invIndex = InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory);
+ invIndex = InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(harness
+ .getVirtualBufferCache(), harness.getVirtualFreePageManager(), invListTypeTraits,
+ invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, new FileReference(
+ new File(harness.getOnDiskDir())));
break;
}
case ONDISK: {
@@ -142,22 +145,20 @@
break;
}
case LSM: {
- invIndex = InvertedIndexUtils.createLSMInvertedIndex(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), harness.getDiskFileMapProvider(), invListTypeTraits,
- invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- harness.getDiskBufferCache(), harness.getIOManager(), harness.getOnDiskDir(),
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ invIndex = InvertedIndexUtils.createLSMInvertedIndex(harness.getVirtualBufferCache(),
+ harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+ tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getIOManager(),
+ harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
break;
}
case PARTITIONED_LSM: {
- invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), harness.getDiskFileMapProvider(), invListTypeTraits,
- invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- harness.getDiskBufferCache(), harness.getIOManager(), harness.getOnDiskDir(),
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(harness.getVirtualBufferCache(),
+ harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+ tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getIOManager(),
+ harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
break;
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
index edd24b4..f579738 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
@@ -54,11 +54,11 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
index 360c02c..315b0ea 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
@@ -54,11 +54,11 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index 0c68fa6..ecd7e02 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -37,11 +37,11 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
- valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
- harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
+ rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
index f39b139..e790cb2 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
@@ -54,11 +54,11 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
index 8bee460..f186fb2 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
@@ -52,19 +52,22 @@
@Override
public void setup() throws Exception {
harness.setUp();
- testCtx = LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ testCtx = LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+ harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
index = testCtx.getIndex();
}
@Override
public void tearDown() throws Exception {
- index.deactivate();
- index.destroy();
+ try {
+ index.deactivate();
+ } catch (Exception e) {
+ } finally {
+ index.destroy();
+ }
harness.tearDown();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
index 5d7d31b..1702e99 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
@@ -53,11 +53,11 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
index 8ef0a9f..81db7bc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
@@ -54,11 +54,11 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
index f778aa9..efff30c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
@@ -54,12 +54,11 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
- harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
- numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
-
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
index 44d3d1b..485b958f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -54,12 +54,11 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
- harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
- numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
-
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
index 8b12224..b6b5195 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -37,11 +37,10 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
- harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
- btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
+ rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+ harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
index 96ef868..fecb1c6 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
@@ -54,12 +54,11 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
- harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
- numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
-
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
index 012559d..fe9ffe4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
@@ -52,18 +52,22 @@
@Override
public void setup() throws Exception {
harness.setUp();
- testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
- harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
- numKeys, RTreePolicyType.RTREE, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
+ harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+ harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
+ harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
index = testCtx.getIndex();
}
@Override
public void tearDown() throws Exception {
- index.deactivate();
- index.destroy();
+ try {
+ index.deactivate();
+ } catch (Exception e) {
+ } finally {
+ index.destroy();
+ }
harness.tearDown();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
index 1c67cd0..bb15e6d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
@@ -53,11 +53,11 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
- harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
- numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
index fe61afd..d93129d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
@@ -54,12 +54,11 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
- harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
- numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
-
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index 1d2271c..11071ea 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -57,11 +57,11 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
- valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
- harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
+ rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
index aa5023d..4fffdfd 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -57,11 +57,10 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getMemBufferCache(),
- harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
- harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
- btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+ return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(), harness.getIOManager(),
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
+ rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+ harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index fb9fb23..df95773 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -23,14 +23,13 @@
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTree;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
@@ -67,23 +66,22 @@
return lsmTree.getComparatorFactories();
}
- public static LSMRTreeTestContext create(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IOManager ioManager, FileReference file,
- IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
- IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
- double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+ public static LSMRTreeTestContext create(IVirtualBufferCache virtualBufferCache, IOManager ioManager,
+ FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+ ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
+ int numKeyFields, RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
.serdesToComparatorFactories(fieldSerdes, numKeyFields);
IBinaryComparatorFactory[] btreeCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
fieldSerdes.length);
- LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
- diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
- valueProviderFactories, rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
- ioScheduler, ioOpCallbackProvider,
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), ioDeviceId);
+ LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
+ diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
+ rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
+ ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
+ ioDeviceId);
LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
return testCtx;
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
index 4c2e83b..5f0072e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -28,20 +28,17 @@
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -65,13 +62,12 @@
protected int ioDeviceId;
protected IBufferCache diskBufferCache;
protected IFileMapProvider diskFileMapProvider;
- protected IInMemoryBufferCache memBufferCache;
- protected IInMemoryFreePageManager memFreePageManager;
+ protected IVirtualBufferCache virtualBufferCache;
protected IHyracksTaskContext ctx;
protected ILSMIOOperationScheduler ioScheduler;
protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
protected ILSMMergePolicy mergePolicy;
- protected ILSMOperationTrackerFactory opTrackerFactory;
+ protected ILSMOperationTracker opTracker;
protected final Random rnd = new Random();
protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
@@ -89,7 +85,7 @@
this.hyracksFrameSize = AccessMethodTestsConfig.LSM_RTREE_HYRACKS_FRAME_SIZE;
this.ioScheduler = SynchronousScheduler.INSTANCE;
this.mergePolicy = NoMergePolicy.INSTANCE;
- this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+ this.opTracker = new ThreadCountingTracker();
this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
}
@@ -104,7 +100,7 @@
this.hyracksFrameSize = hyracksFrameSize;
this.ioScheduler = SynchronousScheduler.INSTANCE;
this.mergePolicy = NoMergePolicy.INSTANCE;
- this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+ this.opTracker = new ThreadCountingTracker();
}
public void setUp() throws HyracksException {
@@ -114,8 +110,7 @@
TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
- memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages, new LIFOMetaDataFrameFactory());
+ virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
ioManager = TestStorageManagerComponentHolder.getIOManager();
ioDeviceId = 0;
rnd.setSeed(RANDOM_SEED);
@@ -181,18 +176,14 @@
return diskFileMapProvider;
}
- public IInMemoryBufferCache getMemBufferCache() {
- return memBufferCache;
+ public IVirtualBufferCache getVirtualBufferCache() {
+ return virtualBufferCache;
}
public double getBoomFilterFalsePositiveRate() {
return bloomFilterFalsePositiveRate;
}
- public IInMemoryFreePageManager getMemFreePageManager() {
- return memFreePageManager;
- }
-
public IHyracksTaskContext getHyracksTastContext() {
return ctx;
}
@@ -213,8 +204,8 @@
return ioScheduler;
}
- public ILSMOperationTrackerFactory getOperationTrackerFactory() {
- return opTrackerFactory;
+ public ILSMOperationTracker getOperationTracker() {
+ return opTracker;
}
public ILSMMergePolicy getMergePolicy() {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
index 48cc481..dbb4e28 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -23,14 +23,13 @@
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuples;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
@@ -67,23 +66,22 @@
return lsmTree.getComparatorFactories();
}
- public static LSMRTreeWithAntiMatterTuplesTestContext create(IInMemoryBufferCache memBufferCache,
- IInMemoryFreePageManager memFreePageManager, IOManager ioManager, FileReference file,
- IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
+ public static LSMRTreeWithAntiMatterTuplesTestContext create(IVirtualBufferCache virtualBufferCache,
+ IOManager ioManager, FileReference file, IBufferCache diskBufferCache,
+ IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
- ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId)
- throws Exception {
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
.serdesToComparatorFactories(fieldSerdes, numKeyFields);
IBinaryComparatorFactory[] btreeCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
fieldSerdes.length);
- LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(memBufferCache,
- memFreePageManager, ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits,
- rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider,
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), ioDeviceId);
+ LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCache,
+ ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories,
+ btreeCmpFactories, valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler,
+ ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
+ ioDeviceId);
LSMRTreeWithAntiMatterTuplesTestContext testCtx = new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes,
lsmTree);
return testCtx;
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeLifecycleTest.java
index 69c7e15..5fa074d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeLifecycleTest.java
@@ -42,8 +42,12 @@
@Override
public void tearDown() throws HyracksDataException {
- testCtx.getIndex().deactivate();
- testCtx.getIndex().destroy();
+ try {
+ testCtx.getIndex().deactivate();
+ } catch (Exception e) {
+ } finally {
+ testCtx.getIndex().destroy();
+ }
harness.tearDown();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTests.java b/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java
similarity index 71%
rename from hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTests.java
rename to hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java
index a649aa7..cc73e34 100644
--- a/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTests.java
+++ b/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java
@@ -9,7 +9,6 @@
import org.junit.Test;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IFileHandle;
import edu.uci.ics.hyracks.api.io.IIOManager;
@@ -22,7 +21,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestUtils;
-public class BufferCacheRegressionTests {
+public class BufferCacheRegressionTest {
protected static final String tmpDir = System.getProperty("java.io.tmpdir");
protected static final String sep = System.getProperty("file.separator");
@@ -121,51 +120,4 @@
}
bufferCache.close();
}
-
- // Tests the behavior of the BufferCache when more than all pages are
- // pinned. We expect an exception.
- @Test
- public void testPinningAllPages() throws HyracksDataException {
- int numPages = 10;
- TestStorageManagerComponentHolder.init(PAGE_SIZE, numPages, 1);
-
- IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
-
- FileReference firstFileRef = new FileReference(new File(fileName));
- bufferCache.createFile(firstFileRef);
- int fileId = fmp.lookupFileId(firstFileRef);
- bufferCache.openFile(fileId);
-
- // Pin all pages.
- ICachedPage[] pages = new ICachedPage[numPages];
- for (int i = 0; i < numPages; ++i) {
- pages[i] = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, i), true);
- }
-
- // Try to pin another page. We expect a HyracksDataException.
- ICachedPage errorPage = null;
- try {
- errorPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, numPages), true);
- } catch (HyracksDataException e) {
- // This is the expected outcome.
- // The BufferCache should still be able to function properly.
- // Try unpinning all pages.
- for (int i = 0; i < numPages; ++i) {
- bufferCache.unpin(pages[i]);
- }
- // Now try pinning the page that failed above again.
- errorPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, numPages), true);
- // Unpin it.
- bufferCache.unpin(errorPage);
- // Cleanup.
- bufferCache.closeFile(fileId);
- bufferCache.close();
- return;
- } catch (Exception e) {
- fail("Expected a HyracksDataException when pinning more pages than available but got another exception: "
- + e.getMessage());
- }
- fail("Expected a HyracksDataException when pinning more pages than available.");
- }
}
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
index 92f2c11..2b32545 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
@@ -82,7 +82,7 @@
resourceIdFactory = new ResourceIdFactory(0);
}
- public void close() {
+ public void close() throws HyracksDataException {
for (Entry<Long, List<FileReference>> entry : iterationToFiles.entrySet())
for (FileReference fileRef : entry.getValue())
fileRef.delete();