Merged hyracks master back into VXQuery branch.
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
index 9a84173..7f447c6 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
@@ -68,7 +68,7 @@
}
public void open() throws HyracksDataException {
- lastReadSender = 0;
+ lastReadSender = -1;
}
public IInputChannel[] getChannels() {
@@ -77,14 +77,9 @@
public synchronized int findNextSender() throws HyracksDataException {
while (true) {
- switch (lastReadSender) {
- default:
- lastReadSender = frameAvailability.nextSetBit(lastReadSender + 1);
- if (lastReadSender >= 0) {
- break;
- }
- case 0:
- lastReadSender = frameAvailability.nextSetBit(0);
+ lastReadSender = frameAvailability.nextSetBit(lastReadSender + 1);
+ if (lastReadSender < 0) {
+ lastReadSender = frameAvailability.nextSetBit(0);
}
if (lastReadSender >= 0) {
assert availableFrameCounts[lastReadSender] > 0;
diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
index 6811e1b..fbece8b 100644
--- a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
@@ -217,16 +217,14 @@
throw new HyracksDataException("Cannot create a bloom filter with his huge number of pages.");
}
numPages = (int) tmp;
- if (this.numElements > 0) {
- persistBloomFilterMetaData();
- readBloomFilterMetaData();
- int currentPageId = 1;
- while (currentPageId <= numPages) {
- ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
- page.acquireWriteLatch();
- bloomFilterPages.add(page);
- ++currentPageId;
- }
+ persistBloomFilterMetaData();
+ readBloomFilterMetaData();
+ int currentPageId = 1;
+ while (currentPageId <= numPages) {
+ ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
+ page.acquireWriteLatch();
+ bloomFilterPages.add(page);
+ ++currentPageId;
}
}
diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilterSpecification.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilterSpecification.java
index a1e5517..0c5d7c8 100644
--- a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilterSpecification.java
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilterSpecification.java
@@ -16,12 +16,12 @@
package edu.uci.ics.hyracks.storage.am.bloomfilter.impls;
public final class BloomFilterSpecification {
- private final int numBucketsPerElement;
private final int numHashes;
+ private final int numBucketsPerElement;
- public BloomFilterSpecification(int numBucketsPerElement, int numHashes) {
- this.numBucketsPerElement = numBucketsPerElement;
+ public BloomFilterSpecification(int numHashes, int numBucketsPerElement) {
this.numHashes = numHashes;
+ this.numBucketsPerElement = numBucketsPerElement;
}
public int getNumBucketsPerElements() {
@@ -31,4 +31,4 @@
public int getNumHashes() {
return numHashes;
}
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index a79daef..83fb5ee 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -119,7 +119,8 @@
FrameUtils.flushFrame(writeBuffer, writer);
appender.reset(writeBuffer, true);
if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
+ throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
+ + appender.getBuffer().capacity() + ")");
}
}
}
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 f217419..35c9bcb 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
@@ -15,6 +15,8 @@
package edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow;
+import java.util.List;
+
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.ITreeIndex;
@@ -31,25 +33,25 @@
public class LSMBTreeDataflowHelper extends AbstractLSMIndexDataflowHelper {
public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IVirtualBufferCache virtualBufferCache, ILSMMergePolicy mergePolicy,
+ List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
+ this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
opTrackerFactory, ioScheduler, ioOpCallbackProvider);
}
public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+ List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- super(opDesc, ctx, partition, virtualBufferCache, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+ super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
ioScheduler, ioOpCallbackProvider);
}
@Override
public ITreeIndex createIndexInstance() throws HyracksDataException {
AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
- return LSMBTreeUtils.createLSMTree(virtualBufferCache, file, opDesc.getStorageManager().getBufferCache(ctx),
+ return LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, opDesc.getStorageManager().getBufferCache(ctx),
opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc.getTreeIndexBloomFilterKeyFields(),
bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler,
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 d331880..f4be88f 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
@@ -41,7 +41,7 @@
public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
return new LSMBTreeDataflowHelper(opDesc, ctx, partition,
- virtualBufferCacheProvider.getVirtualBufferCache(ctx), bloomFilterFalsePositiveRate,
+ virtualBufferCacheProvider.getVirtualBufferCaches(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 59af99e..c8ed0d0 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
@@ -67,6 +67,7 @@
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;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -74,38 +75,42 @@
public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
- // In-memory components.
- private final LSMBTreeMutableComponent mutableComponent;
-
// For creating BTree's used in flush and merge.
- private final LSMBTreeImmutableComponentFactory componentFactory;
+ private final LSMBTreeDiskComponentFactory componentFactory;
// For creating BTree's used in bulk load. Different from diskBTreeFactory
// because it should have a different tuple writer in it's leaf frames.
- private final LSMBTreeImmutableComponentFactory bulkLoadComponentFactory;
+ private final LSMBTreeDiskComponentFactory bulkLoadComponentFactory;
// Common for in-memory and on-disk components.
private final ITreeIndexFrameFactory insertLeafFrameFactory;
private final ITreeIndexFrameFactory deleteLeafFrameFactory;
private final IBinaryComparatorFactory[] cmpFactories;
- public LSMBTree(IVirtualBufferCache virtualBufferCache, ITreeIndexFrameFactory interiorFrameFactory,
+ public LSMBTree(List<IVirtualBufferCache> virtualBufferCaches, 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(virtualBufferCache, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
+ super(virtualBufferCaches, 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);
+ int i = 0;
+ for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
+ LSMBTreeMemoryComponent mutableComponent = new LSMBTreeMemoryComponent(new BTree(virtualBufferCache,
+ virtualBufferCache.getFileMapProvider(), new VirtualFreePageManager(
+ virtualBufferCache.getNumPages()), interiorFrameFactory, insertLeafFrameFactory,
+ cmpFactories, fieldCount, new FileReference(new File(fileManager.getBaseDir() + "_virtual_" + i))),
+ virtualBufferCache, i == 0 ? true : false);
+ memoryComponents.add(mutableComponent);
+ ++i;
+ }
+
this.insertLeafFrameFactory = insertLeafFrameFactory;
this.deleteLeafFrameFactory = deleteLeafFrameFactory;
this.cmpFactories = cmpFactories;
- componentFactory = new LSMBTreeImmutableComponentFactory(diskBTreeFactory, bloomFilterFactory);
- bulkLoadComponentFactory = new LSMBTreeImmutableComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory);
+ componentFactory = new LSMBTreeDiskComponentFactory(diskBTreeFactory, bloomFilterFactory);
+ bulkLoadComponentFactory = new LSMBTreeDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory);
}
@Override
@@ -116,7 +121,7 @@
fileManager.deleteDirs();
fileManager.createDirs();
- componentsRef.get().clear();
+ diskComponents.clear();
}
@Override
@@ -124,11 +129,13 @@
if (isActivated) {
throw new HyracksDataException("Failed to activate the index since it is already activated.");
}
-
- ((IVirtualBufferCache) mutableComponent.getBTree().getBufferCache()).open();
- mutableComponent.getBTree().create();
- mutableComponent.getBTree().activate();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ for (ILSMComponent c : memoryComponents) {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
+ ((IVirtualBufferCache) mutableComponent.getBTree().getBufferCache()).open();
+ mutableComponent.getBTree().create();
+ mutableComponent.getBTree().activate();
+ }
+ List<ILSMComponent> immutableComponents = diskComponents;
immutableComponents.clear();
List<LSMComponentFileReferences> validFileReferences;
try {
@@ -137,7 +144,7 @@
throw new HyracksDataException(e);
}
for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
- LSMBTreeImmutableComponent component;
+ LSMBTreeDiskComponent component;
try {
component = createDiskComponent(componentFactory,
lsmComonentFileReference.getInsertIndexFileReference(),
@@ -168,17 +175,20 @@
}
}
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) c;
+ LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
BTree btree = component.getBTree();
BloomFilter bloomFilter = component.getBloomFilter();
btree.deactivate();
bloomFilter.deactivate();
}
- mutableComponent.getBTree().deactivate();
- mutableComponent.getBTree().destroy();
- ((IVirtualBufferCache) mutableComponent.getBTree().getBufferCache()).close();
+ for (ILSMComponent c : memoryComponents) {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
+ mutableComponent.getBTree().deactivate();
+ mutableComponent.getBTree().destroy();
+ ((IVirtualBufferCache) mutableComponent.getBTree().getBufferCache()).close();
+ }
isActivated = false;
}
@@ -193,13 +203,16 @@
throw new HyracksDataException("Failed to destroy the index since it is activated.");
}
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) c;
+ LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
component.getBTree().destroy();
component.getBloomFilter().destroy();
}
- mutableComponent.getBTree().destroy();
+ for (ILSMComponent c : memoryComponents) {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
+ mutableComponent.getBTree().destroy();
+ }
fileManager.deleteDirs();
}
@@ -209,11 +222,14 @@
throw new HyracksDataException("Failed to clear the index since it is not activated.");
}
- List<ILSMComponent> immutableComponents = componentsRef.get();
- mutableComponent.getBTree().clear();
- mutableComponent.reset();
+ List<ILSMComponent> immutableComponents = diskComponents;
+ for (ILSMComponent c : memoryComponents) {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
+ mutableComponent.getBTree().clear();
+ mutableComponent.reset();
+ }
for (ILSMComponent c : immutableComponents) {
- LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) c;
+ LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
component.getBloomFilter().deactivate();
component.getBTree().deactivate();
component.getBloomFilter().destroy();
@@ -224,20 +240,32 @@
@Override
public void getOperationalComponents(ILSMIndexOperationContext ctx) {
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
operationalComponents.clear();
+ int cmc = currentMutableComponentId.get();
+ ctx.setCurrentMutableComponentId(cmc);
+ int numMutableComponents = memoryComponents.size();
switch (ctx.getOperation()) {
case UPDATE:
case UPSERT:
case PHYSICALDELETE:
case FLUSH:
case DELETE:
- operationalComponents.add(mutableComponent);
+ operationalComponents.add(memoryComponents.get(cmc));
break;
case SEARCH:
case INSERT:
- operationalComponents.add(mutableComponent);
+ for (int i = 0; i < numMutableComponents - 1; i++) {
+ ILSMComponent c = memoryComponents.get((cmc + i + 1) % numMutableComponents);
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
+ if (mutableComponent.isReadable()) {
+ // Make sure newest components are added first
+ operationalComponents.add(0, mutableComponent);
+ }
+ }
+ // The current mutable component is always added
+ operationalComponents.add(0, memoryComponents.get(cmc));
operationalComponents.addAll(immutableComponents);
break;
case MERGE:
@@ -253,27 +281,28 @@
LSMBTreeOpContext ctx = (LSMBTreeOpContext) ictx;
switch (ctx.getOperation()) {
case PHYSICALDELETE:
- ctx.memBTreeAccessor.delete(tuple);
+ ctx.currentMutableBTreeAccessor.delete(tuple);
break;
case INSERT:
insert(tuple, ctx);
break;
default:
- ctx.memBTreeAccessor.upsert(tuple);
+ ctx.currentMutableBTreeAccessor.upsert(tuple);
break;
}
- mutableComponent.setIsModified();
}
private boolean insert(ITupleReference tuple, LSMBTreeOpContext ctx) throws HyracksDataException, IndexException {
+ ILSMComponent c = ctx.getComponentHolder().get(0);
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
MultiComparator comparator = MultiComparator.createIgnoreFieldLength(mutableComponent.getBTree()
.getComparatorFactories());
LSMBTreePointSearchCursor searchCursor = new LSMBTreePointSearchCursor(ctx);
- IIndexCursor memCursor = new BTreeRangeSearchCursor(ctx.memBTreeOpCtx.leafFrame, false);
+ IIndexCursor memCursor = new BTreeRangeSearchCursor(ctx.currentMutableBTreeOpCtx.leafFrame, false);
RangePredicate predicate = new RangePredicate(tuple, tuple, true, true, comparator, comparator);
// first check the inmemory component
- ctx.memBTreeAccessor.search(memCursor, predicate);
+ ctx.currentMutableBTreeAccessor.search(memCursor, predicate);
try {
if (memCursor.hasNext()) {
memCursor.next();
@@ -282,7 +311,8 @@
throw new TreeIndexDuplicateKeyException("Failed to insert key since key already exists.");
} else {
memCursor.close();
- ctx.memBTreeAccessor.upsertIfConditionElseInsert(tuple, AntimatterAwareTupleAcceptor.INSTANCE);
+ ctx.currentMutableBTreeAccessor.upsertIfConditionElseInsert(tuple,
+ AntimatterAwareTupleAcceptor.INSTANCE);
return true;
}
}
@@ -296,6 +326,9 @@
// the mutable component?
// the key was not in the inmemory component, so check the disk
// components
+
+ // This is a hack to avoid searching the current active mutable component twice. It is critical to add it back once the search is over.
+ ILSMComponent firstComponent = ctx.getComponentHolder().remove(0);
search(ctx, searchCursor, predicate);
try {
if (searchCursor.hasNext()) {
@@ -303,9 +336,11 @@
}
} finally {
searchCursor.close();
+ // Add the current active mutable component back
+ ctx.getComponentHolder().add(0, firstComponent);
}
- ctx.memBTreeAccessor.upsertIfConditionElseInsert(tuple, AntimatterAwareTupleAcceptor.INSTANCE);
+ ctx.currentMutableBTreeAccessor.upsertIfConditionElseInsert(tuple, AntimatterAwareTupleAcceptor.INSTANCE);
return true;
}
@@ -314,38 +349,30 @@
throws HyracksDataException, IndexException {
LSMBTreeOpContext ctx = (LSMBTreeOpContext) ictx;
List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
- int numBTrees = operationalComponents.size();
- assert numBTrees > 0;
- boolean includeMutableComponent = operationalComponents.get(0) == mutableComponent;
- LSMBTreeCursorInitialState initialState = new LSMBTreeCursorInitialState(numBTrees, insertLeafFrameFactory,
- ctx.cmp, ctx.bloomFilterCmp, includeMutableComponent, lsmHarness, ctx.memBTreeAccessor, pred,
- ctx.searchCallback, operationalComponents);
+ LSMBTreeCursorInitialState initialState = new LSMBTreeCursorInitialState(insertLeafFrameFactory, ctx.cmp,
+ ctx.bloomFilterCmp, lsmHarness, pred, ctx.searchCallback, operationalComponents);
cursor.open(initialState, pred);
}
@Override
- public boolean scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+ public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException {
- if (!mutableComponent.isModified()) {
- return false;
- }
+ ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
LSMBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
assert ctx.getComponentHolder().size() == 1;
- ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
opCtx.setOperation(IndexOperation.FLUSH);
opCtx.getComponentHolder().add(flushingComponent);
ILSMIndexAccessorInternal flushAccessor = new LSMBTreeAccessor(lsmHarness, opCtx);
ioScheduler.scheduleOperation(new LSMBTreeFlushOperation(flushAccessor, flushingComponent, componentFileRefs
.getInsertIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), callback));
- return true;
}
@Override
public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
LSMBTreeFlushOperation flushOp = (LSMBTreeFlushOperation) operation;
- LSMBTreeMutableComponent flushingComponent = (LSMBTreeMutableComponent) flushOp.getFlushingComponent();
+ LSMBTreeMemoryComponent flushingComponent = (LSMBTreeMemoryComponent) flushOp.getFlushingComponent();
IIndexAccessor accessor = flushingComponent.getBTree().createAccessor(NoOpOperationCallback.INSTANCE,
NoOpOperationCallback.INSTANCE);
@@ -368,7 +395,7 @@
BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
bloomFilterFalsePositiveRate);
- LSMBTreeImmutableComponent component = createDiskComponent(componentFactory, flushOp.getBTreeFlushTarget(),
+ LSMBTreeDiskComponent component = createDiskComponent(componentFactory, flushOp.getBTreeFlushTarget(),
flushOp.getBloomFilterFlushTarget(), true);
IIndexBulkLoader bulkLoader = component.getBTree().createBulkLoader(1.0f, false, numElements, false);
IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numElements,
@@ -390,18 +417,16 @@
return component;
}
+ @Override
public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException, IndexException {
LSMBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
- opCtx.getComponentHolder().addAll(mergingComponents);
- ITreeIndexCursor cursor = new LSMBTreeRangeSearchCursor(opCtx);
- RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
- search(opCtx, cursor, rangePred);
-
opCtx.setOperation(IndexOperation.MERGE);
- BTree firstBTree = (BTree) ((LSMBTreeImmutableComponent) mergingComponents.get(0)).getBTree();
- BTree lastBTree = (BTree) ((LSMBTreeImmutableComponent) mergingComponents.get(mergingComponents.size() - 1))
+ List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+ ITreeIndexCursor cursor = new LSMBTreeRangeSearchCursor(opCtx);
+
+ BTree firstBTree = (BTree) ((LSMBTreeDiskComponent) mergingComponents.get(0)).getBTree();
+ BTree lastBTree = (BTree) ((LSMBTreeDiskComponent) mergingComponents.get(mergingComponents.size() - 1))
.getBTree();
FileReference firstFile = diskFileMapProvider.lookupFileName(firstBTree.getFileId());
FileReference lastFile = diskFileMapProvider.lookupFileName(lastBTree.getFileId());
@@ -413,21 +438,24 @@
}
@Override
- public ILSMComponent merge(List<ILSMComponent> mergedComponents, ILSMIOOperation operation)
- throws HyracksDataException, IndexException {
+ public ILSMComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException {
LSMBTreeMergeOperation mergeOp = (LSMBTreeMergeOperation) operation;
ITreeIndexCursor cursor = mergeOp.getCursor();
- mergedComponents.addAll(mergeOp.getMergingComponents());
+ RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
+ ILSMIndexOperationContext opCtx = ((LSMIndexSearchCursor) cursor).getOpCtx();
+ opCtx.getComponentHolder().addAll(mergeOp.getMergingComponents());
+ search(opCtx, cursor, rangePred);
+ List<ILSMComponent> mergedComponents = mergeOp.getMergingComponents();
long numElements = 0L;
for (int i = 0; i < mergedComponents.size(); ++i) {
- numElements += ((LSMBTreeImmutableComponent) mergedComponents.get(i)).getBloomFilter().getNumElements();
+ numElements += ((LSMBTreeDiskComponent) mergedComponents.get(i)).getBloomFilter().getNumElements();
}
int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
bloomFilterFalsePositiveRate);
- LSMBTreeImmutableComponent mergedComponent = createDiskComponent(componentFactory,
+ LSMBTreeDiskComponent mergedComponent = createDiskComponent(componentFactory,
mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
IIndexBulkLoader bulkLoader = mergedComponent.getBTree().createBulkLoader(1.0f, false, numElements, false);
@@ -448,11 +476,11 @@
return mergedComponent;
}
- private LSMBTreeImmutableComponent createDiskComponent(LSMBTreeImmutableComponentFactory factory,
+ private LSMBTreeDiskComponent createDiskComponent(LSMBTreeDiskComponentFactory factory,
FileReference btreeFileRef, FileReference bloomFilterFileRef, boolean createComponent)
throws HyracksDataException, IndexException {
// Create new BTree instance.
- LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) factory
+ LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) factory
.createLSMComponentInstance(new LSMComponentFileReferences(btreeFileRef, null, bloomFilterFileRef));
if (createComponent) {
component.getBTree().create();
@@ -484,7 +512,7 @@
public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException {
// The order of forcing the dirty page to be flushed is critical. The
// bloom filter must be always done first.
- LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) lsmComponent;
+ LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) lsmComponent;
// Flush the bloom filter first.
int fileId = component.getBloomFilter().getFileId();
IBufferCache bufferCache = component.getBTree().getBufferCache();
@@ -513,13 +541,13 @@
} catch (HyracksDataException | IndexException e) {
throw new TreeIndexException(e);
}
- bulkLoader = (BTreeBulkLoader) ((LSMBTreeImmutableComponent) component).getBTree().createBulkLoader(
+ bulkLoader = (BTreeBulkLoader) ((LSMBTreeDiskComponent) component).getBTree().createBulkLoader(
fillFactor, verifyInput, numElementsHint, false);
int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
bloomFilterFalsePositiveRate);
- builder = ((LSMBTreeImmutableComponent) component).getBloomFilter().createBuilder(numElementsHint,
+ builder = ((LSMBTreeDiskComponent) component).getBloomFilter().createBuilder(numElementsHint,
bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
}
@@ -545,10 +573,10 @@
builder.end();
endedBloomFilterLoad = true;
}
- ((LSMBTreeImmutableComponent) component).getBTree().deactivate();
- ((LSMBTreeImmutableComponent) component).getBTree().destroy();
- ((LSMBTreeImmutableComponent) component).getBloomFilter().deactivate();
- ((LSMBTreeImmutableComponent) component).getBloomFilter().destroy();
+ ((LSMBTreeDiskComponent) component).getBTree().deactivate();
+ ((LSMBTreeDiskComponent) component).getBTree().destroy();
+ ((LSMBTreeDiskComponent) component).getBloomFilter().deactivate();
+ ((LSMBTreeDiskComponent) component).getBloomFilter().destroy();
}
}
@@ -571,7 +599,7 @@
public LSMBTreeOpContext createOpContext(IModificationOperationCallback modificationCallback,
ISearchOperationCallback searchCallback) {
- return new LSMBTreeOpContext(mutableComponent.getBTree(), insertLeafFrameFactory, deleteLeafFrameFactory,
+ return new LSMBTreeOpContext(memoryComponents, insertLeafFrameFactory, deleteLeafFrameFactory,
modificationCallback, searchCallback, componentFactory.getBloomFilterKeyFields().length);
}
@@ -608,50 +636,66 @@
@Override
public ITreeIndexFrameFactory getInteriorFrameFactory() {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getBTree().getInteriorFrameFactory();
}
@Override
public int getFieldCount() {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getBTree().getFieldCount();
}
@Override
public int getFileId() {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getBTree().getFileId();
}
@Override
public IFreePageManager getFreePageManager() {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getBTree().getFreePageManager();
}
@Override
public ITreeIndexFrameFactory getLeafFrameFactory() {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getBTree().getLeafFrameFactory();
}
@Override
public long getMemoryAllocationSize() {
- IBufferCache virtualBufferCache = mutableComponent.getBTree().getBufferCache();
- return virtualBufferCache.getNumPages() * virtualBufferCache.getPageSize();
+ long size = 0;
+ for (ILSMComponent c : memoryComponents) {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
+ IBufferCache virtualBufferCache = mutableComponent.getBTree().getBufferCache();
+ size += virtualBufferCache.getNumPages() * virtualBufferCache.getPageSize();
+ }
+ return size;
}
@Override
public int getRootPageId() {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getBTree().getRootPageId();
}
- public boolean isEmptyIndex() throws HyracksDataException {
- return componentsRef.get().isEmpty() && !mutableComponent.isModified();
- }
-
@Override
public void validate() throws HyracksDataException {
- mutableComponent.getBTree().validate();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ for (ILSMComponent c : memoryComponents) {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
+ mutableComponent.getBTree().validate();
+ }
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- BTree btree = (BTree) ((LSMBTreeImmutableComponent) c).getBTree();
+ BTree btree = (BTree) ((LSMBTreeDiskComponent) c).getBTree();
btree.validate();
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java
index b5c6a45..37f89ff 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java
@@ -18,7 +18,6 @@
import java.util.List;
import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
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.ITreeIndexFrameFactory;
@@ -29,39 +28,28 @@
public class LSMBTreeCursorInitialState implements ICursorInitialState {
- private final int numBTrees;
private final ITreeIndexFrameFactory leafFrameFactory;
private MultiComparator cmp;
private final MultiComparator bloomFilterCmp;
- private final boolean includeMemComponent;
private final ILSMHarness lsmHarness;
- private final IIndexAccessor memBtreeAccessor;
private final ISearchPredicate predicate;
private ISearchOperationCallback searchCallback;
private final List<ILSMComponent> operationalComponents;
- public LSMBTreeCursorInitialState(int numBTrees, ITreeIndexFrameFactory leafFrameFactory, MultiComparator cmp,
- MultiComparator bloomFilterCmp, boolean includeMemComponent, ILSMHarness lsmHarness,
- IIndexAccessor memBtreeAccessor, ISearchPredicate predicate, ISearchOperationCallback searchCallback,
- List<ILSMComponent> operationalComponents) {
- this.numBTrees = numBTrees;
+ public LSMBTreeCursorInitialState(ITreeIndexFrameFactory leafFrameFactory, MultiComparator cmp,
+ MultiComparator bloomFilterCmp, ILSMHarness lsmHarness, ISearchPredicate predicate,
+ ISearchOperationCallback searchCallback, List<ILSMComponent> operationalComponents) {
this.leafFrameFactory = leafFrameFactory;
this.cmp = cmp;
this.bloomFilterCmp = bloomFilterCmp;
- this.includeMemComponent = includeMemComponent;
this.lsmHarness = lsmHarness;
this.searchCallback = searchCallback;
- this.memBtreeAccessor = memBtreeAccessor;
this.predicate = predicate;
this.operationalComponents = operationalComponents;
}
- public int getNumBTrees() {
- return numBTrees;
- }
-
public ITreeIndexFrameFactory getLeafFrameFactory() {
return leafFrameFactory;
}
@@ -75,10 +63,6 @@
public void setPage(ICachedPage page) {
}
- public boolean getIncludeMemComponent() {
- return includeMemComponent;
- }
-
public ILSMHarness getLSMHarness() {
return lsmHarness;
}
@@ -97,10 +81,6 @@
return operationalComponents;
}
- public IIndexAccessor getMemBTreeAccessor() {
- return memBtreeAccessor;
- }
-
public ISearchPredicate getSearchPredicate() {
return predicate;
}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponent.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
similarity index 84%
rename from hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponent.java
rename to hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
index affdef8..0b2d7cf 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponent.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
@@ -17,13 +17,13 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractImmutableLSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractDiskLSMComponent;
-public class LSMBTreeImmutableComponent extends AbstractImmutableLSMComponent {
+public class LSMBTreeDiskComponent extends AbstractDiskLSMComponent {
private final BTree btree;
private final BloomFilter bloomFilter;
- public LSMBTreeImmutableComponent(BTree btree, BloomFilter bloomFilter) {
+ public LSMBTreeDiskComponent(BTree btree, BloomFilter bloomFilter) {
this.btree = btree;
this.bloomFilter = bloomFilter;
}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponentFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java
similarity index 86%
rename from hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponentFactory.java
rename to hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java
index 0efdc65..7b1e0de 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponentFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java
@@ -25,11 +25,11 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-public class LSMBTreeImmutableComponentFactory implements ILSMComponentFactory {
+public class LSMBTreeDiskComponentFactory implements ILSMComponentFactory {
private final TreeIndexFactory<BTree> btreeFactory;
private final BloomFilterFactory bloomFilterFactory;
- public LSMBTreeImmutableComponentFactory(TreeIndexFactory<BTree> btreeFactory, BloomFilterFactory bloomFilterFactory) {
+ public LSMBTreeDiskComponentFactory(TreeIndexFactory<BTree> btreeFactory, BloomFilterFactory bloomFilterFactory) {
this.btreeFactory = btreeFactory;
this.bloomFilterFactory = bloomFilterFactory;
}
@@ -37,7 +37,7 @@
@Override
public ILSMComponent createLSMComponentInstance(LSMComponentFileReferences cfr) throws IndexException,
HyracksDataException {
- return new LSMBTreeImmutableComponent(btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+ return new LSMBTreeDiskComponent(btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()));
}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
index 1aab213..668e727 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
@@ -58,8 +58,9 @@
}
@Override
- public void perform() throws HyracksDataException, IndexException {
+ public Boolean call() throws HyracksDataException, IndexException {
accessor.flush(this);
+ return true;
}
@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/LSMBTreeMemoryComponent.java
similarity index 79%
rename from hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMutableComponent.java
rename to hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
index 0f3e3ce..7550349 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/LSMBTreeMemoryComponent.java
@@ -18,16 +18,15 @@
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.lsm.common.api.IVirtualBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMutableLSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMemoryLSMComponent;
-public class LSMBTreeMutableComponent extends AbstractMutableLSMComponent {
+public class LSMBTreeMemoryComponent extends AbstractMemoryLSMComponent {
private final BTree btree;
- private final IVirtualBufferCache vbc;
- public LSMBTreeMutableComponent(BTree btree, IVirtualBufferCache vbc) {
+ public LSMBTreeMemoryComponent(BTree btree, IVirtualBufferCache vbc, boolean isActive) {
+ super(vbc, isActive);
this.btree = btree;
- this.vbc = vbc;
}
public BTree getBTree() {
@@ -35,11 +34,6 @@
}
@Override
- protected boolean isFull() {
- return vbc.isFull();
- }
-
- @Override
protected void reset() throws HyracksDataException {
super.reset();
btree.deactivate();
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java
index fd3f1fd..3a608fe 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java
@@ -53,7 +53,7 @@
public Set<IODeviceHandle> getReadDevices() {
Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
for (ILSMComponent o : mergingComponents) {
- LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) o;
+ LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) o;
devs.add(component.getBTree().getFileReference().getDeviceHandle());
devs.add(component.getBloomFilter().getFileReference().getDeviceHandle());
}
@@ -69,8 +69,9 @@
}
@Override
- public void perform() throws HyracksDataException, IndexException {
+ public Boolean call() throws HyracksDataException, IndexException {
accessor.merge(this);
+ return true;
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index ac62d9f..6d2d7c0 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -37,9 +37,11 @@
public ITreeIndexFrameFactory deleteLeafFrameFactory;
public IBTreeLeafFrame insertLeafFrame;
public IBTreeLeafFrame deleteLeafFrame;
- public final BTree memBTree;
- public BTree.BTreeAccessor memBTreeAccessor;
- public BTreeOpContext memBTreeOpCtx;
+ public final BTree[] mutableBTrees;
+ public BTree.BTreeAccessor[] mutableBTreeAccessors;
+ public BTreeOpContext[] mutableBTreeOpCtxs;
+ public BTree.BTreeAccessor currentMutableBTreeAccessor;
+ public BTreeOpContext currentMutableBTreeOpCtx;
public IndexOperation op;
public final MultiComparator cmp;
public final MultiComparator bloomFilterCmp;
@@ -47,19 +49,30 @@
public final ISearchOperationCallback searchCallback;
private final List<ILSMComponent> componentHolder;
- public LSMBTreeOpContext(BTree memBTree, ITreeIndexFrameFactory insertLeafFrameFactory,
+ public LSMBTreeOpContext(List<ILSMComponent> mutableComponents, ITreeIndexFrameFactory insertLeafFrameFactory,
ITreeIndexFrameFactory deleteLeafFrameFactory, IModificationOperationCallback modificationCallback,
ISearchOperationCallback searchCallback, int numBloomFilterKeyFields) {
- IBinaryComparatorFactory cmpFactories[] = memBTree.getComparatorFactories();
+ LSMBTreeMemoryComponent c = (LSMBTreeMemoryComponent) mutableComponents.get(0);
+ IBinaryComparatorFactory cmpFactories[] = c.getBTree().getComparatorFactories();
if (cmpFactories[0] != null) {
- this.cmp = MultiComparator.create(memBTree.getComparatorFactories());
+ this.cmp = MultiComparator.create(c.getBTree().getComparatorFactories());
} else {
this.cmp = null;
}
- bloomFilterCmp = MultiComparator.create(memBTree.getComparatorFactories(), 0, numBloomFilterKeyFields);
+ bloomFilterCmp = MultiComparator.create(c.getBTree().getComparatorFactories(), 0, numBloomFilterKeyFields);
- this.memBTree = memBTree;
+ mutableBTrees = new BTree[mutableComponents.size()];
+ mutableBTreeAccessors = new BTree.BTreeAccessor[mutableComponents.size()];
+ mutableBTreeOpCtxs = new BTreeOpContext[mutableComponents.size()];
+ for (int i = 0; i < mutableComponents.size(); i++) {
+ LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) mutableComponents.get(i);
+ mutableBTrees[i] = mutableComponent.getBTree();
+ mutableBTreeAccessors[i] = (BTree.BTreeAccessor) mutableBTrees[i].createAccessor(modificationCallback,
+ NoOpOperationCallback.INSTANCE);
+ mutableBTreeOpCtxs[i] = mutableBTreeAccessors[i].getOpContext();
+ }
+
this.insertLeafFrameFactory = insertLeafFrameFactory;
this.deleteLeafFrameFactory = deleteLeafFrameFactory;
this.insertLeafFrame = (IBTreeLeafFrame) insertLeafFrameFactory.createFrame();
@@ -79,48 +92,16 @@
public void setOperation(IndexOperation newOp) {
reset();
this.op = newOp;
- switch (newOp) {
- case SEARCH:
- setMemBTreeAccessor();
- break;
- case DISKORDERSCAN:
- case UPDATE:
- // Attention: It is important to leave the leafFrame and
- // leafFrameFactory of the memBTree as is when doing an update.
- // Update will only be set if a previous attempt to delete or
- // insert failed, so we must preserve the semantics of the
- // previously requested operation.
- setMemBTreeAccessor();
- return;
- case UPSERT:
- case INSERT:
- setInsertMode();
- break;
- case PHYSICALDELETE:
- case DELETE:
- setDeleteMode();
- break;
- }
- }
-
- private void setMemBTreeAccessor() {
- if (memBTreeAccessor == null) {
- memBTreeAccessor = (BTree.BTreeAccessor) memBTree.createAccessor(modificationCallback,
- NoOpOperationCallback.INSTANCE);
- memBTreeOpCtx = memBTreeAccessor.getOpContext();
- }
}
public void setInsertMode() {
- setMemBTreeAccessor();
- memBTreeOpCtx.leafFrame = insertLeafFrame;
- memBTreeOpCtx.leafFrameFactory = insertLeafFrameFactory;
+ currentMutableBTreeOpCtx.leafFrame = insertLeafFrame;
+ currentMutableBTreeOpCtx.leafFrameFactory = insertLeafFrameFactory;
}
public void setDeleteMode() {
- setMemBTreeAccessor();
- memBTreeOpCtx.leafFrame = deleteLeafFrame;
- memBTreeOpCtx.leafFrameFactory = deleteLeafFrameFactory;
+ currentMutableBTreeOpCtx.leafFrame = deleteLeafFrame;
+ currentMutableBTreeOpCtx.leafFrameFactory = deleteLeafFrameFactory;
}
@Override
@@ -146,4 +127,30 @@
public IModificationOperationCallback getModificationCallback() {
return modificationCallback;
}
+
+ @Override
+ public void setCurrentMutableComponentId(int currentMutableComponentId) {
+ currentMutableBTreeAccessor = mutableBTreeAccessors[currentMutableComponentId];
+ currentMutableBTreeOpCtx = mutableBTreeOpCtxs[currentMutableComponentId];
+ switch (op) {
+ case SEARCH:
+ break;
+ case DISKORDERSCAN:
+ case UPDATE:
+ // Attention: It is important to leave the leafFrame and
+ // leafFrameFactory of the mutableBTree as is when doing an update.
+ // Update will only be set if a previous attempt to delete or
+ // insert failed, so we must preserve the semantics of the
+ // previously requested operation.
+ break;
+ case UPSERT:
+ case INSERT:
+ setInsertMode();
+ break;
+ case PHYSICALDELETE:
+ case DELETE:
+ setDeleteMode();
+ break;
+ }
+ }
}
\ No newline at end of file
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 64e0c28..fc09a74 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
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
-import java.util.ListIterator;
+import java.util.List;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -32,6 +32,7 @@
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.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
@@ -45,14 +46,14 @@
private final ILSMIndexOperationContext opCtx;
private ISearchOperationCallback searchCallback;
private RangePredicate predicate;
- private IIndexAccessor memBTreeAccessor;
- private boolean includeMemComponent;
+ private boolean includeMutableComponent;
private int numBTrees;
- private IIndexAccessor[] bTreeAccessors;
+ private IIndexAccessor[] btreeAccessors;
private ILSMHarness lsmHarness;
private boolean nextHasBeenCalled;
private boolean foundTuple;
private ITupleReference frameTuple;
+ private List<ILSMComponent> operationalComponents;
public LSMBTreePointSearchCursor(ILSMIndexOperationContext opCtx) {
this.opCtx = opCtx;
@@ -67,7 +68,7 @@
}
boolean reconciled = false;
for (int i = 0; i < numBTrees; ++i) {
- bTreeAccessors[i].search(rangeCursors[i], predicate);
+ btreeAccessors[i].search(rangeCursors[i], predicate);
if (rangeCursors[i].hasNext()) {
rangeCursors[i].next();
// We use the predicate's to lock the key instead of the tuple that we get from cursor to avoid copying the tuple when we do the "unlatch dance"
@@ -83,14 +84,14 @@
return true;
}
}
- if (i == 0 && includeMemComponent) {
+ if (i == 0 && includeMutableComponent) {
// unlatch/unpin
rangeCursors[i].reset();
searchCallback.reconcile(predicate.getLowKey());
reconciled = true;
// retraverse
- memBTreeAccessor.search(rangeCursors[i], predicate);
+ btreeAccessors[0].search(rangeCursors[i], predicate);
searchCallback.complete(predicate.getLowKey());
if (rangeCursors[i].hasNext()) {
rangeCursors[i].next();
@@ -123,14 +124,14 @@
@Override
public void reset() throws HyracksDataException, IndexException {
try {
- 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;
- nextHasBeenCalled = false;
- foundTuple = false;
+ rangeCursors = null;
+ nextHasBeenCalled = false;
+ foundTuple = false;
} finally {
if (lsmHarness != null) {
lsmHarness.endSearch(opCtx);
@@ -141,41 +142,30 @@
@Override
public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
- includeMemComponent = lsmInitialState.getIncludeMemComponent();
+ operationalComponents = lsmInitialState.getOperationalComponents();
lsmHarness = lsmInitialState.getLSMHarness();
searchCallback = lsmInitialState.getSearchOperationCallback();
- memBTreeAccessor = lsmInitialState.getMemBTreeAccessor();
predicate = (RangePredicate) lsmInitialState.getSearchPredicate();
-
- numBTrees = lsmInitialState.getNumBTrees();
+ numBTrees = operationalComponents.size();
rangeCursors = new IIndexCursor[numBTrees];
- int i = 0;
- if (includeMemComponent) {
- // No need for a bloom filter for the in-memory BTree.
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getLeafFrameFactory().createFrame();
- rangeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
- ++i;
- }
- for (; i < numBTrees; ++i) {
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getLeafFrameFactory().createFrame();
- rangeCursors[i] = new BloomFilterAwareBTreePointSearchCursor(leafFrame, false,
- ((LSMBTreeImmutableComponent) lsmInitialState.getOperationalComponents().get(i)).getBloomFilter());
- }
+ btreeAccessors = new IIndexAccessor[numBTrees];
+ includeMutableComponent = false;
- bTreeAccessors = new IIndexAccessor[numBTrees];
- int cursorIx = 0;
- ListIterator<ILSMComponent> btreesIter = lsmInitialState.getOperationalComponents().listIterator();
- if (includeMemComponent) {
- bTreeAccessors[cursorIx] = memBTreeAccessor;
- ++cursorIx;
- btreesIter.next();
- }
-
- while (btreesIter.hasNext()) {
- BTree diskBTree = ((LSMBTreeImmutableComponent) btreesIter.next()).getBTree();
- bTreeAccessors[cursorIx] = diskBTree.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- cursorIx++;
+ for (int i = 0; i < numBTrees; i++) {
+ ILSMComponent component = operationalComponents.get(i);
+ BTree btree;
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getLeafFrameFactory().createFrame();
+ if (component.getType() == LSMComponentType.MEMORY) {
+ includeMutableComponent = true;
+ // No need for a bloom filter for the in-memory BTree.
+ rangeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
+ btree = (BTree) ((LSMBTreeMemoryComponent) component).getBTree();
+ } else {
+ rangeCursors[i] = new BloomFilterAwareBTreePointSearchCursor(leafFrame, false,
+ ((LSMBTreeDiskComponent) component).getBloomFilter());
+ btree = (BTree) ((LSMBTreeDiskComponent) component).getBTree();
+ }
+ btreeAccessors[i] = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
}
nextHasBeenCalled = false;
foundTuple = false;
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 4bdebae..6eada4b 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -16,7 +16,6 @@
package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
import java.util.Iterator;
-import java.util.ListIterator;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -35,6 +34,7 @@
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.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
@@ -44,7 +44,7 @@
private ISearchOperationCallback searchCallback;
private RangePredicate predicate;
- private IIndexAccessor memBTreeAccessor;
+ private IIndexAccessor[] btreeAccessors;
private ArrayTupleBuilder tupleBuilder;
private boolean proceed = true;
@@ -72,25 +72,25 @@
if (!outputPriorityQueue.isEmpty()) {
PriorityQueueElement checkElement = outputPriorityQueue.peek();
if (proceed && !searchCallback.proceed(checkElement.getTuple())) {
- if (includeMemComponent) {
- PriorityQueueElement inMemElement = null;
- boolean inMemElementFound = false;
- // scan the PQ for the in-memory component's element
+ if (includeMutableComponent) {
+ PriorityQueueElement mutableElement = null;
+ boolean mutableElementFound = false;
+ // scan the PQ for the mutable component's element
Iterator<PriorityQueueElement> it = outputPriorityQueue.iterator();
while (it.hasNext()) {
- inMemElement = it.next();
- if (inMemElement.getCursorIndex() == 0) {
- inMemElementFound = true;
+ mutableElement = it.next();
+ if (mutableElement.getCursorIndex() == 0) {
+ mutableElementFound = true;
it.remove();
break;
}
}
- if (inMemElementFound) {
+ if (mutableElementFound) {
// copy the in-mem tuple
if (tupleBuilder == null) {
tupleBuilder = new ArrayTupleBuilder(cmp.getKeyFieldCount());
}
- TupleUtils.copyTuple(tupleBuilder, inMemElement.getTuple(), cmp.getKeyFieldCount());
+ TupleUtils.copyTuple(tupleBuilder, mutableElement.getTuple(), cmp.getKeyFieldCount());
copyTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
// unlatch/unpin
@@ -105,11 +105,11 @@
}
// retraverse
reusablePred.setLowKey(copyTuple, true);
- memBTreeAccessor.search(rangeCursors[0], reusablePred);
- boolean isNotExhaustedCursor = pushIntoPriorityQueue(inMemElement);
-
+ btreeAccessors[0].search(rangeCursors[0], reusablePred);
+ boolean isNotExhaustedCursor = pushIntoPriorityQueue(mutableElement);
+
if (checkElement.getCursorIndex() == 0) {
- if (!isNotExhaustedCursor || cmp.compare(copyTuple, inMemElement.getTuple()) != 0) {
+ if (!isNotExhaustedCursor || cmp.compare(copyTuple, mutableElement.getTuple()) != 0) {
searchCallback.complete(copyTuple);
searchCallback.cancel(copyTuple);
continue;
@@ -117,7 +117,7 @@
searchCallback.complete(copyTuple);
}
} else {
- // the in-memory cursor is exhausted
+ // the mutable cursor is exhausted
searchCallback.reconcile(checkElement.getTuple());
}
} else {
@@ -174,46 +174,34 @@
IndexException {
LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
cmp = lsmInitialState.getOriginalKeyComparator();
- includeMemComponent = lsmInitialState.getIncludeMemComponent();
operationalComponents = lsmInitialState.getOperationalComponents();
lsmHarness = lsmInitialState.getLSMHarness();
searchCallback = lsmInitialState.getSearchOperationCallback();
- memBTreeAccessor = lsmInitialState.getMemBTreeAccessor();
predicate = (RangePredicate) lsmInitialState.getSearchPredicate();
reusablePred.setLowKeyComparator(cmp);
reusablePred.setHighKey(predicate.getHighKey(), predicate.isHighKeyInclusive());
reusablePred.setHighKeyComparator(predicate.getHighKeyComparator());
+ includeMutableComponent = false;
- int numBTrees = lsmInitialState.getNumBTrees();
+ int numBTrees = operationalComponents.size();
rangeCursors = new IIndexCursor[numBTrees];
+
+ btreeAccessors = new ITreeIndexAccessor[numBTrees];
for (int i = 0; i < numBTrees; i++) {
+ ILSMComponent component = operationalComponents.get(i);
+ BTree btree;
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getLeafFrameFactory().createFrame();
rangeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
+ if (component.getType() == LSMComponentType.MEMORY) {
+ includeMutableComponent = true;
+ btree = (BTree) ((LSMBTreeMemoryComponent) component).getBTree();
+ } else {
+ btree = (BTree) ((LSMBTreeDiskComponent) component).getBTree();
+ }
+ btreeAccessors[i] = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ btreeAccessors[i].search(rangeCursors[i], searchPred);
}
setPriorityQueueComparator();
-
- int cursorIx = 0;
- ListIterator<ILSMComponent> btreesIter = operationalComponents.listIterator();
- if (includeMemComponent) {
- // Open cursor of in-memory BTree at index 0.
- memBTreeAccessor.search(rangeCursors[cursorIx], searchPred);
- // Skip 0 because it is the in-memory BTree.
- ++cursorIx;
- btreesIter.next();
- }
-
- // Open cursors of on-disk BTrees.
- int numDiskComponents = includeMemComponent ? numBTrees - 1 : numBTrees;
- ITreeIndexAccessor[] diskBTreeAccessors = new ITreeIndexAccessor[numDiskComponents];
- int diskBTreeIx = 0;
- while (btreesIter.hasNext()) {
- BTree diskBTree = (BTree) ((LSMBTreeImmutableComponent) btreesIter.next()).getBTree();
- diskBTreeAccessors[diskBTreeIx] = diskBTree.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- diskBTreeAccessors[diskBTreeIx].search(rangeCursors[cursorIx], searchPred);
- cursorIx++;
- diskBTreeIx++;
- }
initPriorityQueue();
proceed = true;
}
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 507ec02..f43c8a6 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
@@ -15,6 +15,8 @@
package edu.uci.ics.hyracks.storage.am.lsm.btree.util;
+import java.util.List;
+
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;
@@ -43,7 +45,7 @@
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public class LSMBTreeUtils {
- public static LSMBTree createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+ public static LSMBTree createLSMTree(List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
@@ -73,7 +75,7 @@
ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(diskFileMapProvider, file, diskBTreeFactory);
- LSMBTree lsmTree = new LSMBTree(virtualBufferCache, interiorFrameFactory, insertLeafFrameFactory,
+ LSMBTree lsmTree = new LSMBTree(virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length, cmpFactories, mergePolicy,
opTracker, ioScheduler, ioOpCallbackProvider);
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMComponent.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMComponent.java
index 72a3e1f..afbabe0 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMComponent.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMComponent.java
@@ -18,7 +18,27 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
public interface ILSMComponent {
- public boolean threadEnter(LSMOperationType opType) throws InterruptedException;
- public void threadExit(LSMOperationType opType, boolean failedOperation) throws HyracksDataException;
+ enum LSMComponentType {
+ MEMORY,
+ DISK
+ }
+
+ enum ComponentState {
+ INACTIVE,
+ READABLE_WRITABLE,
+ READABLE_UNWRITABLE,
+ READABLE_UNWRITABLE_FLUSHING,
+ UNREADABLE_UNWRITABLE,
+ READABLE_MERGING
+ }
+
+ public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) throws HyracksDataException;
+
+ public void threadExit(LSMOperationType opType, boolean failedOperation, boolean isMutableComponent)
+ throws HyracksDataException;
+
+ public LSMComponentType getType();
+
+ public ComponentState getState();
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMHarness.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMHarness.java
index a006b97..bc7cbf7 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMHarness.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMHarness.java
@@ -29,8 +29,6 @@
public boolean modify(ILSMIndexOperationContext ctx, boolean tryOperation, ITupleReference tuple)
throws HyracksDataException, IndexException;
- public void noOp(ILSMIndexOperationContext ctx) throws HyracksDataException;
-
public void search(ILSMIndexOperationContext ctx, IIndexCursor cursor, ISearchPredicate pred)
throws HyracksDataException, IndexException;
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
index c23dcea..2c3940f 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
@@ -15,17 +15,18 @@
package edu.uci.ics.hyracks.storage.am.lsm.common.api;
import java.util.Set;
+import java.util.concurrent.Callable;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-public interface ILSMIOOperation {
+public interface ILSMIOOperation extends Callable<Boolean> {
public Set<IODeviceHandle> getReadDevices();
public Set<IODeviceHandle> getWriteDevices();
- public void perform() throws HyracksDataException, IndexException;
+ public Boolean call() throws HyracksDataException, IndexException;
public ILSMIOOperationCallback getCallback();
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
index 105b53a..416a15b 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
@@ -32,13 +32,11 @@
* concurrent searches/updates/merges may be ongoing.
*/
public interface ILSMIndex extends IIndex {
-
+
public void deactivate(boolean flushOnExit) throws HyracksDataException;
public ILSMIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
- ISearchOperationCallback searchCallback);
-
- public boolean getFlushStatus();
+ ISearchOperationCallback searchCallback) throws HyracksDataException;
public ILSMOperationTracker getOperationTracker();
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
index cd60ded..3405b60 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
@@ -109,14 +109,4 @@
public void forceInsert(ITupleReference tuple) throws HyracksDataException, IndexException;
public void forceDelete(ITupleReference tuple) throws HyracksDataException, IndexException;
-
- /**
- * This method can be used to increase the number of 'active' operations of an index artificially,
- * without actually modifying the index.
- * This method does not block and is guaranteed to trigger the {@link ILSMOperationTracker}'s beforeOperation
- * and afterOperation calls.
- *
- * @throws HyracksDataException
- */
- public void noOp() throws HyracksDataException;
}
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 c65c7cb..6020093 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
@@ -28,14 +28,14 @@
public interface ILSMIndexInternal extends ILSMIndex {
public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback,
- ISearchOperationCallback searchCallback);
+ ISearchOperationCallback searchCallback) throws HyracksDataException;
public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException;
public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
throws HyracksDataException, IndexException;
- public boolean scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+ public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException;
public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException;
@@ -43,13 +43,18 @@
public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException, IndexException;
- public ILSMComponent merge(List<ILSMComponent> mergedComponents, ILSMIOOperation operation)
- throws HyracksDataException, IndexException;
+ public ILSMComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException;
public void addComponent(ILSMComponent index);
public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents);
+ public void changeMutableComponent();
+
+ public void changeFlushStatusForCurrentMutableCompoent(boolean needsFlush);
+
+ public boolean hasFlushRequestForCurrentMutableComponent();
+
/**
* Populates the context's component holder with a snapshot of the components involved in the operation.
*
@@ -58,12 +63,6 @@
*/
public void getOperationalComponents(ILSMIndexOperationContext ctx);
- 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/ILSMIndexOperationContext.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
index bbd2ca8..fcd4037 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
@@ -26,4 +26,6 @@
public ISearchOperationCallback getSearchOperationCallback();
public IModificationOperationCallback getModificationCallback();
+
+ public void setCurrentMutableComponentId(int currentMutableComponentId);
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMMergePolicy.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMMergePolicy.java
index 5eaa6f6..1473071 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMMergePolicy.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMMergePolicy.java
@@ -19,5 +19,5 @@
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
public interface ILSMMergePolicy {
- public void diskComponentAdded(ILSMIndex index, int totalNumDiskComponents) throws HyracksDataException, IndexException;
+ public void diskComponentAdded(ILSMIndex index) throws HyracksDataException, IndexException;
}
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
index cd75df0..eff390f 100644
--- 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
@@ -15,9 +15,10 @@
package edu.uci.ics.hyracks.storage.am.lsm.common.api;
import java.io.Serializable;
+import java.util.List;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
public interface IVirtualBufferCacheProvider extends Serializable {
- public IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx);
+ public List<IVirtualBufferCache> getVirtualBufferCaches(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 e79a394..b1bf6b4 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
@@ -15,6 +15,8 @@
package edu.uci.ics.hyracks.storage.am.lsm.common.dataflow;
+import java.util.List;
+
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
@@ -30,26 +32,26 @@
protected final double bloomFilterFalsePositiveRate;
- protected final IVirtualBufferCache virtualBufferCache;
+ protected final List<IVirtualBufferCache> virtualBufferCaches;
protected final ILSMMergePolicy mergePolicy;
protected final ILSMIOOperationScheduler ioScheduler;
protected final ILSMOperationTrackerProvider opTrackerFactory;
protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IVirtualBufferCache virtualBufferCache, ILSMMergePolicy mergePolicy,
+ List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
+ this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
opTrackerFactory, ioScheduler, ioOpCallbackProvider);
}
public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
super(opDesc, ctx, partition);
- this.virtualBufferCache = virtualBufferCache;
+ this.virtualBufferCaches = virtualBufferCaches;
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/LSMIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
index 4c69634..5bf52e4 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
@@ -46,7 +46,6 @@
if (tupleFilter != null) {
frameTuple.reset(accessor, i);
if (!tupleFilter.accept(frameTuple)) {
- lsmAccessor.noOp();
continue;
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractDiskLSMComponent.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractDiskLSMComponent.java
new file mode 100644
index 0000000..bc6baeb
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractDiskLSMComponent.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2009-2013 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.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractDiskLSMComponent extends AbstractLSMComponent {
+
+ public AbstractDiskLSMComponent() {
+ super();
+ state = ComponentState.READABLE_UNWRITABLE;
+ }
+
+ @Override
+ public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) {
+ assert state != ComponentState.INACTIVE;
+
+ switch (opType) {
+ case FORCE_MODIFICATION:
+ case MODIFICATION:
+ case SEARCH:
+ readerCount++;
+ break;
+ case MERGE:
+ if (state == ComponentState.READABLE_MERGING) {
+ // This should never happen unless there are two concurrent merges that were scheduled
+ // concurrently and they have interleaving components to be merged.
+ // This should be handled properly by the merge policy, but we guard against that here anyway.
+ return false;
+ }
+ state = ComponentState.READABLE_MERGING;
+ readerCount++;
+ break;
+ default:
+ throw new UnsupportedOperationException("Unsupported operation " + opType);
+ }
+ return true;
+ }
+
+ @Override
+ public void threadExit(LSMOperationType opType, boolean failedOperation, boolean isMutableComponent)
+ throws HyracksDataException {
+ switch (opType) {
+ case MERGE:
+ // In case two merge operations were scheduled to merge an overlapping set of components, the second merge will fail and it must
+ // reset those components back to their previous state.
+ if (failedOperation) {
+ state = ComponentState.READABLE_UNWRITABLE;
+ }
+ case FORCE_MODIFICATION:
+ case MODIFICATION:
+ case SEARCH:
+ readerCount--;
+ if (readerCount == 0 && state == ComponentState.READABLE_MERGING) {
+ state = ComponentState.INACTIVE;
+ }
+ break;
+ default:
+ throw new UnsupportedOperationException("Unsupported operation " + opType);
+ }
+ assert readerCount > -1;
+ }
+
+ @Override
+ public LSMComponentType getType() {
+ return LSMComponentType.DISK;
+ }
+
+ @Override
+ public ComponentState getState() {
+ return state;
+ }
+
+ protected abstract void destroy() throws HyracksDataException;
+
+}
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
deleted file mode 100644
index 376c2e4..0000000
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractImmutableLSMComponent.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Copyright 2009-2013 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.impls;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
-
-public abstract class AbstractImmutableLSMComponent implements ILSMComponent {
-
- private ComponentState state;
- private int readerCount;
-
- private enum ComponentState {
- READABLE,
- READABLE_MERGING,
- KILLED
- }
-
- public AbstractImmutableLSMComponent() {
- state = ComponentState.READABLE;
- readerCount = 0;
- }
-
- @Override
- public synchronized boolean threadEnter(LSMOperationType opType) {
- if (state == ComponentState.KILLED) {
- return false;
- }
-
- switch (opType) {
- case FORCE_MODIFICATION:
- case MODIFICATION:
- case SEARCH:
- readerCount++;
- break;
- case MERGE:
- if (state == ComponentState.READABLE_MERGING) {
- return false;
- }
- state = ComponentState.READABLE_MERGING;
- readerCount++;
- break;
- default:
- throw new UnsupportedOperationException("Unsupported operation " + opType);
- }
- return true;
- }
-
- @Override
- public synchronized void threadExit(LSMOperationType opType, boolean failedOperation) throws HyracksDataException {
- switch (opType) {
- case MERGE:
- if (failedOperation) {
- state = ComponentState.READABLE;
- }
- case FORCE_MODIFICATION:
- case MODIFICATION:
- case SEARCH:
- readerCount--;
-
- if (readerCount == 0 && state == ComponentState.READABLE_MERGING) {
- destroy();
- state = ComponentState.KILLED;
- }
- break;
- default:
- throw new UnsupportedOperationException("Unsupported operation " + opType);
- }
- }
-
- protected abstract void destroy() throws HyracksDataException;
-
-}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
new file mode 100644
index 0000000..5dab25f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2013 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.impls;
+
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+
+public abstract class AbstractLSMComponent implements ILSMComponent {
+
+ protected ComponentState state;
+ protected int readerCount;
+
+ public AbstractLSMComponent() {
+ readerCount = 0;
+ }
+
+ @Override
+ public ComponentState getState() {
+ return state;
+ }
+}
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 d9da9e5..794f440 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
@@ -18,7 +18,8 @@
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
-import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
@@ -44,24 +45,26 @@
protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
// In-memory components.
- protected final IVirtualBufferCache virtualBufferCache;
+ protected final List<ILSMComponent> memoryComponents;
+ protected final List<IVirtualBufferCache> virtualBufferCaches;
+ protected AtomicInteger currentMutableComponentId;
// On-disk components.
protected final IBufferCache diskBufferCache;
protected final ILSMIndexFileManager fileManager;
protected final IFileMapProvider diskFileMapProvider;
- protected final AtomicReference<List<ILSMComponent>> componentsRef;
+ protected final List<ILSMComponent> diskComponents;
protected final double bloomFilterFalsePositiveRate;
protected boolean isActivated;
- private boolean needsFlush = false;
+ protected final AtomicBoolean[] flushRequests;
- public AbstractLSMIndex(IVirtualBufferCache virtualBufferCache, IBufferCache diskBufferCache,
+ public AbstractLSMIndex(List<IVirtualBufferCache> virtualBufferCaches, IBufferCache diskBufferCache,
ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- this.virtualBufferCache = virtualBufferCache;
+ this.virtualBufferCaches = virtualBufferCaches;
this.diskBufferCache = diskBufferCache;
this.diskFileMapProvider = diskFileMapProvider;
this.fileManager = fileManager;
@@ -70,8 +73,13 @@
this.ioOpCallbackProvider = ioOpCallbackProvider;
lsmHarness = new LSMHarness(this, mergePolicy, opTracker);
isActivated = false;
- componentsRef = new AtomicReference<List<ILSMComponent>>();
- componentsRef.set(new LinkedList<ILSMComponent>());
+ diskComponents = new LinkedList<ILSMComponent>();
+ memoryComponents = new ArrayList<ILSMComponent>();
+ currentMutableComponentId = new AtomicInteger();
+ flushRequests = new AtomicBoolean[virtualBufferCaches.size()];
+ for (int i = 0; i < virtualBufferCaches.size(); i++) {
+ flushRequests[i] = new AtomicBoolean();
+ }
}
protected void forceFlushDirtyPages(ITreeIndex treeIndex) throws HyracksDataException {
@@ -129,44 +137,35 @@
@Override
public void addComponent(ILSMComponent c) {
- List<ILSMComponent> oldList = componentsRef.get();
- List<ILSMComponent> newList = new ArrayList<ILSMComponent>();
- newList.add(c);
- for (ILSMComponent oc : oldList) {
- newList.add(oc);
- }
- componentsRef.set(newList);
+ diskComponents.add(0, c);
}
@Override
public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents) {
- List<ILSMComponent> oldList = componentsRef.get();
- List<ILSMComponent> newList = new ArrayList<ILSMComponent>();
- int swapIndex = oldList.indexOf(mergedComponents.get(0));
- int swapSize = mergedComponents.size();
- for (int i = 0; i < oldList.size(); i++) {
- if (i < swapIndex || i >= swapIndex + swapSize) {
- newList.add(oldList.get(i));
- } else if (i == swapIndex) {
- newList.add(newComponent);
- }
- }
- componentsRef.set(newList);
+ int swapIndex = diskComponents.indexOf(mergedComponents.get(0));
+ diskComponents.removeAll(mergedComponents);
+ diskComponents.add(swapIndex, newComponent);
+ }
+
+ @Override
+ public void changeMutableComponent() {
+ currentMutableComponentId.set((currentMutableComponentId.get() + 1) % memoryComponents.size());
+ ((AbstractMemoryLSMComponent) memoryComponents.get(currentMutableComponentId.get())).setActive();
}
@Override
public List<ILSMComponent> getImmutableComponents() {
- return componentsRef.get();
+ return diskComponents;
}
@Override
- public void setFlushStatus(boolean needsFlush) {
- this.needsFlush = needsFlush;
+ public void changeFlushStatusForCurrentMutableCompoent(boolean needsFlush) {
+ flushRequests[currentMutableComponentId.get()].set(needsFlush);
}
@Override
- public boolean getFlushStatus() {
- return needsFlush;
+ public boolean hasFlushRequestForCurrentMutableComponent() {
+ return flushRequests[currentMutableComponentId.get()].get();
}
@Override
@@ -180,15 +179,22 @@
}
@Override
- public boolean isFull() {
- return virtualBufferCache.isFull();
- }
-
- @Override
public IBufferCache getBufferCache() {
return diskBufferCache;
}
+ public boolean isEmptyIndex() throws HyracksDataException {
+ boolean isModified = false;
+ for (ILSMComponent c : memoryComponents) {
+ AbstractMemoryLSMComponent mutableComponent = (AbstractMemoryLSMComponent) c;
+ if (mutableComponent.isModified()) {
+ isModified = true;
+ break;
+ }
+ }
+ return diskComponents.isEmpty() && !isModified;
+ }
+
@Override
public String toString() {
return "LSMIndex [" + fileManager.getBaseDir() + "]";
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMemoryLSMComponent.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMemoryLSMComponent.java
new file mode 100644
index 0000000..ce4817b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMemoryLSMComponent.java
@@ -0,0 +1,178 @@
+/*
+ * Copyright 2009-2013 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.impls;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+
+public abstract class AbstractMemoryLSMComponent extends AbstractLSMComponent {
+
+ private int writerCount;
+ private final IVirtualBufferCache vbc;
+ private final AtomicBoolean isModified;
+ private boolean requestedToBeActive;
+
+ public AbstractMemoryLSMComponent(IVirtualBufferCache vbc, boolean isActive) {
+ super();
+ this.vbc = vbc;
+ writerCount = 0;
+ if (isActive) {
+ state = ComponentState.READABLE_WRITABLE;
+ } else {
+ state = ComponentState.INACTIVE;
+ }
+ isModified = new AtomicBoolean();
+ }
+
+ @Override
+ public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) throws HyracksDataException {
+ if (state == ComponentState.INACTIVE && requestedToBeActive) {
+ state = ComponentState.READABLE_WRITABLE;
+ requestedToBeActive = false;
+ }
+ switch (opType) {
+ case FORCE_MODIFICATION:
+ if (isMutableComponent) {
+ if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE) {
+ writerCount++;
+ } else {
+ return false;
+ }
+ } else {
+ if (state == ComponentState.READABLE_UNWRITABLE
+ || state == ComponentState.READABLE_UNWRITABLE_FLUSHING) {
+ readerCount++;
+ } else {
+ return false;
+ }
+ }
+ break;
+ case MODIFICATION:
+ if (isMutableComponent) {
+ if (state == ComponentState.READABLE_WRITABLE) {
+ writerCount++;
+ } else {
+ return false;
+ }
+ } else {
+ if (state == ComponentState.READABLE_UNWRITABLE
+ || state == ComponentState.READABLE_UNWRITABLE_FLUSHING) {
+ readerCount++;
+ } else {
+ return false;
+ }
+ }
+ break;
+ case SEARCH:
+ if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE
+ || state == ComponentState.READABLE_UNWRITABLE_FLUSHING) {
+ readerCount++;
+ } else {
+ return false;
+ }
+ break;
+ case FLUSH:
+ if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE) {
+ assert writerCount == 0;
+ state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
+ readerCount++;
+ } else {
+ return false;
+ }
+ break;
+ default:
+ throw new UnsupportedOperationException("Unsupported operation " + opType);
+ }
+ return true;
+ }
+
+ @Override
+ public void threadExit(LSMOperationType opType, boolean failedOperation, boolean isMutableComponent)
+ throws HyracksDataException {
+ switch (opType) {
+ case FORCE_MODIFICATION:
+ case MODIFICATION:
+ if (isMutableComponent) {
+ writerCount--;
+ if (state == ComponentState.READABLE_WRITABLE && isFull()) {
+ state = ComponentState.READABLE_UNWRITABLE;
+ }
+ } else {
+ readerCount--;
+ if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
+ state = ComponentState.INACTIVE;
+ }
+ }
+ break;
+ case SEARCH:
+ readerCount--;
+ if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
+ state = ComponentState.INACTIVE;
+ }
+ break;
+ case FLUSH:
+ assert state == ComponentState.READABLE_UNWRITABLE_FLUSHING;
+ readerCount--;
+ if (readerCount == 0) {
+ state = ComponentState.INACTIVE;
+ } else {
+ state = ComponentState.UNREADABLE_UNWRITABLE;
+ }
+ break;
+ default:
+ throw new UnsupportedOperationException("Unsupported operation " + opType);
+ }
+ assert readerCount > -1 && writerCount > -1;
+ }
+
+ public boolean isReadable() {
+ if (state == ComponentState.INACTIVE || state == ComponentState.UNREADABLE_UNWRITABLE) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public LSMComponentType getType() {
+ return LSMComponentType.MEMORY;
+ }
+
+ @Override
+ public ComponentState getState() {
+ return state;
+ }
+
+ public void setActive() {
+ requestedToBeActive = true;
+ }
+
+ public void setIsModified() {
+ isModified.set(true);
+ }
+
+ public boolean isModified() {
+ return isModified.get();
+ }
+
+ public boolean isFull() {
+ return vbc.isFull();
+ }
+
+ protected void reset() throws HyracksDataException {
+ isModified.set(false);
+ }
+}
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
deleted file mode 100644
index 6ca6581..0000000
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMutableLSMComponent.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Copyright 2009-2013 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.impls;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
-
-public abstract class AbstractMutableLSMComponent implements ILSMComponent {
-
- private int readerCount;
- private int writerCount;
- private ComponentState state;
-
- private boolean isModified;
-
- private enum ComponentState {
- READABLE_WRITABLE,
- READABLE_UNWRITABLE,
- READABLE_UNWRITABLE_FLUSHING,
- UNREADABLE_UNWRITABLE
- }
-
- public AbstractMutableLSMComponent() {
- readerCount = 0;
- writerCount = 0;
- state = ComponentState.READABLE_WRITABLE;
- isModified = false;
- }
-
- @Override
- public synchronized boolean threadEnter(LSMOperationType opType) throws InterruptedException {
- switch (opType) {
- case FORCE_MODIFICATION:
- if (state != ComponentState.READABLE_WRITABLE && state != ComponentState.READABLE_UNWRITABLE) {
- return false;
- }
- writerCount++;
- break;
- case MODIFICATION:
- if (state != ComponentState.READABLE_WRITABLE) {
- return false;
- }
- writerCount++;
- break;
- case SEARCH:
- if (state == ComponentState.UNREADABLE_UNWRITABLE) {
- return false;
- }
- readerCount++;
- break;
- case FLUSH:
- if (state == ComponentState.READABLE_UNWRITABLE_FLUSHING
- || state == ComponentState.UNREADABLE_UNWRITABLE) {
- return false;
- }
-
- state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
- while (writerCount > 0) {
- wait();
- }
- readerCount++;
- break;
- default:
- throw new UnsupportedOperationException("Unsupported operation " + opType);
- }
- return true;
- }
-
- @Override
- public synchronized void threadExit(LSMOperationType opType, boolean failedOperation) throws HyracksDataException {
- switch (opType) {
- case FORCE_MODIFICATION:
- case MODIFICATION:
- writerCount--;
- if (state == ComponentState.READABLE_WRITABLE && isFull()) {
- state = ComponentState.READABLE_UNWRITABLE;
- }
- break;
- case SEARCH:
- readerCount--;
- if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
- reset();
- state = ComponentState.READABLE_WRITABLE;
- } else if (state == ComponentState.READABLE_WRITABLE && isFull()) {
- state = ComponentState.READABLE_UNWRITABLE;
- }
- break;
- case FLUSH:
- if (failedOperation) {
- state = isFull() ? ComponentState.READABLE_UNWRITABLE : ComponentState.READABLE_WRITABLE;
- }
- readerCount--;
- if (readerCount == 0) {
- reset();
- state = ComponentState.READABLE_WRITABLE;
- } else if (state == ComponentState.READABLE_UNWRITABLE_FLUSHING) {
- state = ComponentState.UNREADABLE_UNWRITABLE;
- }
- break;
- default:
- throw new UnsupportedOperationException("Unsupported operation " + opType);
- }
- notifyAll();
- }
-
- public void setIsModified() {
- isModified = true;
- }
-
- public boolean isModified() {
- return isModified;
- }
-
- protected abstract boolean isFull();
-
- protected void reset() throws HyracksDataException {
- isModified = false;
- }
-}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
new file mode 100644
index 0000000..25894f1
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2013 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.impls;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+
+public class AsynchronousScheduler implements ILSMIOOperationScheduler {
+ public final static AsynchronousScheduler INSTANCE = new AsynchronousScheduler();
+ private ExecutorService executor;
+
+ public void init(ThreadFactory threadFactory) {
+ executor = Executors.newCachedThreadPool(threadFactory);
+ }
+
+ @Override
+ public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
+ executor.submit(operation);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
index 2510311..a7ca95f 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
@@ -32,7 +32,7 @@
public synchronized void waitForIO() throws InterruptedException {
if (!notified) {
- this.wait();
+ wait();
}
notified = false;
}
@@ -51,7 +51,7 @@
@Override
public synchronized void afterFinalize(ILSMComponent newComponent) throws HyracksDataException {
wrappedCallback.afterFinalize(newComponent);
- this.notifyAll();
+ notifyAll();
notified = true;
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
index 0ecfeb5..b6f5657 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
@@ -31,9 +31,8 @@
}
@Override
- public void diskComponentAdded(final ILSMIndex index, int totalNumDiskComponents) throws HyracksDataException,
- IndexException {
- if (totalNumDiskComponents >= threshold) {
+ public void diskComponentAdded(final ILSMIndex index) throws HyracksDataException, IndexException {
+ if (index.getImmutableComponents().size() >= threshold) {
ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
NoOpOperationCallback.INSTANCE);
accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
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 1632725..145bfe9 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
@@ -15,7 +15,6 @@
package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
-import java.util.ArrayList;
import java.util.List;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -27,6 +26,7 @@
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.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
@@ -48,63 +48,151 @@
this.mergePolicy = mergePolicy;
}
- private void threadExit(ILSMIndexOperationContext opCtx, LSMOperationType opType) throws HyracksDataException {
- if (!lsmIndex.getFlushStatus() && lsmIndex.isFull()) {
- lsmIndex.setFlushStatus(true);
- }
- opTracker.afterOperation(lsmIndex, opType, opCtx.getSearchOperationCallback(), opCtx.getModificationCallback());
- }
-
- private boolean getAndEnterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, boolean tryOperation)
+ private boolean getAndEnterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, boolean isTryOperation)
throws HyracksDataException {
- int numEntered = 0;
- boolean entranceSuccessful = false;
- List<ILSMComponent> entered = new ArrayList<ILSMComponent>();
-
- while (!entranceSuccessful) {
- entered.clear();
- lsmIndex.getOperationalComponents(ctx);
- List<ILSMComponent> components = ctx.getComponentHolder();
- try {
- for (ILSMComponent c : components) {
- if (!c.threadEnter(opType)) {
- break;
- }
- numEntered++;
- entered.add(c);
- }
- entranceSuccessful = numEntered == components.size();
- } catch (InterruptedException e) {
- entranceSuccessful = false;
- throw new HyracksDataException(e);
- } finally {
- if (!entranceSuccessful) {
- for (ILSMComponent c : components) {
- if (numEntered <= 0) {
- break;
+ synchronized (opTracker) {
+ while (true) {
+ lsmIndex.getOperationalComponents(ctx);
+ // Before entering the components, prune those corner cases that indeed should not proceed.
+ switch (opType) {
+ case FLUSH:
+ ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
+ if (!((AbstractMemoryLSMComponent) flushingComponent).isModified()) {
+ // The mutable component has not been modified by any writer. There is nothing to flush.
+ return false;
}
- c.threadExit(opType, true);
- numEntered--;
- }
+ break;
+ case MERGE:
+ if (ctx.getComponentHolder().size() < 2) {
+ // There is only a single component. There is nothing to merge.
+ return false;
+ }
+ default:
+ break;
+ }
+ if (enterComponents(ctx, opType)) {
+ return true;
+ } else if (isTryOperation) {
+ return false;
+ }
+ try {
+ // Flush and merge operations should never reach this wait call, because they are always try operations.
+ // If they fail to enter the components, then it means that there are an ongoing flush/merge operation on
+ // the same components, so they should not proceed.
+ opTracker.wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
}
}
- if (tryOperation && !entranceSuccessful) {
+ }
+ }
+
+ private boolean enterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType) throws HyracksDataException {
+ List<ILSMComponent> components = ctx.getComponentHolder();
+ int numEntered = 0;
+ boolean entranceSuccessful = false;
+ try {
+ for (ILSMComponent c : components) {
+ boolean isMutableComponent = numEntered == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
+ if (!c.threadEnter(opType, isMutableComponent)) {
+ break;
+ }
+ numEntered++;
+ }
+ entranceSuccessful = numEntered == components.size();
+ } finally {
+ if (!entranceSuccessful) {
+ int i = 0;
+ for (ILSMComponent c : components) {
+ if (numEntered == 0) {
+ break;
+ }
+ boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
+ c.threadExit(opType, true, isMutableComponent);
+ i++;
+ numEntered--;
+ }
return false;
}
}
-
+ // Check if there is any action that is needed to be taken based on the operation type
+ switch (opType) {
+ case FLUSH:
+ // Changing the flush status should *always* precede changing the mutable component.
+ lsmIndex.changeFlushStatusForCurrentMutableCompoent(false);
+ lsmIndex.changeMutableComponent();
+ // Notify all waiting threads whenever a flush has been scheduled since they will check
+ // again if they can grab and enter the mutable component.
+ opTracker.notifyAll();
+ break;
+ default:
+ break;
+ }
opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
return true;
}
- private void exitComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, boolean failedOperation)
- throws HyracksDataException {
- try {
- for (ILSMComponent c : ctx.getComponentHolder()) {
- c.threadExit(opType, failedOperation);
+ private void exitComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, ILSMComponent newComponent,
+ boolean failedOperation) throws HyracksDataException, IndexException {
+ synchronized (opTracker) {
+ try {
+ int i = 0;
+ // First check if there is any action that is needed to be taken based on the state of each component.
+ for (ILSMComponent c : ctx.getComponentHolder()) {
+ boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
+ c.threadExit(opType, failedOperation, isMutableComponent);
+ if (c.getType() == LSMComponentType.MEMORY) {
+ switch (c.getState()) {
+ case READABLE_UNWRITABLE:
+ if (isMutableComponent
+ && (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION)) {
+ lsmIndex.changeFlushStatusForCurrentMutableCompoent(true);
+ }
+ break;
+ case INACTIVE:
+ ((AbstractMemoryLSMComponent) c).reset();
+ // Notify all waiting threads whenever the mutable component's has change to inactive. This is important because
+ // even though we switched the mutable components, it is possible that the component that we just switched
+ // to is still busy flushing its data to disk. Thus, the notification that was issued upon scheduling the flush
+ // is not enough.
+ opTracker.notifyAll();
+ break;
+ default:
+ break;
+ }
+ } else {
+ switch (c.getState()) {
+ case INACTIVE:
+ ((AbstractDiskLSMComponent) c).destroy();
+ break;
+ default:
+ break;
+ }
+ }
+ i++;
+ }
+ // Then, perform any action that is needed to be taken based on the operation type.
+ switch (opType) {
+ case FLUSH:
+ // newComponent is null if the flush op. was not performed.
+ if (newComponent != null) {
+ lsmIndex.addComponent(newComponent);
+ mergePolicy.diskComponentAdded(lsmIndex);
+ }
+ break;
+ case MERGE:
+ // newComponent is null if the merge op. was not performed.
+ if (newComponent != null) {
+ lsmIndex.subsumeMergedComponents(newComponent, ctx.getComponentHolder());
+ }
+ break;
+ default:
+ break;
+ }
+ } finally {
+ opTracker.afterOperation(lsmIndex, opType, ctx.getSearchOperationCallback(),
+ ctx.getModificationCallback());
}
- } finally {
- threadExit(ctx, opType);
}
}
@@ -129,10 +217,12 @@
}
try {
lsmIndex.modify(ctx, tuple);
+ // The mutable component is always in the first index.
+ AbstractMemoryLSMComponent mutableComponent = (AbstractMemoryLSMComponent) ctx.getComponentHolder().get(0);
+ mutableComponent.setIsModified();
} finally {
- exitComponents(ctx, opType, false);
+ exitComponents(ctx, opType, null, false);
}
-
return true;
}
@@ -143,11 +233,8 @@
getAndEnterComponents(ctx, opType, false);
try {
lsmIndex.search(ctx, cursor, pred);
- } catch (HyracksDataException e) {
- exitComponents(ctx, opType, true);
- throw e;
- } catch (IndexException e) {
- exitComponents(ctx, opType, true);
+ } catch (HyracksDataException | IndexException e) {
+ exitComponents(ctx, opType, null, true);
throw e;
}
}
@@ -155,95 +242,92 @@
@Override
public void endSearch(ILSMIndexOperationContext ctx) throws HyracksDataException {
if (ctx.getOperation() == IndexOperation.SEARCH) {
- exitComponents(ctx, LSMOperationType.SEARCH, false);
+ try {
+ exitComponents(ctx, LSMOperationType.SEARCH, null, false);
+ } catch (IndexException e) {
+ throw new HyracksDataException(e);
+ }
}
}
@Override
- public void noOp(ILSMIndexOperationContext ctx) throws HyracksDataException {
- LSMOperationType opType = LSMOperationType.NOOP;
- opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
- threadExit(ctx, opType);
- }
-
- @Override
public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException {
if (!getAndEnterComponents(ctx, LSMOperationType.FLUSH, true)) {
- return;
- }
-
- lsmIndex.setFlushStatus(false);
-
- if (!lsmIndex.scheduleFlush(ctx, callback)) {
callback.beforeOperation();
callback.afterOperation(null, null);
callback.afterFinalize(null);
- exitComponents(ctx, LSMOperationType.FLUSH, false);
+ return;
}
+ lsmIndex.scheduleFlush(ctx, callback);
}
@Override
public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException,
IndexException {
- operation.getCallback().beforeOperation();
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(lsmIndex + ": flushing");
+ LOGGER.info("Started a flush operation for index: " + lsmIndex + " ...");
}
- ILSMComponent newComponent = lsmIndex.flush(operation);
- operation.getCallback().afterOperation(null, newComponent);
- lsmIndex.markAsValid(newComponent);
- operation.getCallback().afterFinalize(newComponent);
-
- lsmIndex.addComponent(newComponent);
- int numComponents = lsmIndex.getImmutableComponents().size();
-
- mergePolicy.diskComponentAdded(lsmIndex, numComponents);
- exitComponents(ctx, LSMOperationType.FLUSH, false);
+ ILSMComponent newComponent = null;
+ try {
+ operation.getCallback().beforeOperation();
+ newComponent = lsmIndex.flush(operation);
+ operation.getCallback().afterOperation(null, newComponent);
+ lsmIndex.markAsValid(newComponent);
+ } finally {
+ exitComponents(ctx, LSMOperationType.FLUSH, newComponent, false);
+ operation.getCallback().afterFinalize(newComponent);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Finished the flush operation for index: " + lsmIndex);
+ }
}
@Override
public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException, IndexException {
- LSMOperationType opType = LSMOperationType.MERGE;
- if (!getAndEnterComponents(ctx, opType, false)) {
+ // Merge should always be a try operation, because it should never fail to enter the components unless the merge policy is erroneous.
+ if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
+ callback.beforeOperation();
+ callback.afterOperation(null, null);
+ callback.afterFinalize(null);
return;
}
- if (ctx.getComponentHolder().size() > 1) {
- lsmIndex.scheduleMerge(ctx, callback);
- } else {
- exitComponents(ctx, opType, true);
- }
+ lsmIndex.scheduleMerge(ctx, callback);
}
@Override
public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException,
IndexException {
- List<ILSMComponent> mergedComponents = new ArrayList<ILSMComponent>();
- operation.getCallback().beforeOperation();
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(lsmIndex + ": merging");
+ LOGGER.info("Started a merge operation for index: " + lsmIndex + " ...");
}
- ILSMComponent newComponent = lsmIndex.merge(mergedComponents, operation);
- ctx.getComponentHolder().addAll(mergedComponents);
- operation.getCallback().afterOperation(mergedComponents, newComponent);
- lsmIndex.markAsValid(newComponent);
- operation.getCallback().afterFinalize(newComponent);
- lsmIndex.subsumeMergedComponents(newComponent, mergedComponents);
- exitComponents(ctx, LSMOperationType.MERGE, false);
+
+ ILSMComponent newComponent = null;
+ try {
+ operation.getCallback().beforeOperation();
+ newComponent = lsmIndex.merge(operation);
+ operation.getCallback().afterOperation(ctx.getComponentHolder(), newComponent);
+ lsmIndex.markAsValid(newComponent);
+ } finally {
+ exitComponents(ctx, LSMOperationType.MERGE, newComponent, false);
+ operation.getCallback().afterFinalize(newComponent);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Finished the merge operation for index: " + lsmIndex);
+ }
}
@Override
public void addBulkLoadedComponent(ILSMComponent c) throws HyracksDataException, IndexException {
lsmIndex.markAsValid(c);
lsmIndex.addComponent(c);
- int numComponents = lsmIndex.getImmutableComponents().size();
- mergePolicy.diskComponentAdded(lsmIndex, numComponents);
+ mergePolicy.diskComponentAdded(lsmIndex);
}
@Override
public ILSMOperationTracker getOperationTracker() {
return opTracker;
}
-}
+}
\ No newline at end of file
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 b6ca21b..45cc69b 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
@@ -39,7 +39,7 @@
protected PriorityQueueComparator pqCmp;
protected MultiComparator cmp;
protected boolean needPush;
- protected boolean includeMemComponent;
+ protected boolean includeMutableComponent;
protected ILSMHarness lsmHarness;
protected final ILSMIndexOperationContext opCtx;
@@ -51,6 +51,10 @@
needPush = false;
}
+ public ILSMIndexOperationContext getOpCtx() {
+ return opCtx;
+ }
+
public void initPriorityQueue() throws HyracksDataException, IndexException {
int pqInitSize = (rangeCursors.length > 0) ? rangeCursors.length : 1;
outputPriorityQueue = new PriorityQueue<PriorityQueueElement>(pqInitSize, pqCmp);
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMOperationType.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMOperationType.java
index d932e6d..853b6d0 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMOperationType.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMOperationType.java
@@ -19,6 +19,5 @@
MODIFICATION,
FORCE_MODIFICATION,
FLUSH,
- MERGE,
- NOOP
+ MERGE
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
index 6328b1d..f11a061 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
@@ -122,11 +122,6 @@
}
@Override
- public void noOp() throws HyracksDataException {
- lsmHarness.noOp(ctx);
- }
-
- @Override
public void forcePhysicalDelete(ITupleReference tuple) throws HyracksDataException, IndexException {
ctx.setOperation(IndexOperation.PHYSICALDELETE);
lsmHarness.forceModify(ctx, tuple);
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoMergePolicy.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoMergePolicy.java
index 80ac6d8..17d1b17 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoMergePolicy.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoMergePolicy.java
@@ -21,7 +21,7 @@
INSTANCE;
@Override
- public void diskComponentAdded(ILSMIndex index, int totalNumDiskComponents) {
+ public void diskComponentAdded(ILSMIndex index) {
// Do nothing
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
index 5eb63a9..44c20f5 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
@@ -25,7 +25,7 @@
@Override
public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
try {
- operation.perform();
+ operation.call();
} catch (IndexException e) {
throw new HyracksDataException(e);
}
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 19ee4b8..db75d7b 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
@@ -22,6 +22,7 @@
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexInternal;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
public class ThreadCountingTracker implements ILSMOperationTracker {
@@ -51,7 +52,8 @@
IModificationOperationCallback modificationCallback) throws HyracksDataException {
// Flush will only be handled by last exiting thread.
if (opType == LSMOperationType.MODIFICATION) {
- if (threadRefCount.decrementAndGet() == 0 && index.getFlushStatus()) {
+ if (threadRefCount.decrementAndGet() == 0
+ && ((ILSMIndexInternal) index).hasFlushRequestForCurrentMutableComponent()) {
ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
NoOpOperationCallback.INSTANCE);
accessor.scheduleFlush(NoOpIOOperationCallback.INSTANCE);
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
index 18c87a1..bd3b278 100644
--- 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
@@ -41,7 +41,7 @@
private final CacheBucket[] buckets;
private final ArrayList<VirtualPage> pages;
- private int nextFree;
+ private volatile int nextFree;
private boolean open;
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 1afd9d5..ee8febe 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
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow;
+import java.util.List;
+
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.IIndex;
@@ -34,18 +36,18 @@
public final class LSMInvertedIndexDataflowHelper extends AbstractLSMIndexDataflowHelper {
public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IVirtualBufferCache virtualBufferCache, ILSMMergePolicy mergePolicy,
+ List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
+ this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
opTrackerFactory, ioScheduler, ioOpCallbackProvider);
}
public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- super(opDesc, ctx, partition, virtualBufferCache, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+ List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
ioScheduler, ioOpCallbackProvider);
}
@@ -55,7 +57,7 @@
try {
IBufferCache diskBufferCache = opDesc.getStorageManager().getBufferCache(ctx);
IFileMapProvider diskFileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
- LSMInvertedIndex invIndex = InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCache,
+ LSMInvertedIndex invIndex = InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCaches,
diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
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 c9f81fc..85aa312 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
@@ -41,7 +41,7 @@
public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
return new LSMInvertedIndexDataflowHelper(opDesc, ctx, partition,
- virtualBufferCacheProvider.getVirtualBufferCache(ctx), bloomFilterFalsePositiveRate,
+ virtualBufferCacheProvider.getVirtualBufferCaches(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/PartitionedLSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
index 0dbd06b..1acb437 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
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow;
+import java.util.List;
+
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.IIndex;
@@ -34,7 +36,7 @@
public final class PartitionedLSMInvertedIndexDataflowHelper extends AbstractLSMIndexDataflowHelper {
public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
- int partition, IVirtualBufferCache virtualBufferCache, ILSMMergePolicy mergePolicy,
+ int partition, List<IVirtualBufferCache> virtualBufferCache, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
@@ -42,10 +44,10 @@
}
public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
- int partition, IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate,
+ int partition, List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- super(opDesc, ctx, partition, virtualBufferCache, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+ super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
ioScheduler, ioOpCallbackProvider);
}
@@ -56,7 +58,7 @@
IBufferCache diskBufferCache = opDesc.getStorageManager().getBufferCache(ctx);
IFileMapProvider diskFileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
PartitionedLSMInvertedIndex invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(
- virtualBufferCache, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
+ virtualBufferCaches, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
diskBufferCache, file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
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 67cab6d..63d99aa 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
@@ -41,7 +41,7 @@
public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
return new PartitionedLSMInvertedIndexDataflowHelper(opDesc, ctx, partition,
- virtualBufferCacheProvider.getVirtualBufferCache(ctx), bloomFilterFalsePositiveRate,
+ virtualBufferCacheProvider.getVirtualBufferCaches(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 244dfa0..f2f058a 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
@@ -48,6 +48,7 @@
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.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
@@ -65,6 +66,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
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.inmemory.InMemoryInvertedIndex;
@@ -79,9 +81,6 @@
public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex {
- // In-memory components.
- protected final LSMInvertedIndexMutableComponent mutableComponent;
- protected final IVirtualFreePageManager virtualFreePageManager;
protected final IBinaryTokenizerFactory tokenizerFactory;
// On-disk components.
@@ -94,31 +93,39 @@
protected final ITypeTraits[] tokenTypeTraits;
protected final IBinaryComparatorFactory[] tokenCmpFactories;
- public LSMInvertedIndex(IVirtualBufferCache virtualBufferCache, OnDiskInvertedIndexFactory diskInvIndexFactory,
- BTreeFactory deletedKeysBTreeFactory, BloomFilterFactory bloomFilterFactory,
- double bloomFilterFalsePositiveRate, ILSMIndexFileManager fileManager,
- IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+ public LSMInvertedIndex(List<IVirtualBufferCache> virtualBufferCaches,
+ OnDiskInvertedIndexFactory diskInvIndexFactory, BTreeFactory deletedKeysBTreeFactory,
+ BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
+ ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
- super(virtualBufferCache, diskInvIndexFactory.getBufferCache(), fileManager, diskFileMapProvider,
+ super(virtualBufferCaches, 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(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,
+
+ componentFactory = new LSMInvertedIndexDiskComponentFactory(diskInvIndexFactory, deletedKeysBTreeFactory,
bloomFilterFactory);
+
+ int i = 0;
+ for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
+ InMemoryInvertedIndex memInvIndex = createInMemoryInvertedIndex(virtualBufferCache,
+ new VirtualFreePageManager(virtualBufferCache.getNumPages()), i);
+ 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_" + i)));
+ LSMInvertedIndexMemoryComponent mutableComponent = new LSMInvertedIndexMemoryComponent(memInvIndex,
+ deleteKeysBTree, virtualBufferCache, i == 0 ? true : false);
+ memoryComponents.add(mutableComponent);
+ ++i;
+ }
}
@Override
@@ -129,7 +136,7 @@
fileManager.deleteDirs();
fileManager.createDirs();
- componentsRef.get().clear();
+ diskComponents.clear();
}
@Override
@@ -139,16 +146,20 @@
}
try {
- List<ILSMComponent> immutableComponents = componentsRef.get();
- ((IVirtualBufferCache) mutableComponent.getInvIndex().getBufferCache()).open();
- mutableComponent.getInvIndex().create();
- mutableComponent.getInvIndex().activate();
- mutableComponent.getDeletedKeysBTree().create();
- mutableComponent.getDeletedKeysBTree().activate();
+ List<ILSMComponent> immutableComponents = diskComponents;
+ for (ILSMComponent c : memoryComponents) {
+ LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
+ ((IVirtualBufferCache) mutableComponent.getInvIndex().getBufferCache()).open();
+ mutableComponent.getInvIndex().create();
+ mutableComponent.getInvIndex().activate();
+ mutableComponent.getDeletedKeysBTree().create();
+ mutableComponent.getDeletedKeysBTree().activate();
+ }
+
immutableComponents.clear();
List<LSMComponentFileReferences> validFileReferences = fileManager.cleanupAndGetValidFiles();
for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
- LSMInvertedIndexImmutableComponent component;
+ LSMInvertedIndexDiskComponent component;
try {
component = createDiskInvIndexComponent(componentFactory,
lsmComonentFileReference.getInsertIndexFileReference(),
@@ -171,12 +182,16 @@
if (!isActivated) {
throw new HyracksDataException("Failed to clear the index since it is not activated.");
}
- List<ILSMComponent> immutableComponents = componentsRef.get();
- mutableComponent.getInvIndex().clear();
- mutableComponent.getDeletedKeysBTree().clear();
- mutableComponent.reset();
+
+ for (ILSMComponent c : memoryComponents) {
+ LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
+ mutableComponent.getInvIndex().clear();
+ mutableComponent.getDeletedKeysBTree().clear();
+ mutableComponent.reset();
+ }
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) c;
+ LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) c;
component.getBloomFilter().deactivate();
component.getInvIndex().deactivate();
component.getDeletedKeysBTree().deactivate();
@@ -195,30 +210,32 @@
isActivated = false;
if (flushOnExit) {
- BlockingIOOperationCallbackWrapper blockingCallBack = new BlockingIOOperationCallbackWrapper(
+ BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
ioOpCallbackProvider.getIOOperationCallback(this));
- ILSMIndexAccessor accessor = (ILSMIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- accessor.scheduleFlush(blockingCallBack);
+ ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ accessor.scheduleFlush(cb);
try {
- blockingCallBack.waitForIO();
+ cb.waitForIO();
} catch (InterruptedException e) {
throw new HyracksDataException(e);
}
}
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) c;
+ LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) c;
component.getBloomFilter().deactivate();
component.getInvIndex().deactivate();
component.getDeletedKeysBTree().deactivate();
}
- mutableComponent.getInvIndex().deactivate();
- mutableComponent.getDeletedKeysBTree().deactivate();
- mutableComponent.getInvIndex().destroy();
- mutableComponent.getDeletedKeysBTree().destroy();
- ((IVirtualBufferCache) mutableComponent.getInvIndex().getBufferCache()).close();
+ for (ILSMComponent c : memoryComponents) {
+ LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
+ mutableComponent.getInvIndex().deactivate();
+ mutableComponent.getDeletedKeysBTree().deactivate();
+ mutableComponent.getInvIndex().destroy();
+ mutableComponent.getDeletedKeysBTree().destroy();
+ ((IVirtualBufferCache) mutableComponent.getInvIndex().getBufferCache()).close();
+ }
}
@Override
@@ -232,11 +249,14 @@
throw new HyracksDataException("Failed to destroy the index since it is activated.");
}
- mutableComponent.getInvIndex().destroy();
- mutableComponent.getDeletedKeysBTree().destroy();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ for (ILSMComponent c : memoryComponents) {
+ LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
+ mutableComponent.getInvIndex().destroy();
+ mutableComponent.getDeletedKeysBTree().destroy();
+ }
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) c;
+ LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) c;
component.getInvIndex().destroy();
component.getDeletedKeysBTree().destroy();
component.getBloomFilter().destroy();
@@ -246,17 +266,29 @@
@Override
public void getOperationalComponents(ILSMIndexOperationContext ctx) {
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
operationalComponents.clear();
+ int cmc = currentMutableComponentId.get();
+ ctx.setCurrentMutableComponentId(cmc);
+ int numMutableComponents = memoryComponents.size();
switch (ctx.getOperation()) {
case FLUSH:
case DELETE:
case INSERT:
- operationalComponents.add(mutableComponent);
+ operationalComponents.add(memoryComponents.get(cmc));
break;
case SEARCH:
- operationalComponents.add(mutableComponent);
+ for (int i = 0; i < numMutableComponents - 1; i++) {
+ ILSMComponent c = memoryComponents.get((cmc + i + 1) % numMutableComponents);
+ LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
+ if (mutableComponent.isReadable()) {
+ // Make sure newest components are added first
+ operationalComponents.add(0, mutableComponent);
+ }
+ }
+ // The current mutable component is always added
+ operationalComponents.add(0, memoryComponents.get(cmc));
operationalComponents.addAll(immutableComponents);
break;
case MERGE:
@@ -295,16 +327,16 @@
switch (ctx.getOperation()) {
case INSERT: {
// Insert into the in-memory inverted index.
- ctx.memInvIndexAccessor.insert(tuple);
+ ctx.currentMutableInvIndexAccessors.insert(tuple);
break;
}
case DELETE: {
// First remove all entries in the in-memory inverted index (if any).
- ctx.memInvIndexAccessor.delete(tuple);
+ ctx.currentMutableInvIndexAccessors.delete(tuple);
// Insert key into the deleted-keys BTree.
ctx.keysOnlyTuple.reset(tuple);
try {
- ctx.deletedKeysBTreeAccessor.insert(ctx.keysOnlyTuple);
+ ctx.currentDeletedKeysBTreeAccessors.insert(ctx.keysOnlyTuple);
} catch (TreeIndexDuplicateKeyException e) {
// Key has already been deleted.
}
@@ -314,7 +346,6 @@
throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
}
}
- mutableComponent.setIsModified();
}
@Override
@@ -323,58 +354,59 @@
List<ILSMComponent> operationalComponents = ictx.getComponentHolder();
int numComponents = operationalComponents.size();
assert numComponents > 0;
- boolean includeMutableComponent = operationalComponents.get(0) == mutableComponent;
+ boolean includeMutableComponent = false;
ArrayList<IIndexAccessor> indexAccessors = new ArrayList<IIndexAccessor>(numComponents);
ArrayList<IIndexAccessor> deletedKeysBTreeAccessors = new ArrayList<IIndexAccessor>(numComponents);
- if (includeMutableComponent) {
- IIndexAccessor invIndexAccessor = mutableComponent.getInvIndex().createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- indexAccessors.add(invIndexAccessor);
- IIndexAccessor deletedKeysAccessor = mutableComponent.getDeletedKeysBTree().createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- deletedKeysBTreeAccessors.add(deletedKeysAccessor);
- }
- for (int i = includeMutableComponent ? 1 : 0; i < operationalComponents.size(); i++) {
- LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) operationalComponents
- .get(i);
- IIndexAccessor invIndexAccessor = component.getInvIndex().createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- indexAccessors.add(invIndexAccessor);
- IIndexAccessor deletedKeysAccessor = component.getDeletedKeysBTree().createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- deletedKeysBTreeAccessors.add(deletedKeysAccessor);
+ for (int i = 0; i < operationalComponents.size(); i++) {
+ ILSMComponent component = operationalComponents.get(i);
+ if (component.getType() == LSMComponentType.MEMORY) {
+ includeMutableComponent = true;
+ IIndexAccessor invIndexAccessor = ((LSMInvertedIndexMemoryComponent) component).getInvIndex()
+ .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ indexAccessors.add(invIndexAccessor);
+ IIndexAccessor deletedKeysAccessor = ((LSMInvertedIndexMemoryComponent) component)
+ .getDeletedKeysBTree().createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ deletedKeysBTreeAccessors.add(deletedKeysAccessor);
+ } else {
+ IIndexAccessor invIndexAccessor = ((LSMInvertedIndexDiskComponent) component).getInvIndex()
+ .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ indexAccessors.add(invIndexAccessor);
+ IIndexAccessor deletedKeysAccessor = ((LSMInvertedIndexDiskComponent) component).getDeletedKeysBTree()
+ .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ deletedKeysBTreeAccessors.add(deletedKeysAccessor);
+ }
}
ICursorInitialState initState = createCursorInitialState(pred, ictx, includeMutableComponent, indexAccessors,
- deletedKeysBTreeAccessors);
+ deletedKeysBTreeAccessors, operationalComponents);
cursor.open(initState, pred);
}
private ICursorInitialState createCursorInitialState(ISearchPredicate pred, IIndexOperationContext ictx,
boolean includeMutableComponent, ArrayList<IIndexAccessor> indexAccessors,
- ArrayList<IIndexAccessor> deletedKeysBTreeAccessors) {
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ ArrayList<IIndexAccessor> deletedKeysBTreeAccessors, List<ILSMComponent> operationalComponents) {
ICursorInitialState initState = null;
PermutingTupleReference keysOnlyTuple = createKeysOnlyTupleReference();
MultiComparator keyCmp = MultiComparator.createIgnoreFieldLength(invListCmpFactories);
- List<ILSMComponent> operationalComponents = new ArrayList<ILSMComponent>();
- if (includeMutableComponent) {
- operationalComponents.add(mutableComponent);
- }
- operationalComponents.addAll(immutableComponents);
// TODO: This check is not pretty, but it does the job. Come up with something more OO in the future.
// Distinguish between regular searches and range searches (mostly used in merges).
if (pred instanceof InvertedIndexSearchPredicate) {
initState = new LSMInvertedIndexSearchCursorInitialState(keyCmp, keysOnlyTuple, indexAccessors,
- deletedKeysBTreeAccessors, mutableComponent.getDeletedKeysBTree().getLeafFrameFactory(), ictx,
- includeMutableComponent, lsmHarness, operationalComponents);
+ deletedKeysBTreeAccessors,
+ ((LSMInvertedIndexMemoryComponent) memoryComponents.get(currentMutableComponentId.get()))
+ .getDeletedKeysBTree().getLeafFrameFactory(), ictx, includeMutableComponent, lsmHarness,
+ operationalComponents);
} else {
+ LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
InMemoryInvertedIndex memInvIndex = (InMemoryInvertedIndex) mutableComponent.getInvIndex();
MultiComparator tokensAndKeysCmp = MultiComparator.create(memInvIndex.getBTree().getComparatorFactories());
initState = new LSMInvertedIndexRangeSearchCursorInitialState(tokensAndKeysCmp, keyCmp, keysOnlyTuple,
- mutableComponent.getDeletedKeysBTree().getLeafFrameFactory(), includeMutableComponent, lsmHarness,
+ ((LSMInvertedIndexMemoryComponent) memoryComponents.get(currentMutableComponentId.get()))
+ .getDeletedKeysBTree().getLeafFrameFactory(), includeMutableComponent, lsmHarness,
indexAccessors, deletedKeysBTreeAccessors, pred, operationalComponents);
}
return initState;
@@ -394,22 +426,18 @@
}
@Override
- public boolean scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+ public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException {
- if (!mutableComponent.isModified()) {
- return false;
- }
+ ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
LSMInvertedIndexOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE,
NoOpOperationCallback.INSTANCE);
- ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
opCtx.setOperation(IndexOperation.FLUSH);
opCtx.getComponentHolder().add(flushingComponent);
- ioScheduler.scheduleOperation(new LSMInvertedIndexFlushOperation(new LSMInvertedIndexAccessor(this, lsmHarness,
- fileManager, opCtx), mutableComponent, componentFileRefs.getInsertIndexFileReference(),
- componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
- callback));
- return true;
+ ioScheduler.scheduleOperation(new LSMInvertedIndexFlushOperation(
+ new LSMInvertedIndexAccessor(lsmHarness, opCtx), flushingComponent, componentFileRefs
+ .getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(),
+ componentFileRefs.getBloomFilterFileReference(), callback));
}
@Override
@@ -417,13 +445,14 @@
LSMInvertedIndexFlushOperation flushOp = (LSMInvertedIndexFlushOperation) operation;
// Create an inverted index instance to be bulk loaded.
- LSMInvertedIndexImmutableComponent component = createDiskInvIndexComponent(componentFactory,
+ LSMInvertedIndexDiskComponent component = createDiskInvIndexComponent(componentFactory,
flushOp.getDictBTreeFlushTarget(), flushOp.getDeletedKeysBTreeFlushTarget(),
flushOp.getBloomFilterFlushTarget(), true);
IInvertedIndex diskInvertedIndex = component.getInvIndex();
// Create a scan cursor on the BTree underlying the in-memory inverted index.
- LSMInvertedIndexMutableComponent flushingComponent = flushOp.getFlushingComponent();
+ LSMInvertedIndexMemoryComponent flushingComponent = (LSMInvertedIndexMemoryComponent) flushOp
+ .getFlushingComponent();
InMemoryInvertedIndexAccessor memInvIndexAccessor = (InMemoryInvertedIndexAccessor) flushingComponent
.getInvIndex().createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
BTreeAccessor memBTreeAccessor = memInvIndexAccessor.getBTreeAccessor();
@@ -496,44 +525,43 @@
public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException, IndexException {
LSMInvertedIndexOpContext ictx = createOpContext(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
- ictx.getComponentHolder().addAll(mergingComponents);
- IIndexCursor cursor = new LSMInvertedIndexRangeSearchCursor(ictx);
- RangePredicate mergePred = new RangePredicate(null, null, true, true, null, null);
-
- // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
- search(ictx, cursor, mergePred);
-
ictx.setOperation(IndexOperation.MERGE);
- LSMInvertedIndexImmutableComponent firstComponent = (LSMInvertedIndexImmutableComponent) mergingComponents
- .get(0);
+ List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+ IIndexCursor cursor = new LSMInvertedIndexRangeSearchCursor(ictx);
+
+ LSMInvertedIndexDiskComponent firstComponent = (LSMInvertedIndexDiskComponent) mergingComponents.get(0);
OnDiskInvertedIndex firstInvIndex = (OnDiskInvertedIndex) firstComponent.getInvIndex();
String firstFileName = firstInvIndex.getBTree().getFileReference().getFile().getName();
- LSMInvertedIndexImmutableComponent lastComponent = (LSMInvertedIndexImmutableComponent) mergingComponents
+ LSMInvertedIndexDiskComponent lastComponent = (LSMInvertedIndexDiskComponent) mergingComponents
.get(mergingComponents.size() - 1);
OnDiskInvertedIndex lastInvIndex = (OnDiskInvertedIndex) lastComponent.getInvIndex();
String lastFileName = lastInvIndex.getBTree().getFileReference().getFile().getName();
LSMComponentFileReferences relMergeFileRefs = fileManager.getRelMergeFileReference(firstFileName, lastFileName);
- ILSMIndexAccessorInternal accessor = new LSMInvertedIndexAccessor(this, lsmHarness, fileManager, ictx);
+ ILSMIndexAccessorInternal accessor = new LSMInvertedIndexAccessor(lsmHarness, ctx);
ioScheduler.scheduleOperation(new LSMInvertedIndexMergeOperation(accessor, mergingComponents, cursor,
relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getDeleteIndexFileReference(),
relMergeFileRefs.getBloomFilterFileReference(), callback));
}
@Override
- public ILSMComponent merge(List<ILSMComponent> mergedComponents, ILSMIOOperation operation)
- throws HyracksDataException, IndexException {
+ public ILSMComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException {
LSMInvertedIndexMergeOperation mergeOp = (LSMInvertedIndexMergeOperation) operation;
+ IIndexCursor cursor = mergeOp.getCursor();
+
+ RangePredicate mergePred = new RangePredicate(null, null, true, true, null, null);
+ ILSMIndexOperationContext opCtx = ((LSMIndexSearchCursor) cursor).getOpCtx();
+ opCtx.getComponentHolder().addAll(mergeOp.getMergingComponents());
+ // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
+ search(opCtx, cursor, mergePred);
// Create an inverted index instance.
- LSMInvertedIndexImmutableComponent component = createDiskInvIndexComponent(componentFactory,
+ LSMInvertedIndexDiskComponent component = createDiskInvIndexComponent(componentFactory,
mergeOp.getDictBTreeMergeTarget(), mergeOp.getDeletedKeysBTreeMergeTarget(),
mergeOp.getBloomFilterMergeTarget(), true);
IInvertedIndex mergedDiskInvertedIndex = component.getInvIndex();
- IIndexCursor cursor = mergeOp.getCursor();
IIndexBulkLoader invIndexBulkLoader = mergedDiskInvertedIndex.createBulkLoader(1.0f, true, 0L, false);
try {
while (cursor.hasNext()) {
@@ -545,10 +573,6 @@
cursor.close();
}
invIndexBulkLoader.end();
-
- // Add the merged components for cleanup.
- mergedComponents.addAll(mergeOp.getMergingComponents());
-
return component;
}
@@ -568,10 +592,6 @@
}
}
- public boolean isEmptyIndex() throws HyracksDataException {
- return componentsRef.get().isEmpty() && !mutableComponent.isModified();
- }
-
public class LSMInvertedIndexBulkLoader implements IIndexBulkLoader {
private final ILSMComponent component;
private final IIndexBulkLoader invIndexBulkLoader;
@@ -590,8 +610,8 @@
} catch (HyracksDataException | IndexException e) {
throw new IndexException(e);
}
- invIndexBulkLoader = ((LSMInvertedIndexImmutableComponent) component).getInvIndex().createBulkLoader(
- fillFactor, verifyInput, numElementsHint, false);
+ invIndexBulkLoader = ((LSMInvertedIndexDiskComponent) component).getInvIndex().createBulkLoader(fillFactor,
+ verifyInput, numElementsHint, false);
}
@Override
@@ -610,12 +630,12 @@
protected void cleanupArtifacts() throws HyracksDataException {
if (!cleanedUpArtifacts) {
cleanedUpArtifacts = true;
- ((LSMInvertedIndexImmutableComponent) component).getInvIndex().deactivate();
- ((LSMInvertedIndexImmutableComponent) component).getInvIndex().destroy();
- ((LSMInvertedIndexImmutableComponent) component).getDeletedKeysBTree().deactivate();
- ((LSMInvertedIndexImmutableComponent) component).getDeletedKeysBTree().destroy();
- ((LSMInvertedIndexImmutableComponent) component).getBloomFilter().deactivate();
- ((LSMInvertedIndexImmutableComponent) component).getBloomFilter().destroy();
+ ((LSMInvertedIndexDiskComponent) component).getInvIndex().deactivate();
+ ((LSMInvertedIndexDiskComponent) component).getInvIndex().destroy();
+ ((LSMInvertedIndexDiskComponent) component).getDeletedKeysBTree().deactivate();
+ ((LSMInvertedIndexDiskComponent) component).getDeletedKeysBTree().destroy();
+ ((LSMInvertedIndexDiskComponent) component).getBloomFilter().deactivate();
+ ((LSMInvertedIndexDiskComponent) component).getBloomFilter().destroy();
}
}
@@ -632,17 +652,17 @@
}
}
- protected InMemoryInvertedIndex createInMemoryInvertedIndex(IVirtualBufferCache virtualBufferCache)
- throws IndexException {
+ protected InMemoryInvertedIndex createInMemoryInvertedIndex(IVirtualBufferCache virtualBufferCache,
+ IVirtualFreePageManager virtualFreePageManager, int id) throws IndexException {
return InvertedIndexUtils.createInMemoryBTreeInvertedindex(virtualBufferCache, virtualFreePageManager,
invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- new FileReference(new File(fileManager.getBaseDir() + "_virtual_vocab")));
+ new FileReference(new File(fileManager.getBaseDir() + "_virtual_vocab_" + id)));
}
- protected LSMInvertedIndexImmutableComponent createDiskInvIndexComponent(ILSMComponentFactory factory,
+ protected LSMInvertedIndexDiskComponent createDiskInvIndexComponent(ILSMComponentFactory factory,
FileReference dictBTreeFileRef, FileReference btreeFileRef, FileReference bloomFilterFileRef, boolean create)
throws HyracksDataException, IndexException {
- LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) factory
+ LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) factory
.createLSMComponentInstance(new LSMComponentFileReferences(dictBTreeFileRef, btreeFileRef,
bloomFilterFileRef));
if (create) {
@@ -659,15 +679,13 @@
@Override
public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback,
- ISearchOperationCallback searchCallback) {
- return new LSMInvertedIndexAccessor(this, lsmHarness, fileManager, createOpContext(modificationCallback,
- searchCallback));
+ ISearchOperationCallback searchCallback) throws HyracksDataException {
+ return new LSMInvertedIndexAccessor(lsmHarness, createOpContext(modificationCallback, searchCallback));
}
private LSMInvertedIndexOpContext createOpContext(IModificationOperationCallback modificationCallback,
- ISearchOperationCallback searchCallback) {
- return new LSMInvertedIndexOpContext(mutableComponent.getInvIndex(), mutableComponent.getDeletedKeysBTree(),
- modificationCallback, searchCallback);
+ ISearchOperationCallback searchCallback) throws HyracksDataException {
+ return new LSMInvertedIndexOpContext(memoryComponents, modificationCallback, searchCallback);
}
@Override
@@ -693,8 +711,13 @@
@Override
public long getMemoryAllocationSize() {
- IBufferCache virtualBufferCache = mutableComponent.getInvIndex().getBufferCache();
- return virtualBufferCache.getNumPages() * virtualBufferCache.getPageSize();
+ long size = 0;
+ for (ILSMComponent c : memoryComponents) {
+ LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
+ IBufferCache virtualBufferCache = mutableComponent.getInvIndex().getBufferCache();
+ size += virtualBufferCache.getNumPages() * virtualBufferCache.getPageSize();
+ }
+ return size;
}
@Override
@@ -721,7 +744,7 @@
@Override
public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException {
- LSMInvertedIndexImmutableComponent invIndexComponent = (LSMInvertedIndexImmutableComponent) lsmComponent;
+ LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) lsmComponent;
OnDiskInvertedIndex invIndex = (OnDiskInvertedIndex) invIndexComponent.getInvIndex();
// Flush the bloom filter first.
int fileId = invIndexComponent.getBloomFilter().getFileId();
@@ -742,11 +765,14 @@
@Override
public void validate() throws HyracksDataException {
- mutableComponent.getInvIndex().validate();
- mutableComponent.getDeletedKeysBTree().validate();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ for (ILSMComponent c : memoryComponents) {
+ LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
+ mutableComponent.getInvIndex().validate();
+ mutableComponent.getDeletedKeysBTree().validate();
+ }
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) c;
+ LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) c;
component.getInvIndex().validate();
component.getDeletedKeysBTree().validate();
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
index 16f1fa1..e31af9a 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
@@ -25,7 +25,6 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
-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.invertedindex.api.IInvertedIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
@@ -33,16 +32,11 @@
public class LSMInvertedIndexAccessor implements ILSMIndexAccessorInternal, IInvertedIndexAccessor {
protected final ILSMHarness lsmHarness;
- protected final ILSMIndexFileManager fileManager;
protected final ILSMIndexOperationContext ctx;
- protected final LSMInvertedIndex invIndex;
- public LSMInvertedIndexAccessor(LSMInvertedIndex invIndex, ILSMHarness lsmHarness,
- ILSMIndexFileManager fileManager, ILSMIndexOperationContext ctx) {
+ public LSMInvertedIndexAccessor(ILSMHarness lsmHarness, ILSMIndexOperationContext ctx) {
this.lsmHarness = lsmHarness;
- this.fileManager = fileManager;
this.ctx = ctx;
- this.invIndex = invIndex;
}
@Override
@@ -112,11 +106,6 @@
}
@Override
- public void noOp() throws HyracksDataException {
- lsmHarness.noOp(ctx);
- }
-
- @Override
public void forcePhysicalDelete(ITupleReference tuple) throws HyracksDataException, IndexException {
throw new UnsupportedOperationException("Physical delete not supported by lsm inverted index.");
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexImmutableComponent.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
similarity index 85%
rename from hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexImmutableComponent.java
rename to hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
index 829d5d4..323edd1 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexImmutableComponent.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
@@ -17,16 +17,16 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractImmutableLSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractDiskLSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
-public class LSMInvertedIndexImmutableComponent extends AbstractImmutableLSMComponent {
+public class LSMInvertedIndexDiskComponent extends AbstractDiskLSMComponent {
private final IInvertedIndex invIndex;
private final BTree deletedKeysBTree;
private final BloomFilter bloomFilter;
- public LSMInvertedIndexImmutableComponent(IInvertedIndex invIndex, BTree deletedKeysBTree, BloomFilter bloomFilter) {
+ public LSMInvertedIndexDiskComponent(IInvertedIndex invIndex, BTree deletedKeysBTree, BloomFilter bloomFilter) {
this.invIndex = invIndex;
this.deletedKeysBTree = deletedKeysBTree;
this.bloomFilter = bloomFilter;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexComponentFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentFactory.java
similarity index 89%
rename from hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexComponentFactory.java
rename to hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentFactory.java
index 1455647..f83f529 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexComponentFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentFactory.java
@@ -26,12 +26,12 @@
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-public class LSMInvertedIndexComponentFactory implements ILSMComponentFactory {
+public class LSMInvertedIndexDiskComponentFactory implements ILSMComponentFactory {
private final OnDiskInvertedIndexFactory diskInvIndexFactory;
private final TreeIndexFactory<BTree> btreeFactory;
private final BloomFilterFactory bloomFilterFactory;
- public LSMInvertedIndexComponentFactory(OnDiskInvertedIndexFactory diskInvIndexFactory,
+ public LSMInvertedIndexDiskComponentFactory(OnDiskInvertedIndexFactory diskInvIndexFactory,
TreeIndexFactory<BTree> btreeFactory, BloomFilterFactory bloomFilterFactory) {
this.diskInvIndexFactory = diskInvIndexFactory;
this.btreeFactory = btreeFactory;
@@ -41,7 +41,7 @@
@Override
public ILSMComponent createLSMComponentInstance(LSMComponentFileReferences cfr) throws IndexException,
HyracksDataException {
- return new LSMInvertedIndexImmutableComponent(diskInvIndexFactory.createIndexInstance(cfr
+ return new LSMInvertedIndexDiskComponent(diskInvIndexFactory.createIndexInstance(cfr
.getInsertIndexFileReference()), btreeFactory.createIndexInstance(cfr.getDeleteIndexFileReference()),
bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()));
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java
index a8973d1..45433e7 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java
@@ -23,22 +23,22 @@
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
public class LSMInvertedIndexFlushOperation implements ILSMIOOperation {
private final ILSMIndexAccessorInternal accessor;
- private final LSMInvertedIndexMutableComponent flushingComponent;
+ private final ILSMComponent flushingComponent;
private final FileReference dictBTreeFlushTarget;
private final FileReference deletedKeysBTreeFlushTarget;
private final FileReference bloomFilterFlushTarget;
private final ILSMIOOperationCallback callback;
- public LSMInvertedIndexFlushOperation(ILSMIndexAccessorInternal accessor,
- LSMInvertedIndexMutableComponent flushingComponent, FileReference dictBTreeFlushTarget,
- FileReference deletedKeysBTreeFlushTarget, FileReference bloomFilterFlushTarget,
- ILSMIOOperationCallback callback) {
+ public LSMInvertedIndexFlushOperation(ILSMIndexAccessorInternal accessor, ILSMComponent flushingComponent,
+ FileReference dictBTreeFlushTarget, FileReference deletedKeysBTreeFlushTarget,
+ FileReference bloomFilterFlushTarget, ILSMIOOperationCallback callback) {
this.accessor = accessor;
this.flushingComponent = flushingComponent;
this.dictBTreeFlushTarget = dictBTreeFlushTarget;
@@ -63,8 +63,9 @@
}
@Override
- public void perform() throws HyracksDataException, IndexException {
+ public Boolean call() throws HyracksDataException, IndexException {
accessor.flush(this);
+ return true;
}
@Override
@@ -84,7 +85,7 @@
return bloomFilterFlushTarget;
}
- public LSMInvertedIndexMutableComponent getFlushingComponent() {
+ public ILSMComponent getFlushingComponent() {
return flushingComponent;
}
}
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/LSMInvertedIndexMemoryComponent.java
similarity index 82%
rename from hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMutableComponent.java
rename to hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
index 7084803..eb1f915 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/LSMInvertedIndexMemoryComponent.java
@@ -18,19 +18,19 @@
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.lsm.common.api.IVirtualBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMutableLSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMemoryLSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
-public class LSMInvertedIndexMutableComponent extends AbstractMutableLSMComponent {
+public class LSMInvertedIndexMemoryComponent extends AbstractMemoryLSMComponent {
private final IInvertedIndex invIndex;
private final BTree deletedKeysBTree;
- private final IVirtualBufferCache vbc;
- public LSMInvertedIndexMutableComponent(IInvertedIndex invIndex, BTree deletedKeysBTree, IVirtualBufferCache vbc) {
+ public LSMInvertedIndexMemoryComponent(IInvertedIndex invIndex, BTree deletedKeysBTree, IVirtualBufferCache vbc,
+ boolean isActive) {
+ super(vbc, isActive);
this.invIndex = invIndex;
this.deletedKeysBTree = deletedKeysBTree;
- this.vbc = vbc;
}
public IInvertedIndex getInvIndex() {
@@ -42,11 +42,6 @@
}
@Override
- protected boolean isFull() {
- return vbc.isFull();
- }
-
- @Override
protected void reset() throws HyracksDataException {
super.reset();
invIndex.deactivate();
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java
index 7714808..7cd921a 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java
@@ -55,7 +55,7 @@
public Set<IODeviceHandle> getReadDevices() {
Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
for (Object o : mergingComponents) {
- LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) o;
+ LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) o;
OnDiskInvertedIndex invIndex = (OnDiskInvertedIndex) component.getInvIndex();
devs.add(invIndex.getBTree().getFileReference().getDeviceHandle());
devs.add(component.getDeletedKeysBTree().getFileReference().getDeviceHandle());
@@ -74,8 +74,9 @@
}
@Override
- public void perform() throws HyracksDataException, IndexException {
+ public Boolean call() throws HyracksDataException, IndexException {
accessor.merge(this);
+ return true;
}
@Override
@@ -102,5 +103,4 @@
public List<ILSMComponent> getMergingComponents() {
return mergingComponents;
}
-
}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
index fd83bfb..1a9303f 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
@@ -19,7 +19,6 @@
import java.util.List;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
@@ -28,7 +27,6 @@
import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
public class LSMInvertedIndexOpContext implements ILSMIndexOperationContext {
@@ -36,8 +34,6 @@
private static final int NUM_DOCUMENT_FIELDS = 1;
private IndexOperation op;
- private final IInvertedIndex memInvIndex;
- private final IIndex memDeletedKeysBTree;
private final List<ILSMComponent> componentHolder;
public final IModificationOperationCallback modificationCallback;
@@ -46,18 +42,43 @@
// Tuple that only has the inverted-index elements (aka keys), projecting away the document fields.
public PermutingTupleReference keysOnlyTuple;
- // Accessor to the in-memory inverted index.
- public IInvertedIndexAccessor memInvIndexAccessor;
- // Accessor to the deleted-keys BTree.
- public IIndexAccessor deletedKeysBTreeAccessor;
+ // Accessor to the in-memory inverted indexes.
+ public IInvertedIndexAccessor[] mutableInvIndexAccessors;
+ // Accessor to the deleted-keys BTrees.
+ public IIndexAccessor[] deletedKeysBTreeAccessors;
- public LSMInvertedIndexOpContext(IInvertedIndex memInvIndex, IIndex memDeletedKeysBTree,
- IModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback) {
- this.memInvIndex = memInvIndex;
- this.memDeletedKeysBTree = memDeletedKeysBTree;
+ public IInvertedIndexAccessor currentMutableInvIndexAccessors;
+ public IIndexAccessor currentDeletedKeysBTreeAccessors;
+
+ public LSMInvertedIndexOpContext(List<ILSMComponent> mutableComponents,
+ IModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback)
+ throws HyracksDataException {
this.componentHolder = new LinkedList<ILSMComponent>();
this.modificationCallback = modificationCallback;
this.searchCallback = searchCallback;
+
+ mutableInvIndexAccessors = new IInvertedIndexAccessor[mutableComponents.size()];
+ deletedKeysBTreeAccessors = new IIndexAccessor[mutableComponents.size()];
+
+ for (int i = 0; i < mutableComponents.size(); i++) {
+ LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) mutableComponents
+ .get(i);
+ mutableInvIndexAccessors[i] = (IInvertedIndexAccessor) mutableComponent.getInvIndex().createAccessor(
+ NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ deletedKeysBTreeAccessors[i] = mutableComponent.getDeletedKeysBTree().createAccessor(
+ NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ }
+
+ assert mutableComponents.size() > 0;
+
+ // Project away the document fields, leaving only the key fields.
+ LSMInvertedIndexMemoryComponent c = (LSMInvertedIndexMemoryComponent) mutableComponents.get(0);
+ int numKeyFields = c.getInvIndex().getInvListTypeTraits().length;
+ int[] keyFieldPermutation = new int[numKeyFields];
+ for (int i = 0; i < numKeyFields; i++) {
+ keyFieldPermutation[i] = NUM_DOCUMENT_FIELDS + i;
+ }
+ keysOnlyTuple = new PermutingTupleReference(keyFieldPermutation);
}
@Override
@@ -69,26 +90,6 @@
// TODO: Ignore opcallback for now.
public void setOperation(IndexOperation newOp) throws HyracksDataException {
reset();
- switch (newOp) {
- case INSERT:
- case DELETE:
- case PHYSICALDELETE: {
- if (deletedKeysBTreeAccessor == null) {
- memInvIndexAccessor = (IInvertedIndexAccessor) memInvIndex.createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- deletedKeysBTreeAccessor = memDeletedKeysBTree.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- // Project away the document fields, leaving only the key fields.
- int numKeyFields = memInvIndex.getInvListTypeTraits().length;
- int[] keyFieldPermutation = new int[numKeyFields];
- for (int i = 0; i < numKeyFields; i++) {
- keyFieldPermutation[i] = NUM_DOCUMENT_FIELDS + i;
- }
- keysOnlyTuple = new PermutingTupleReference(keyFieldPermutation);
- }
- break;
- }
- }
op = newOp;
}
@@ -111,4 +112,10 @@
public IModificationOperationCallback getModificationCallback() {
return modificationCallback;
}
+
+ @Override
+ public void setCurrentMutableComponentId(int currentMutableComponentId) {
+ currentMutableInvIndexAccessors = mutableInvIndexAccessors[currentMutableComponentId];
+ currentDeletedKeysBTreeAccessors = deletedKeysBTreeAccessors[currentMutableComponentId];
+ }
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
index 820b516..cd0dde3 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
@@ -27,7 +27,9 @@
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
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.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
@@ -63,7 +65,7 @@
}
lsmHarness = lsmInitState.getLSMHarness();
operationalComponents = lsmInitState.getOperationalComponents();
- includeMemComponent = lsmInitState.getIncludeMemComponent();
+ includeMutableComponent = lsmInitState.getIncludeMemComponent();
// For searching the deleted-keys BTrees.
this.keysOnlyTuple = lsmInitState.getKeysOnlyTuple();
@@ -71,18 +73,17 @@
if (!deletedKeysBTreeAccessors.isEmpty()) {
deletedKeysBTreeCursors = new IIndexCursor[deletedKeysBTreeAccessors.size()];
- int i = 0;
- if (includeMemComponent) {
- // No need for a bloom filter for the in-memory BTree.
- deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor();
- ++i;
+ for (int i = 0; i < operationalComponents.size(); i++) {
+ ILSMComponent component = operationalComponents.get(i);
+ if (component.getType() == LSMComponentType.MEMORY) {
+ // No need for a bloom filter for the in-memory BTree.
+ deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor();
+ } else {
+ deletedKeysBTreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitState
+ .getgetDeletedKeysBTreeLeafFrameFactory().createFrame(), false,
+ ((LSMInvertedIndexDiskComponent) operationalComponents.get(i)).getBloomFilter());
+ }
}
- for (; i < deletedKeysBTreeCursors.length; i++) {
- deletedKeysBTreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitState
- .getgetDeletedKeysBTreeLeafFrameFactory().createFrame(), false,
- ((LSMInvertedIndexImmutableComponent) operationalComponents.get(i)).getBloomFilter());
- }
-
}
MultiComparator keyCmp = lsmInitState.getKeyComparator();
keySearchPred = new RangePredicate(keysOnlyTuple, keysOnlyTuple, true, true, keyCmp, keyCmp);
@@ -113,7 +114,7 @@
}
return false;
}
-
+
@Override
protected void checkPriorityQueue() throws HyracksDataException, IndexException {
while (!outputPriorityQueue.isEmpty() || needPush == true) {
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 882d1a1..19acbfb 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
@@ -28,6 +28,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
@@ -44,7 +45,6 @@
private int accessorIndex = -1;
private boolean tupleConsumed = true;
private ILSMHarness harness;
- private boolean includeMemComponent;
private List<IIndexAccessor> indexAccessors;
private ISearchPredicate searchPred;
private ISearchOperationCallback searchCallback;
@@ -61,7 +61,6 @@
public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
LSMInvertedIndexSearchCursorInitialState lsmInitState = (LSMInvertedIndexSearchCursorInitialState) initialState;
harness = lsmInitState.getLSMHarness();
- includeMemComponent = lsmInitState.getIncludeMemComponent();
operationalComponents = lsmInitState.getOperationalComponents();
indexAccessors = lsmInitState.getIndexAccessors();
opCtx = lsmInitState.getOpContext();
@@ -72,16 +71,17 @@
// For searching the deleted-keys BTrees.
deletedKeysBTreeAccessors = lsmInitState.getDeletedKeysBTreeAccessors();
deletedKeysBTreeCursors = new IIndexCursor[deletedKeysBTreeAccessors.size()];
- int i = 0;
- if (includeMemComponent) {
- // No need for a bloom filter for the in-memory BTree.
- deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor();
- ++i;
- }
- for (; i < deletedKeysBTreeCursors.length; i++) {
- deletedKeysBTreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitState
- .getgetDeletedKeysBTreeLeafFrameFactory().createFrame(), false,
- ((LSMInvertedIndexImmutableComponent) operationalComponents.get(i)).getBloomFilter());
+
+ for (int i = 0; i < operationalComponents.size(); i++) {
+ ILSMComponent component = operationalComponents.get(i);
+ if (component.getType() == LSMComponentType.MEMORY) {
+ // No need for a bloom filter for the in-memory BTree.
+ deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor();
+ } else {
+ deletedKeysBTreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitState
+ .getgetDeletedKeysBTreeLeafFrameFactory().createFrame(), false,
+ ((LSMInvertedIndexDiskComponent) operationalComponents.get(i)).getBloomFilter());
+ }
}
MultiComparator keyCmp = lsmInitState.getKeyComparator();
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 185321b..5b0c48d 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
@@ -16,11 +16,13 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls;
import java.io.File;
+import java.util.List;
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.IVirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -37,7 +39,7 @@
public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
- public PartitionedLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
+ public PartitionedLSMInvertedIndex(List<IVirtualBufferCache> virtualBufferCaches,
OnDiskInvertedIndexFactory diskInvIndexFactory, BTreeFactory deletedKeysBTreeFactory,
BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
@@ -45,17 +47,18 @@
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
- super(virtualBufferCache, diskInvIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory,
+ super(virtualBufferCaches, diskInvIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory,
bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories,
tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler,
ioOpCallbackProvider);
}
- protected InMemoryInvertedIndex createInMemoryInvertedIndex(IVirtualBufferCache virtualBufferCache)
- throws IndexException {
+ @Override
+ protected InMemoryInvertedIndex createInMemoryInvertedIndex(IVirtualBufferCache virtualBufferCache,
+ IVirtualFreePageManager virtualFreePageManager, int id) throws IndexException {
return InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(virtualBufferCache,
virtualFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
- tokenizerFactory, new FileReference(new File(fileManager.getBaseDir() + "_virtual_vocab")));
+ tokenizerFactory, new FileReference(new File(fileManager.getBaseDir() + "_virtual_vocab_" + id)));
}
}
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 9d85f56..ed02ba1 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
@@ -16,6 +16,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util;
import java.io.File;
+import java.util.List;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -115,7 +116,7 @@
return deletedKeysBTreeFactory;
}
- public static LSMInvertedIndex createLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
+ public static LSMInvertedIndex createLSMInvertedIndex(List<IVirtualBufferCache> virtualBufferCaches,
IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
@@ -143,20 +144,21 @@
diskFileMapProvider, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
tokenCmpFactories, fileManager);
- LSMInvertedIndex invIndex = new LSMInvertedIndex(virtualBufferCache, invIndexFactory, deletedKeysBTreeFactory,
+ LSMInvertedIndex invIndex = new LSMInvertedIndex(virtualBufferCaches, invIndexFactory, deletedKeysBTreeFactory,
bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits,
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
ioScheduler, ioOpCallbackProvider);
return invIndex;
}
- public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
- IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
- IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
- IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- IBufferCache diskBufferCache, String onDiskDir, double bloomFilterFalsePositiveRate,
- ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+ public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(
+ List<IVirtualBufferCache> virtualBufferCaches, IFileMapProvider diskFileMapProvider,
+ ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+ ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+ IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, String onDiskDir,
+ double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+ throws IndexException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
invListCmpFactories, diskBufferCache);
@@ -178,7 +180,7 @@
diskBufferCache, diskFileMapProvider, invListBuilderFactory, invListTypeTraits, invListCmpFactories,
tokenTypeTraits, tokenCmpFactories, fileManager);
- PartitionedLSMInvertedIndex invIndex = new PartitionedLSMInvertedIndex(virtualBufferCache, invIndexFactory,
+ PartitionedLSMInvertedIndex invIndex = new PartitionedLSMInvertedIndex(virtualBufferCaches, invIndexFactory,
deletedKeysBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager,
diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
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 9fc8cea..5d57aae 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
@@ -15,6 +15,8 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow;
+import java.util.List;
+
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -44,24 +46,24 @@
protected final ILinearizeComparatorFactory linearizeCmpFactory;
public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IVirtualBufferCache virtualBufferCache, IBinaryComparatorFactory[] btreeComparatorFactories,
+ List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
ILinearizeComparatorFactory linearizeCmpFactory) {
- this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE,
+ this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE,
btreeComparatorFactories, valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory,
ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
}
public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate,
+ List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
ILinearizeComparatorFactory linearizeCmpFactory) {
- super(opDesc, ctx, partition, virtualBufferCache, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+ super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
ioScheduler, ioOpCallbackProvider);
this.btreeComparatorFactories = btreeComparatorFactories;
this.valueProviderFactories = valueProviderFactories;
@@ -72,14 +74,14 @@
@Override
public ITreeIndex createIndexInstance() throws HyracksDataException {
AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
- return createLSMTree(virtualBufferCache, file, opDesc.getStorageManager().getBufferCache(ctx), opDesc
+ return createLSMTree(virtualBufferCaches, file, opDesc.getStorageManager().getBufferCache(ctx), opDesc
.getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
treeOpDesc.getTreeIndexComparatorFactories(), btreeComparatorFactories,
opTrackerFactory.getOperationTracker(ctx), valueProviderFactories, rtreePolicyType, linearizeCmpFactory);
}
- protected abstract ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+ protected abstract ITreeIndex createLSMTree(List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
ILSMOperationTracker opTracker, IPrimitiveValueProviderFactory[] valueProviderFactories,
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 b47bc02..bc4bca3 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
@@ -15,6 +15,8 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow;
+import java.util.List;
+
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -39,36 +41,36 @@
public class LSMRTreeDataflowHelper extends AbstractLSMRTreeDataflowHelper {
public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IVirtualBufferCache virtualBufferCache, IBinaryComparatorFactory[] btreeComparatorFactories,
+ List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
ILinearizeComparatorFactory linearizeCmpFactory) {
- super(opDesc, ctx, partition, virtualBufferCache, btreeComparatorFactories, valueProviderFactories,
+ super(opDesc, ctx, partition, virtualBufferCaches, btreeComparatorFactories, valueProviderFactories,
rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
}
public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IVirtualBufferCache virtualBufferCache, double bloomFilterFalsePositiveRate,
+ List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
ILinearizeComparatorFactory linearizeCmpFactory) {
- super(opDesc, ctx, partition, virtualBufferCache, bloomFilterFalsePositiveRate, btreeComparatorFactories,
+ super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, btreeComparatorFactories,
valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
ioOpCallbackProvider, linearizeCmpFactory);
}
@Override
- protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+ protected ITreeIndex createLSMTree(List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
ILSMOperationTracker opTracker, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory)
throws HyracksDataException {
try {
- return LSMRTreeUtils.createLSMTree(virtualBufferCache, file, diskBufferCache, diskFileMapProvider,
+ return LSMRTreeUtils.createLSMTree(virtualBufferCaches, file, diskBufferCache, diskFileMapProvider,
typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
linearizeCmpFactory);
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 5d35bc5..99b4219 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
@@ -56,7 +56,7 @@
public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
return new LSMRTreeDataflowHelper(opDesc, ctx, partition,
- virtualBufferCacheProvider.getVirtualBufferCache(ctx), bloomFilterFalsePositiveRate,
+ virtualBufferCacheProvider.getVirtualBufferCaches(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 ea1b1e2..72f3480 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
@@ -15,6 +15,8 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow;
+import java.util.List;
+
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -38,24 +40,24 @@
public class LSMRTreeWithAntiMatterTuplesDataflowHelper extends AbstractLSMRTreeDataflowHelper {
public LSMRTreeWithAntiMatterTuplesDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
- int partition, IVirtualBufferCache virtualBufferCache, IBinaryComparatorFactory[] btreeComparatorFactories,
+ int partition, List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
ILinearizeComparatorFactory linearizeCmpFactory) {
- super(opDesc, ctx, partition, virtualBufferCache, btreeComparatorFactories, valueProviderFactories,
+ super(opDesc, ctx, partition, virtualBufferCaches, btreeComparatorFactories, valueProviderFactories,
rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
}
@Override
- protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+ protected ITreeIndex createLSMTree(List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
ILSMOperationTracker opTracker, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory)
throws HyracksDataException {
try {
- return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCache, file, diskBufferCache,
+ return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCaches, file, diskBufferCache,
diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
} catch (TreeIndexException 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 5fca08f..00268b3 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
@@ -63,7 +63,7 @@
public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
return new LSMRTreeWithAntiMatterTuplesDataflowHelper(opDesc, ctx, partition,
- virtualBufferCacheProvider.getVirtualBufferCache(ctx), btreeComparatorFactories,
+ virtualBufferCacheProvider.getVirtualBufferCaches(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 285010c..04118c8 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
@@ -24,15 +24,15 @@
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.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.IFreePageManager;
+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;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
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.exceptions.TreeIndexDuplicateKeyException;
import edu.uci.ics.hyracks.storage.am.common.exceptions.TreeIndexNonExistentKeyException;
@@ -52,11 +52,9 @@
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;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
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;
@@ -66,11 +64,6 @@
protected final int[] comparatorFields;
protected final IBinaryComparatorFactory[] linearizerArray;
- // In-memory components.
- protected final LSMRTreeMutableComponent mutableComponent;
- protected final IVirtualBufferCache virtualBufferCache;
- protected final IVirtualFreePageManager virtualFreePageManager;
-
protected TreeTupleSorter rTreeTupleSorter;
// On-disk components.
@@ -86,27 +79,35 @@
protected final ITreeIndexFrameFactory rtreeLeafFrameFactory;
protected final ITreeIndexFrameFactory btreeLeafFrameFactory;
- public AbstractLSMRTree(IVirtualBufferCache virtualBufferCache, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
- ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
- ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileManager,
- TreeIndexFactory<RTree> diskRTreeFactory, ILSMComponentFactory componentFactory,
+ public AbstractLSMRTree(List<IVirtualBufferCache> virtualBufferCaches,
+ ITreeIndexFrameFactory rtreeInteriorFrameFactory, ITreeIndexFrameFactory rtreeLeafFrameFactory,
+ ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
+ ILSMIndexFileManager fileManager, 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(virtualBufferCache, diskRTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
+ super(virtualBufferCaches, componentFactory.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;
+ int i = 0;
+ for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
+ RTree memRTree = new RTree(virtualBufferCache,
+ ((IVirtualBufferCache) virtualBufferCache).getFileMapProvider(), new VirtualFreePageManager(
+ virtualBufferCache.getNumPages()), rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
+ rtreeCmpFactories, fieldCount, new FileReference(new File(fileManager.getBaseDir() + "_virtual_r_"
+ + i)));
+ BTree memBTree = new BTree(virtualBufferCache,
+ ((IVirtualBufferCache) virtualBufferCache).getFileMapProvider(), new VirtualFreePageManager(
+ virtualBufferCache.getNumPages()), btreeInteriorFrameFactory, btreeLeafFrameFactory,
+ btreeCmpFactories, fieldCount, new FileReference(new File(fileManager.getBaseDir() + "_virtual_b_"
+ + i)));
+ LSMRTreeMemoryComponent mutableComponent = new LSMRTreeMemoryComponent(memRTree, memBTree,
+ virtualBufferCache, i == 0 ? true : false);
+ memoryComponents.add(mutableComponent);
+ ++i;
+ }
+
this.rtreeInteriorFrameFactory = rtreeInteriorFrameFactory;
this.rtreeLeafFrameFactory = rtreeLeafFrameFactory;
this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
@@ -128,7 +129,7 @@
fileManager.deleteDirs();
fileManager.createDirs();
- componentsRef.get().clear();
+ diskComponents.clear();
}
@Override
@@ -137,11 +138,14 @@
throw new HyracksDataException("Failed to activate the index since it is already activated.");
}
- ((IVirtualBufferCache) mutableComponent.getRTree().getBufferCache()).open();
- mutableComponent.getRTree().create();
- mutableComponent.getBTree().create();
- mutableComponent.getRTree().activate();
- mutableComponent.getBTree().activate();
+ for (ILSMComponent c : memoryComponents) {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) c;
+ ((IVirtualBufferCache) mutableComponent.getRTree().getBufferCache()).open();
+ mutableComponent.getRTree().create();
+ mutableComponent.getBTree().create();
+ mutableComponent.getRTree().activate();
+ mutableComponent.getBTree().activate();
+ }
}
@Override
@@ -153,8 +157,7 @@
if (flushOnExit) {
BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
ioOpCallbackProvider.getIOOperationCallback(this));
- ILSMIndexAccessor accessor = (ILSMIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
+ ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
accessor.scheduleFlush(cb);
try {
cb.waitForIO();
@@ -163,11 +166,14 @@
}
}
- mutableComponent.getRTree().deactivate();
- mutableComponent.getBTree().deactivate();
- mutableComponent.getRTree().destroy();
- mutableComponent.getBTree().destroy();
- ((IVirtualBufferCache) mutableComponent.getRTree().getBufferCache()).close();
+ for (ILSMComponent c : memoryComponents) {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) c;
+ mutableComponent.getRTree().deactivate();
+ mutableComponent.getBTree().deactivate();
+ mutableComponent.getRTree().destroy();
+ mutableComponent.getBTree().destroy();
+ ((IVirtualBufferCache) mutableComponent.getRTree().getBufferCache()).close();
+ }
}
@Override
@@ -183,24 +189,39 @@
throw new HyracksDataException("Failed to clear the index since it is not activated.");
}
- mutableComponent.getRTree().clear();
- mutableComponent.getBTree().clear();
- mutableComponent.reset();
+ for (ILSMComponent c : memoryComponents) {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) c;
+ mutableComponent.getRTree().clear();
+ mutableComponent.getBTree().clear();
+ mutableComponent.reset();
+ }
}
@Override
public void getOperationalComponents(ILSMIndexOperationContext ctx) {
List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
operationalComponents.clear();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
+ int cmc = currentMutableComponentId.get();
+ ctx.setCurrentMutableComponentId(cmc);
+ int numMutableComponents = memoryComponents.size();
switch (ctx.getOperation()) {
case INSERT:
case DELETE:
case FLUSH:
- operationalComponents.add(mutableComponent);
+ operationalComponents.add(memoryComponents.get(cmc));
break;
case SEARCH:
- operationalComponents.add(mutableComponent);
+ for (int i = 0; i < numMutableComponents - 1; i++) {
+ ILSMComponent c = memoryComponents.get((cmc + i + 1) % numMutableComponents);
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) c;
+ if (mutableComponent.isReadable()) {
+ // Make sure newest components are added first
+ operationalComponents.add(0, mutableComponent);
+ }
+ }
+ // The current mutable component is always added
+ operationalComponents.add(0, memoryComponents.get(cmc));
operationalComponents.addAll(immutableComponents);
break;
case MERGE:
@@ -211,10 +232,23 @@
}
}
+ @Override
+ public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
+ throws HyracksDataException, IndexException {
+ LSMRTreeOpContext ctx = (LSMRTreeOpContext) ictx;
+ List<ILSMComponent> operationalComponents = ictx.getComponentHolder();
+
+ LSMRTreeCursorInitialState initialState = new LSMRTreeCursorInitialState(rtreeLeafFrameFactory,
+ rtreeInteriorFrameFactory, btreeLeafFrameFactory, ctx.getBTreeMultiComparator(), lsmHarness,
+ comparatorFields, linearizerArray, ctx.searchCallback, operationalComponents);
+
+ cursor.open(initialState, pred);
+ }
+
protected LSMComponentFileReferences getMergeTargetFileName(List<ILSMComponent> mergingDiskComponents)
throws HyracksDataException {
- RTree firstTree = ((LSMRTreeImmutableComponent) mergingDiskComponents.get(0)).getRTree();
- RTree lastTree = ((LSMRTreeImmutableComponent) mergingDiskComponents.get(mergingDiskComponents.size() - 1))
+ RTree firstTree = ((LSMRTreeDiskComponent) mergingDiskComponents.get(0)).getRTree();
+ RTree lastTree = ((LSMRTreeDiskComponent) mergingDiskComponents.get(mergingDiskComponents.size() - 1))
.getRTree();
FileReference firstFile = diskFileMapProvider.lookupFileName(firstTree.getFileId());
FileReference lastFile = diskFileMapProvider.lookupFileName(lastTree.getFileId());
@@ -223,11 +257,11 @@
return fileRefs;
}
- protected LSMRTreeImmutableComponent createDiskComponent(ILSMComponentFactory factory, FileReference insertFileRef,
+ protected LSMRTreeDiskComponent createDiskComponent(ILSMComponentFactory factory, FileReference insertFileRef,
FileReference deleteFileRef, FileReference bloomFilterFileRef, boolean createComponent)
throws HyracksDataException, IndexException {
// Create new tree instance.
- LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) factory
+ LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) factory
.createLSMComponentInstance(new LSMComponentFileReferences(insertFileRef, deleteFileRef,
bloomFilterFileRef));
if (createComponent) {
@@ -248,31 +282,43 @@
@Override
public ITreeIndexFrameFactory getLeafFrameFactory() {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getRTree().getLeafFrameFactory();
}
@Override
public ITreeIndexFrameFactory getInteriorFrameFactory() {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getRTree().getInteriorFrameFactory();
}
@Override
public IFreePageManager getFreePageManager() {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getRTree().getFreePageManager();
}
@Override
public int getFieldCount() {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getRTree().getFieldCount();
}
@Override
public int getRootPageId() {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getRTree().getRootPageId();
}
@Override
public int getFileId() {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) memoryComponents
+ .get(currentMutableComponentId.get());
return mutableComponent.getRTree().getFileId();
}
@@ -293,8 +339,8 @@
// added RTree tuple.
RangePredicate btreeRangePredicate = new RangePredicate(tuple, tuple, true, true,
ctx.getBTreeMultiComparator(), ctx.getBTreeMultiComparator());
- ITreeIndexCursor cursor = ctx.memBTreeAccessor.createSearchCursor();
- ctx.memBTreeAccessor.search(cursor, btreeRangePredicate);
+ ITreeIndexCursor cursor = ctx.currentMutableBTreeAccessor.createSearchCursor();
+ ctx.currentMutableBTreeAccessor.search(cursor, btreeRangePredicate);
boolean foundTupleInMemoryBTree = false;
try {
if (cursor.hasNext()) {
@@ -305,7 +351,7 @@
}
if (foundTupleInMemoryBTree) {
try {
- ctx.memBTreeAccessor.delete(tuple);
+ ctx.currentMutableBTreeAccessor.delete(tuple);
} catch (TreeIndexNonExistentKeyException e) {
// Tuple has been deleted in the meantime. Do nothing.
// This normally shouldn't happen if we are dealing with
@@ -314,31 +360,24 @@
// insert between them.
}
} else {
- ctx.memRTreeAccessor.insert(tuple);
+ ctx.currentMutableRTreeAccessor.insert(tuple);
}
} else {
try {
- ctx.memBTreeAccessor.insert(tuple);
+ ctx.currentMutableBTreeAccessor.insert(tuple);
} catch (TreeIndexDuplicateKeyException e) {
// Do nothing, because one delete tuple is enough to indicate
// that all the corresponding insert tuples are deleted
}
}
- mutableComponent.setIsModified();
}
protected LSMRTreeOpContext createOpContext(IModificationOperationCallback modCallback) {
- RTreeAccessor rtreeAccessor = (RTree.RTreeAccessor) mutableComponent.getRTree().createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- BTreeAccessor btreeAccessor = (BTree.BTreeAccessor) mutableComponent.getBTree().createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-
- return new LSMRTreeOpContext(rtreeAccessor, (IRTreeLeafFrame) rtreeLeafFrameFactory.createFrame(),
- (IRTreeInteriorFrame) rtreeInteriorFrameFactory.createFrame(), virtualFreePageManager
- .getMetaDataFrameFactory().createFrame(), 4, btreeAccessor, btreeLeafFrameFactory,
- btreeInteriorFrameFactory, virtualFreePageManager.getMetaDataFrameFactory().createFrame(),
- rtreeCmpFactories, btreeCmpFactories, modCallback, NoOpOperationCallback.INSTANCE);
+ return new LSMRTreeOpContext(memoryComponents, (IRTreeLeafFrame) rtreeLeafFrameFactory.createFrame(),
+ (IRTreeInteriorFrame) rtreeInteriorFrameFactory.createFrame(), btreeLeafFrameFactory,
+ btreeInteriorFrameFactory, rtreeCmpFactories, btreeCmpFactories, modCallback,
+ NoOpOperationCallback.INSTANCE);
}
@Override
@@ -346,10 +385,6 @@
return rtreeCmpFactories;
}
- public boolean isEmptyIndex() throws HyracksDataException {
- return componentsRef.get().isEmpty() && !mutableComponent.isModified();
- }
-
@Override
public void validate() throws HyracksDataException {
throw new UnsupportedOperationException("Validation not implemented for LSM R-Trees.");
@@ -357,8 +392,13 @@
@Override
public long getMemoryAllocationSize() {
- IBufferCache virtualBufferCache = mutableComponent.getRTree().getBufferCache();
- return virtualBufferCache.getNumPages() * virtualBufferCache.getPageSize();
+ long size = 0;
+ for (ILSMComponent c : memoryComponents) {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) c;
+ IBufferCache virtualBufferCache = mutableComponent.getRTree().getBufferCache();
+ size += virtualBufferCache.getNumPages() * virtualBufferCache.getPageSize();
+ }
+ return size;
}
@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 0b71373..410fde8 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
@@ -16,7 +16,6 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
import java.util.List;
-import java.util.ListIterator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -66,7 +65,7 @@
public class LSMRTree extends AbstractLSMRTree {
- public LSMRTree(IVirtualBufferCache virtualBufferCache, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
+ public LSMRTree(List<IVirtualBufferCache> virtualBufferCaches, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileNameManager,
TreeIndexFactory<RTree> diskRTreeFactory, TreeIndexFactory<BTree> diskBTreeFactory,
@@ -76,11 +75,11 @@
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);
+ super(virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
+ btreeLeafFrameFactory, fileNameManager, new LSMRTreeDiskComponentFactory(diskRTreeFactory,
+ diskBTreeFactory, bloomFilterFactory), diskFileMapProvider, fieldCount, rtreeCmpFactories,
+ btreeCmpFactories, linearizer, comparatorFields, linearizerArray, bloomFilterFalsePositiveRate,
+ mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
}
/**
@@ -94,7 +93,7 @@
@Override
public synchronized void activate() throws HyracksDataException {
super.activate();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
List<LSMComponentFileReferences> validFileReferences;
try {
validFileReferences = fileManager.cleanupAndGetValidFiles();
@@ -103,7 +102,7 @@
}
immutableComponents.clear();
for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
- LSMRTreeImmutableComponent component;
+ LSMRTreeDiskComponent component;
try {
component = createDiskComponent(componentFactory,
lsmComonentFileReference.getInsertIndexFileReference(),
@@ -120,9 +119,9 @@
@Override
public synchronized void deactivate(boolean flushOnExit) throws HyracksDataException {
super.deactivate(flushOnExit);
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) c;
+ LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
RTree rtree = component.getRTree();
BTree btree = component.getBTree();
BloomFilter bloomFilter = component.getBloomFilter();
@@ -141,9 +140,9 @@
@Override
public synchronized void destroy() throws HyracksDataException {
super.destroy();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) c;
+ LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
component.getBTree().destroy();
component.getBloomFilter().destroy();
component.getRTree().destroy();
@@ -154,9 +153,9 @@
@Override
public synchronized void clear() throws HyracksDataException {
super.clear();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) c;
+ LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
component.getBTree().deactivate();
component.getBloomFilter().deactivate();
component.getRTree().deactivate();
@@ -168,64 +167,23 @@
}
@Override
- public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
- throws HyracksDataException, IndexException {
- LSMRTreeOpContext ctx = (LSMRTreeOpContext) ictx;
- List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
- boolean includeMutableComponent = operationalComponents.get(0) == mutableComponent;
- int numTrees = operationalComponents.size();
-
- ListIterator<ILSMComponent> diskComponentIter = operationalComponents.listIterator();
- ITreeIndexAccessor[] rTreeAccessors = new ITreeIndexAccessor[numTrees];
- ITreeIndexAccessor[] bTreeAccessors = new ITreeIndexAccessor[numTrees];
- int diskComponentIx = 0;
- if (includeMutableComponent) {
- rTreeAccessors[0] = ctx.memRTreeAccessor;
- bTreeAccessors[0] = ctx.memBTreeAccessor;
- diskComponentIx++;
- diskComponentIter.next();
- }
-
- while (diskComponentIter.hasNext()) {
- LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) diskComponentIter.next();
- RTree diskRTree = component.getRTree();
- BTree diskBTree = component.getBTree();
- rTreeAccessors[diskComponentIx] = diskRTree.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- bTreeAccessors[diskComponentIx] = diskBTree.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- diskComponentIx++;
- }
-
- LSMRTreeCursorInitialState initialState = new LSMRTreeCursorInitialState(numTrees, rtreeLeafFrameFactory,
- rtreeInteriorFrameFactory, btreeLeafFrameFactory, ctx.getBTreeMultiComparator(), rTreeAccessors,
- bTreeAccessors, includeMutableComponent, lsmHarness, comparatorFields, linearizerArray,
- ctx.searchCallback, operationalComponents);
- cursor.open(initialState, pred);
- }
-
- @Override
- public boolean scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+ public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException {
- if (!mutableComponent.isModified()) {
- return false;
- }
+ ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
ILSMIndexOperationContext rctx = createOpContext(NoOpOperationCallback.INSTANCE);
- LSMRTreeMutableComponent flushingComponent = (LSMRTreeMutableComponent) ctx.getComponentHolder().get(0);
rctx.setOperation(IndexOperation.FLUSH);
rctx.getComponentHolder().addAll(ctx.getComponentHolder());
LSMRTreeAccessor accessor = new LSMRTreeAccessor(lsmHarness, rctx);
ioScheduler.scheduleOperation(new LSMRTreeFlushOperation(accessor, flushingComponent, componentFileRefs
.getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(), componentFileRefs
.getBloomFilterFileReference(), callback));
- return true;
}
@Override
public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
LSMRTreeFlushOperation flushOp = (LSMRTreeFlushOperation) operation;
- LSMRTreeMutableComponent flushingComponent = (LSMRTreeMutableComponent) flushOp.getFlushingComponent();
+ LSMRTreeMemoryComponent flushingComponent = (LSMRTreeMemoryComponent) flushOp.getFlushingComponent();
// Renaming order is critical because we use assume ordering when we
// read the file names when we open the tree.
// The RTree should be renamed before the BTree.
@@ -236,7 +194,7 @@
RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor();
SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
- LSMRTreeImmutableComponent component = createDiskComponent(componentFactory, flushOp.getRTreeFlushTarget(),
+ LSMRTreeDiskComponent component = createDiskComponent(componentFactory, flushOp.getRTreeFlushTarget(),
flushOp.getBTreeFlushTarget(), flushOp.getBloomFilterFlushTarget(), true);
RTree diskRTree = component.getRTree();
IIndexBulkLoader rTreeBulkloader;
@@ -329,17 +287,10 @@
@Override
public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException, IndexException {
- // Renaming order is critical because we use assume ordering when we
- // read the file names when we open the tree.
- // The RTree should be renamed before the BTree.
- List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
ILSMIndexOperationContext rctx = createOpContext(NoOpOperationCallback.INSTANCE);
- rctx.getComponentHolder().addAll(mergingComponents);
- ITreeIndexCursor cursor = new LSMRTreeSortedCursor(rctx, linearizer);
- ISearchPredicate rtreeSearchPred = new SearchPredicate(null, null);
- search(rctx, cursor, rtreeSearchPred);
-
rctx.setOperation(IndexOperation.MERGE);
+ List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+ ITreeIndexCursor cursor = new LSMRTreeSortedCursor(rctx, linearizer);
LSMComponentFileReferences relMergeFileRefs = getMergeTargetFileName(mergingComponents);
ILSMIndexAccessorInternal accessor = new LSMRTreeAccessor(lsmHarness, rctx);
ioScheduler.scheduleOperation(new LSMRTreeMergeOperation((ILSMIndexAccessorInternal) accessor,
@@ -348,14 +299,16 @@
}
@Override
- public ILSMComponent merge(List<ILSMComponent> mergedComponents, ILSMIOOperation operation)
- throws HyracksDataException, IndexException {
+ public ILSMComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException {
LSMRTreeMergeOperation mergeOp = (LSMRTreeMergeOperation) operation;
ITreeIndexCursor cursor = mergeOp.getCursor();
- mergedComponents.addAll(mergeOp.getMergingComponents());
+ ISearchPredicate rtreeSearchPred = new SearchPredicate(null, null);
+ ILSMIndexOperationContext opCtx = ((LSMRTreeSortedCursor) cursor).getOpCtx();
+ opCtx.getComponentHolder().addAll(mergeOp.getMergingComponents());
+ search(opCtx, cursor, rtreeSearchPred);
- LSMRTreeImmutableComponent mergedComponent = createDiskComponent(componentFactory,
- mergeOp.getRTreeMergeTarget(), mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
+ LSMRTreeDiskComponent mergedComponent = createDiskComponent(componentFactory, mergeOp.getRTreeMergeTarget(),
+ mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
IIndexBulkLoader bulkLoader = mergedComponent.getRTree().createBulkLoader(1.0f, false, 0L, false);
try {
@@ -389,7 +342,7 @@
public MultiComparator getMultiComparator() {
LSMRTreeOpContext concreteCtx = (LSMRTreeOpContext) ctx;
- return concreteCtx.rtreeOpContext.cmp;
+ return concreteCtx.currentRTreeOpContext.cmp;
}
}
@@ -427,7 +380,7 @@
} catch (HyracksDataException | IndexException e) {
throw new TreeIndexException(e);
}
- bulkLoader = ((LSMRTreeImmutableComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
+ bulkLoader = ((LSMRTreeDiskComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
numElementsHint, false);
}
@@ -459,19 +412,19 @@
protected void cleanupArtifacts() throws HyracksDataException {
if (!cleanedUpArtifacts) {
cleanedUpArtifacts = true;
- ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
- ((LSMRTreeImmutableComponent) component).getRTree().destroy();
- ((LSMRTreeImmutableComponent) component).getBTree().deactivate();
- ((LSMRTreeImmutableComponent) component).getBTree().destroy();
- ((LSMRTreeImmutableComponent) component).getBloomFilter().deactivate();
- ((LSMRTreeImmutableComponent) component).getBloomFilter().destroy();
+ ((LSMRTreeDiskComponent) component).getRTree().deactivate();
+ ((LSMRTreeDiskComponent) component).getRTree().destroy();
+ ((LSMRTreeDiskComponent) component).getBTree().deactivate();
+ ((LSMRTreeDiskComponent) component).getBTree().destroy();
+ ((LSMRTreeDiskComponent) component).getBloomFilter().deactivate();
+ ((LSMRTreeDiskComponent) component).getBloomFilter().destroy();
}
}
}
@Override
public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException {
- LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) lsmComponent;
+ LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) lsmComponent;
// Flush the bloom filter first.
int fileId = component.getBloomFilter().getFileId();
IBufferCache bufferCache = component.getBTree().getBufferCache();
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
index 0cd578a..4e322d4 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
@@ -12,146 +12,155 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
-
-import java.util.List;
-
-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.btree.api.IBTreeLeafFrame;
-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.ICursorInitialState;
-import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-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.ILSMIndexOperationContext;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
-import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
-import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-
-public abstract class LSMRTreeAbstractCursor implements ITreeIndexCursor {
-
- protected RTreeSearchCursor[] rtreeCursors;
- protected boolean open = false;
- protected ITreeIndexCursor[] btreeCursors;
- protected ITreeIndexAccessor[] rTreeAccessors;
- protected ITreeIndexAccessor[] bTreeAccessors;
- private MultiComparator btreeCmp;
- protected int numberOfTrees;
- protected SearchPredicate rtreeSearchPredicate;
- protected RangePredicate btreeRangePredicate;
- protected ITupleReference frameTuple;
- protected boolean includeMemRTree;
- protected ILSMHarness lsmHarness;
- protected boolean foundNext;
- protected final ILSMIndexOperationContext opCtx;
-
- protected List<ILSMComponent> operationalComponents;
-
- public LSMRTreeAbstractCursor(ILSMIndexOperationContext opCtx) {
- super();
- this.opCtx = opCtx;
- }
-
- public RTreeSearchCursor getCursor(int cursorIndex) {
- return rtreeCursors[cursorIndex];
- }
-
- @Override
- public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
- LSMRTreeCursorInitialState lsmInitialState = (LSMRTreeCursorInitialState) initialState;
- btreeCmp = lsmInitialState.getBTreeCmp();
- includeMemRTree = lsmInitialState.getIncludeMemComponent();
- operationalComponents = lsmInitialState.getOperationalComponents();
- lsmHarness = lsmInitialState.getLSMHarness();
- numberOfTrees = lsmInitialState.getNumberOfTrees();
- rTreeAccessors = lsmInitialState.getRTreeAccessors();
- bTreeAccessors = lsmInitialState.getBTreeAccessors();
-
- rtreeCursors = new RTreeSearchCursor[numberOfTrees];
- btreeCursors = new ITreeIndexCursor[numberOfTrees];
-
- int i = 0;
- if (includeMemRTree) {
- rtreeCursors[i] = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState
- .getRTreeInteriorFrameFactory().createFrame(), (IRTreeLeafFrame) lsmInitialState
- .getRTreeLeafFrameFactory().createFrame());
-
- // No need for a bloom filter for the in-memory BTree.
- btreeCursors[i] = new BTreeRangeSearchCursor((IBTreeLeafFrame) lsmInitialState.getBTreeLeafFrameFactory()
- .createFrame(), false);
- ++i;
- }
- for (; i < numberOfTrees; i++) {
- rtreeCursors[i] = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState
- .getRTreeInteriorFrameFactory().createFrame(), (IRTreeLeafFrame) lsmInitialState
- .getRTreeLeafFrameFactory().createFrame());
-
- btreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitialState
- .getBTreeLeafFrameFactory().createFrame(), false,
- ((LSMRTreeImmutableComponent) operationalComponents.get(i)).getBloomFilter());
- }
-
- rtreeSearchPredicate = (SearchPredicate) searchPred;
- btreeRangePredicate = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp);
-
- open = true;
- }
-
- @Override
- public ICachedPage getPage() {
- // do nothing
- return null;
- }
-
- @Override
- public void close() throws HyracksDataException {
- if (!open) {
- return;
- }
-
- try {
- if (rtreeCursors != null && btreeCursors != null) {
- for (int i = 0; i < numberOfTrees; i++) {
- rtreeCursors[i].close();
- btreeCursors[i].close();
- }
- }
- rtreeCursors = null;
- btreeCursors = null;
- } finally {
- lsmHarness.endSearch(opCtx);
- }
-
- open = false;
- }
-
- @Override
- public void setBufferCache(IBufferCache bufferCache) {
- // do nothing
- }
-
- @Override
- public void setFileId(int fileId) {
- // do nothing
- }
-
- @Override
- public ITupleReference getTuple() {
- return frameTuple;
- }
-
- @Override
- public boolean exclusiveLatchNodes() {
- return false;
- }
-
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import java.util.List;
+
+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.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+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.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
+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.RTreeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public abstract class LSMRTreeAbstractCursor implements ITreeIndexCursor {
+
+ protected boolean open;
+ protected RTreeSearchCursor[] rtreeCursors;
+ protected ITreeIndexCursor[] btreeCursors;
+ protected ITreeIndexAccessor[] rtreeAccessors;
+ protected ITreeIndexAccessor[] btreeAccessors;
+ private MultiComparator btreeCmp;
+ protected int numberOfTrees;
+ protected SearchPredicate rtreeSearchPredicate;
+ protected RangePredicate btreeRangePredicate;
+ protected ITupleReference frameTuple;
+ protected boolean includeMutableComponent;
+ protected ILSMHarness lsmHarness;
+ protected boolean foundNext;
+ protected final ILSMIndexOperationContext opCtx;
+
+ protected List<ILSMComponent> operationalComponents;
+
+ public LSMRTreeAbstractCursor(ILSMIndexOperationContext opCtx) {
+ super();
+ this.opCtx = opCtx;
+ }
+
+ public RTreeSearchCursor getCursor(int cursorIndex) {
+ return rtreeCursors[cursorIndex];
+ }
+
+ @Override
+ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+ LSMRTreeCursorInitialState lsmInitialState = (LSMRTreeCursorInitialState) initialState;
+ btreeCmp = lsmInitialState.getBTreeCmp();
+
+ operationalComponents = lsmInitialState.getOperationalComponents();
+ lsmHarness = lsmInitialState.getLSMHarness();
+ numberOfTrees = operationalComponents.size();
+
+ rtreeCursors = new RTreeSearchCursor[numberOfTrees];
+ btreeCursors = new ITreeIndexCursor[numberOfTrees];
+ rtreeAccessors = new ITreeIndexAccessor[numberOfTrees];
+ btreeAccessors = new ITreeIndexAccessor[numberOfTrees];
+
+ includeMutableComponent = false;
+ for (int i = 0; i < numberOfTrees; i++) {
+ ILSMComponent component = operationalComponents.get(i);
+ RTree rtree;
+ BTree btree;
+ if (component.getType() == LSMComponentType.MEMORY) {
+ includeMutableComponent = true;
+ // No need for a bloom filter for the in-memory BTree.
+ btreeCursors[i] = new BTreeRangeSearchCursor((IBTreeLeafFrame) lsmInitialState
+ .getBTreeLeafFrameFactory().createFrame(), false);
+ rtree = (RTree) ((LSMRTreeMemoryComponent) component).getRTree();
+ btree = (BTree) ((LSMRTreeMemoryComponent) component).getBTree();
+ } else {
+ btreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitialState
+ .getBTreeLeafFrameFactory().createFrame(), false,
+ ((LSMRTreeDiskComponent) operationalComponents.get(i)).getBloomFilter());
+ rtree = (RTree) ((LSMRTreeDiskComponent) component).getRTree();
+ btree = (BTree) ((LSMRTreeDiskComponent) component).getBTree();
+ }
+ rtreeCursors[i] = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState
+ .getRTreeInteriorFrameFactory().createFrame(), (IRTreeLeafFrame) lsmInitialState
+ .getRTreeLeafFrameFactory().createFrame());
+ rtreeAccessors[i] = rtree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ btreeAccessors[i] = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ }
+
+ rtreeSearchPredicate = (SearchPredicate) searchPred;
+ btreeRangePredicate = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp);
+
+ open = true;
+ }
+
+ @Override
+ public ICachedPage getPage() {
+ // do nothing
+ return null;
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (!open) {
+ return;
+ }
+
+ try {
+ if (rtreeCursors != null && btreeCursors != null) {
+ for (int i = 0; i < numberOfTrees; i++) {
+ rtreeCursors[i].close();
+ btreeCursors[i].close();
+ }
+ }
+ rtreeCursors = null;
+ btreeCursors = null;
+ } finally {
+ lsmHarness.endSearch(opCtx);
+ }
+
+ open = false;
+ }
+
+ @Override
+ public void setBufferCache(IBufferCache bufferCache) {
+ // do nothing
+ }
+
+ @Override
+ public void setFileId(int fileId) {
+ // do nothing
+ }
+
+ @Override
+ public ITupleReference getTuple() {
+ return frameTuple;
+ }
+
+ @Override
+ public boolean exclusiveLatchNodes() {
+ return false;
+ }
+
}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeCursorInitialState.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeCursorInitialState.java
index 21230b0..27545b91 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeCursorInitialState.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeCursorInitialState.java
@@ -20,7 +20,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
-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.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
@@ -29,35 +28,26 @@
public class LSMRTreeCursorInitialState implements ICursorInitialState {
- private final int numberOfTrees;
private final ITreeIndexFrameFactory rtreeInteriorFrameFactory;
private final ITreeIndexFrameFactory rtreeLeafFrameFactory;
private final ITreeIndexFrameFactory btreeLeafFrameFactory;
private final MultiComparator btreeCmp;
private final MultiComparator hilbertCmp;
- private final ITreeIndexAccessor[] rTreeAccessors;
- private final ITreeIndexAccessor[] bTreeAccessors;
- private final boolean includeMemRTree;
private final ILSMHarness lsmHarness;
private final int[] comparatorFields;
private ISearchOperationCallback searchCallback;
private final List<ILSMComponent> operationalComponents;
- public LSMRTreeCursorInitialState(int numberOfTrees, ITreeIndexFrameFactory rtreeLeafFrameFactory,
+ public LSMRTreeCursorInitialState(ITreeIndexFrameFactory rtreeLeafFrameFactory,
ITreeIndexFrameFactory rtreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
- MultiComparator btreeCmp, ITreeIndexAccessor[] rTreeAccessors, ITreeIndexAccessor[] bTreeAccessors,
- boolean includeMemRTree, ILSMHarness lsmHarness, int[] comparatorFields,
+ MultiComparator btreeCmp, ILSMHarness lsmHarness, int[] comparatorFields,
IBinaryComparatorFactory[] linearizerArray, ISearchOperationCallback searchCallback,
List<ILSMComponent> operationalComponents) {
- this.numberOfTrees = numberOfTrees;
this.rtreeLeafFrameFactory = rtreeLeafFrameFactory;
this.rtreeInteriorFrameFactory = rtreeInteriorFrameFactory;
this.btreeLeafFrameFactory = btreeLeafFrameFactory;
this.btreeCmp = btreeCmp;
- this.rTreeAccessors = rTreeAccessors;
- this.bTreeAccessors = bTreeAccessors;
- this.includeMemRTree = includeMemRTree;
this.lsmHarness = lsmHarness;
this.comparatorFields = comparatorFields;
this.hilbertCmp = MultiComparator.create(linearizerArray);
@@ -73,10 +63,6 @@
return comparatorFields;
}
- public int getNumberOfTrees() {
- return numberOfTrees;
- }
-
public ITreeIndexFrameFactory getRTreeInteriorFrameFactory() {
return rtreeInteriorFrameFactory;
}
@@ -106,18 +92,6 @@
return operationalComponents;
}
- public ITreeIndexAccessor[] getRTreeAccessors() {
- return rTreeAccessors;
- }
-
- public ITreeIndexAccessor[] getBTreeAccessors() {
- return bTreeAccessors;
- }
-
- public boolean getIncludeMemComponent() {
- return includeMemRTree;
- }
-
public ILSMHarness getLSMHarness() {
return lsmHarness;
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeImmutableComponent.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponent.java
similarity index 86%
rename from hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeImmutableComponent.java
rename to hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponent.java
index cb290c7..7bc3f79 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeImmutableComponent.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponent.java
@@ -17,15 +17,15 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractImmutableLSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractDiskLSMComponent;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-public class LSMRTreeImmutableComponent extends AbstractImmutableLSMComponent {
+public class LSMRTreeDiskComponent extends AbstractDiskLSMComponent {
private final RTree rtree;
private final BTree btree;
private final BloomFilter bloomFilter;
- public LSMRTreeImmutableComponent(RTree rtree, BTree btree, BloomFilter bloomFilter) {
+ public LSMRTreeDiskComponent(RTree rtree, BTree btree, BloomFilter bloomFilter) {
this.rtree = rtree;
this.btree = btree;
this.bloomFilter = bloomFilter;
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeComponentFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentFactory.java
similarity index 87%
rename from hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeComponentFactory.java
rename to hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentFactory.java
index 8f618da..a130562 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeComponentFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentFactory.java
@@ -26,12 +26,12 @@
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-public class LSMRTreeComponentFactory implements ILSMComponentFactory {
+public class LSMRTreeDiskComponentFactory implements ILSMComponentFactory {
private final TreeIndexFactory<RTree> rtreeFactory;
private final TreeIndexFactory<BTree> btreeFactory;
private final BloomFilterFactory bloomFilterFactory;
- public LSMRTreeComponentFactory(TreeIndexFactory<RTree> rtreeFactory, TreeIndexFactory<BTree> btreeFactory,
+ public LSMRTreeDiskComponentFactory(TreeIndexFactory<RTree> rtreeFactory, TreeIndexFactory<BTree> btreeFactory,
BloomFilterFactory bloomFilterFactory) {
this.rtreeFactory = rtreeFactory;
this.btreeFactory = btreeFactory;
@@ -41,7 +41,7 @@
@Override
public ILSMComponent createLSMComponentInstance(LSMComponentFileReferences cfr) throws IndexException,
HyracksDataException {
- return new LSMRTreeImmutableComponent(rtreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+ return new LSMRTreeDiskComponent(rtreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
btreeFactory.createIndexInstance(cfr.getDeleteIndexFileReference()),
bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()));
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java
index 68098bb..18d7a7e 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java
@@ -64,8 +64,9 @@
}
@Override
- public void perform() throws HyracksDataException, IndexException {
+ public Boolean call() throws HyracksDataException, IndexException {
accessor.flush(this);
+ return true;
}
@Override
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/LSMRTreeMemoryComponent.java
similarity index 82%
rename from hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMutableComponent.java
rename to hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
index 7d572ad..452643b 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/LSMRTreeMemoryComponent.java
@@ -18,19 +18,18 @@
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.lsm.common.api.IVirtualBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMutableLSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMemoryLSMComponent;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-public class LSMRTreeMutableComponent extends AbstractMutableLSMComponent {
+public class LSMRTreeMemoryComponent extends AbstractMemoryLSMComponent {
private final RTree rtree;
private final BTree btree;
- private final IVirtualBufferCache vbc;
- public LSMRTreeMutableComponent(RTree rtree, BTree btree, IVirtualBufferCache vbc) {
+ public LSMRTreeMemoryComponent(RTree rtree, BTree btree, IVirtualBufferCache vbc, boolean isActive) {
+ super(vbc, isActive);
this.rtree = rtree;
this.btree = btree;
- this.vbc = vbc;
}
public RTree getRTree() {
@@ -42,11 +41,6 @@
}
@Override
- protected boolean isFull() {
- return vbc.isFull();
- }
-
- @Override
protected void reset() throws HyracksDataException {
super.reset();
rtree.deactivate();
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
index ef67a2b..da7a2fb 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
@@ -53,7 +53,7 @@
public Set<IODeviceHandle> getReadDevices() {
Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
for (ILSMComponent o : mergingComponents) {
- LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) o;
+ LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) o;
devs.add(component.getRTree().getFileReference().getDeviceHandle());
if (component.getBTree() != null) {
devs.add(component.getBTree().getFileReference().getDeviceHandle());
@@ -75,8 +75,9 @@
}
@Override
- public void perform() throws HyracksDataException, IndexException {
+ public Boolean call() throws HyracksDataException, IndexException {
accessor.merge(this);
+ return true;
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
index 8607a05..b94feba 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
@@ -24,7 +24,6 @@
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.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
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;
@@ -37,43 +36,69 @@
public final class LSMRTreeOpContext implements ILSMIndexOperationContext {
- public RTreeOpContext rtreeOpContext;
- public BTreeOpContext btreeOpContext;
- public final RTree.RTreeAccessor memRTreeAccessor;
- public final BTree.BTreeAccessor memBTreeAccessor;
+ public RTree.RTreeAccessor[] mutableRTreeAccessors;
+ public RTree.RTreeAccessor currentMutableRTreeAccessor;
+ public BTree.BTreeAccessor[] mutableBTreeAccessors;
+ public BTree.BTreeAccessor currentMutableBTreeAccessor;
+
+ public RTreeOpContext[] rtreeOpContexts;
+ public BTreeOpContext[] btreeOpContexts;
+ public RTreeOpContext currentRTreeOpContext;
+ public BTreeOpContext currentBTreeOpContext;
+
private IndexOperation op;
public final List<ILSMComponent> componentHolder;
public final IModificationOperationCallback modificationCallback;
public final ISearchOperationCallback searchCallback;
- public LSMRTreeOpContext(RTree.RTreeAccessor memRtreeAccessor, IRTreeLeafFrame rtreeLeafFrame,
- IRTreeInteriorFrame rtreeInteriorFrame, ITreeIndexMetaDataFrame rtreeMetaFrame, int rTreeHeightHint,
- BTree.BTreeAccessor memBtreeAccessor, ITreeIndexFrameFactory btreeLeafFrameFactory,
- ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexMetaDataFrame btreeMetaFrame,
- IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
- IModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback) {
- this.memRTreeAccessor = memRtreeAccessor;
- this.memBTreeAccessor = memBtreeAccessor;
+ public LSMRTreeOpContext(List<ILSMComponent> mutableComponents, IRTreeLeafFrame rtreeLeafFrame,
+ IRTreeInteriorFrame rtreeInteriorFrame, ITreeIndexFrameFactory btreeLeafFrameFactory,
+ ITreeIndexFrameFactory btreeInteriorFrameFactory, IBinaryComparatorFactory[] rtreeCmpFactories,
+ IBinaryComparatorFactory[] btreeCmpFactories, IModificationOperationCallback modificationCallback,
+ ISearchOperationCallback searchCallback) {
+ mutableRTreeAccessors = new RTree.RTreeAccessor[mutableComponents.size()];
+ mutableBTreeAccessors = new BTree.BTreeAccessor[mutableComponents.size()];
+ rtreeOpContexts = new RTreeOpContext[mutableComponents.size()];
+ btreeOpContexts = new BTreeOpContext[mutableComponents.size()];
+
+ for (int i = 0; i < mutableComponents.size(); i++) {
+ LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) mutableComponents.get(i);
+ mutableRTreeAccessors[i] = (RTree.RTreeAccessor) mutableComponent.getRTree().createAccessor(
+ NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ mutableBTreeAccessors[i] = (BTree.BTreeAccessor) mutableComponent.getBTree().createAccessor(
+ NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+
+ rtreeOpContexts[i] = mutableRTreeAccessors[i].getOpContext();
+ btreeOpContexts[i] = mutableBTreeAccessors[i].getOpContext();
+ }
+
+ assert mutableComponents.size() > 0;
+ currentRTreeOpContext = rtreeOpContexts[0];
+ currentBTreeOpContext = btreeOpContexts[0];
this.componentHolder = new LinkedList<ILSMComponent>();
this.modificationCallback = modificationCallback;
this.searchCallback = searchCallback;
- this.rtreeOpContext = new RTreeOpContext(rtreeLeafFrame, rtreeInteriorFrame, rtreeMetaFrame, rtreeCmpFactories,
- rTreeHeightHint, NoOpOperationCallback.INSTANCE);
- this.btreeOpContext = new BTreeOpContext(memBtreeAccessor, btreeLeafFrameFactory, btreeInteriorFrameFactory,
- btreeMetaFrame, btreeCmpFactories, NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
}
public void setOperation(IndexOperation newOp) {
reset();
- if (newOp == IndexOperation.INSERT) {
- rtreeOpContext.setOperation(newOp);
- } else if (newOp == IndexOperation.DELETE) {
- btreeOpContext.setOperation(IndexOperation.INSERT);
- }
this.op = newOp;
}
@Override
+ public void setCurrentMutableComponentId(int currentMutableComponentId) {
+ currentMutableRTreeAccessor = mutableRTreeAccessors[currentMutableComponentId];
+ currentMutableBTreeAccessor = mutableBTreeAccessors[currentMutableComponentId];
+ currentRTreeOpContext = rtreeOpContexts[currentMutableComponentId];
+ currentBTreeOpContext = btreeOpContexts[currentMutableComponentId];
+ if (op == IndexOperation.INSERT) {
+ currentRTreeOpContext.setOperation(op);
+ } else if (op == IndexOperation.DELETE) {
+ currentBTreeOpContext.setOperation(IndexOperation.INSERT);
+ }
+ }
+
+ @Override
public void reset() {
componentHolder.clear();
}
@@ -84,7 +109,7 @@
}
public MultiComparator getBTreeMultiComparator() {
- return btreeOpContext.cmp;
+ return currentBTreeOpContext.cmp;
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
index 9b0869c..30dd467 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
@@ -61,7 +61,7 @@
if (currentCursor < numberOfTrees) {
rtreeCursors[currentCursor].reset();
try {
- rTreeAccessors[currentCursor].search(rtreeCursors[currentCursor], rtreeSearchPredicate);
+ rtreeAccessors[currentCursor].search(rtreeCursors[currentCursor], rtreeSearchPredicate);
} catch (IndexException e) {
throw new HyracksDataException(e);
}
@@ -80,14 +80,10 @@
boolean killerTupleFound = false;
for (int i = 0; i <= currentCursor; i++) {
- try {
- btreeCursors[i].reset();
- btreeRangePredicate.setHighKey(currentTuple, true);
- btreeRangePredicate.setLowKey(currentTuple, true);
- bTreeAccessors[i].search(btreeCursors[i], btreeRangePredicate);
- } catch (IndexException e) {
- throw new HyracksDataException(e);
- }
+ btreeCursors[i].reset();
+ btreeRangePredicate.setHighKey(currentTuple, true);
+ btreeRangePredicate.setLowKey(currentTuple, true);
+ btreeAccessors[i].search(btreeCursors[i], btreeRangePredicate);
try {
if (btreeCursors[i].hasNext()) {
killerTupleFound = true;
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
index 31dfe27..dd31165 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
@@ -36,6 +36,10 @@
reset();
}
+ public ILSMIndexOperationContext getOpCtx() {
+ return opCtx;
+ }
+
@Override
public void reset() throws HyracksDataException {
depletedRtreeCursors = new boolean[numberOfTrees];
@@ -44,7 +48,7 @@
for (int i = 0; i < numberOfTrees; i++) {
rtreeCursors[i].reset();
try {
- rTreeAccessors[i].search(rtreeCursors[i], rtreeSearchPredicate);
+ rtreeAccessors[i].search(rtreeCursors[i], rtreeSearchPredicate);
} catch (IndexException e) {
throw new HyracksDataException(e);
}
@@ -103,7 +107,7 @@
btreeCursors[i].reset();
btreeRangePredicate.setHighKey(frameTuple, true);
btreeRangePredicate.setLowKey(frameTuple, true);
- bTreeAccessors[i].search(btreeCursors[i], btreeRangePredicate);
+ btreeAccessors[i].search(btreeCursors[i], btreeRangePredicate);
} catch (IndexException e) {
throw new HyracksDataException(e);
}
@@ -138,7 +142,7 @@
for (int i = 0; i < numberOfTrees; i++) {
rtreeCursors[i].reset();
try {
- rTreeAccessors[i].search(rtreeCursors[i], rtreeSearchPredicate);
+ rtreeAccessors[i].search(rtreeCursors[i], rtreeSearchPredicate);
} catch (IndexException 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/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 0be497a..0996257 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
@@ -16,7 +16,6 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
import java.util.List;
-import java.util.ListIterator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -25,7 +24,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.IIndexBulkLoader;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
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;
@@ -51,6 +49,7 @@
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.LSMIndexSearchCursor;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
@@ -67,7 +66,7 @@
// because it should have a different tuple writer in it's leaf frames.
private final ILSMComponentFactory bulkLoaComponentFactory;
- public LSMRTreeWithAntiMatterTuples(IVirtualBufferCache virtualBufferCache,
+ public LSMRTreeWithAntiMatterTuples(List<IVirtualBufferCache> virtualBufferCaches,
ITreeIndexFrameFactory rtreeInteriorFrameFactory, ITreeIndexFrameFactory rtreeLeafFrameFactory,
ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
ILSMIndexFileManager fileManager, TreeIndexFactory<RTree> diskRTreeFactory,
@@ -76,19 +75,18 @@
ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider 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);
+ super(virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
+ btreeLeafFrameFactory, fileManager, new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(diskRTreeFactory),
+ diskFileMapProvider, fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
+ linearizerArray, 0, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ bulkLoaComponentFactory = new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(bulkLoadRTreeFactory);
this.bTreeTupleSorter = null;
}
@Override
public synchronized void activate() throws HyracksDataException {
super.activate();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
immutableComponents.clear();
List<LSMComponentFileReferences> validFileReferences;
try {
@@ -97,7 +95,7 @@
throw new HyracksDataException(e);
}
for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
- LSMRTreeImmutableComponent component;
+ LSMRTreeDiskComponent component;
try {
component = createDiskComponent(componentFactory,
lsmComonentFileReference.getInsertIndexFileReference(), null, null, false);
@@ -112,9 +110,9 @@
@Override
public synchronized void deactivate(boolean flushOnExit) throws HyracksDataException {
super.deactivate(flushOnExit);
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- RTree rtree = (RTree) ((LSMRTreeImmutableComponent) c).getRTree();
+ RTree rtree = (RTree) ((LSMRTreeDiskComponent) c).getRTree();
rtree.deactivate();
}
isActivated = false;
@@ -128,9 +126,9 @@
@Override
public synchronized void destroy() throws HyracksDataException {
super.destroy();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- RTree rtree = (RTree) ((LSMRTreeImmutableComponent) c).getRTree();
+ RTree rtree = (RTree) ((LSMRTreeDiskComponent) c).getRTree();
rtree.destroy();
}
fileManager.deleteDirs();
@@ -139,9 +137,9 @@
@Override
public synchronized void clear() throws HyracksDataException {
super.clear();
- List<ILSMComponent> immutableComponents = componentsRef.get();
+ List<ILSMComponent> immutableComponents = diskComponents;
for (ILSMComponent c : immutableComponents) {
- RTree rtree = (RTree) ((LSMRTreeImmutableComponent) c).getRTree();
+ RTree rtree = (RTree) ((LSMRTreeDiskComponent) c).getRTree();
rtree.deactivate();
rtree.destroy();
}
@@ -149,65 +147,16 @@
}
@Override
- public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
- throws HyracksDataException, IndexException {
- LSMRTreeOpContext ctx = (LSMRTreeOpContext) ictx;
- List<ILSMComponent> operationalComponents = ictx.getComponentHolder();
- boolean includeMutableComponent = operationalComponents.get(0) == mutableComponent;
- LSMRTreeWithAntiMatterTuplesSearchCursor lsmTreeCursor = (LSMRTreeWithAntiMatterTuplesSearchCursor) cursor;
- int numDiskRComponents = operationalComponents.size();
-
- LSMRTreeCursorInitialState initialState;
- ITreeIndexAccessor[] bTreeAccessors = null;
- if (includeMutableComponent) {
- // Only in-memory BTree
- bTreeAccessors = new ITreeIndexAccessor[1];
- bTreeAccessors[0] = ctx.memBTreeAccessor;
- }
-
- initialState = new LSMRTreeCursorInitialState(numDiskRComponents, rtreeLeafFrameFactory,
- rtreeInteriorFrameFactory, btreeLeafFrameFactory, ctx.getBTreeMultiComparator(), null, bTreeAccessors,
- includeMutableComponent, lsmHarness, comparatorFields, linearizerArray, ctx.searchCallback,
- operationalComponents);
-
- lsmTreeCursor.open(initialState, pred);
-
- ListIterator<ILSMComponent> diskComponentsIter = operationalComponents.listIterator();
- int diskComponentIx = 0;
- if (includeMutableComponent) {
- // Open cursor of in-memory RTree
- ctx.memRTreeAccessor.search(lsmTreeCursor.getMemRTreeCursor(), pred);
- diskComponentIx++;
- diskComponentsIter.next();
- }
-
- // Open cursors of on-disk RTrees.
- ITreeIndexAccessor[] diskRTreeAccessors = new ITreeIndexAccessor[numDiskRComponents];
- while (diskComponentsIter.hasNext()) {
- RTree diskRTree = (RTree) ((LSMRTreeImmutableComponent) diskComponentsIter.next()).getRTree();
- diskRTreeAccessors[diskComponentIx] = diskRTree.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- diskRTreeAccessors[diskComponentIx].search(lsmTreeCursor.getCursor(diskComponentIx), pred);
- diskComponentIx++;
- }
- lsmTreeCursor.initPriorityQueue();
- }
-
- @Override
- public boolean scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+ public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException {
- if (!mutableComponent.isModified()) {
- return false;
- }
+ ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
LSMRTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE);
LSMComponentFileReferences relFlushFileRefs = fileManager.getRelFlushFileReference();
- ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
opCtx.setOperation(IndexOperation.FLUSH);
opCtx.getComponentHolder().add(flushingComponent);
ILSMIndexAccessorInternal accessor = new LSMRTreeWithAntiMatterTuplesAccessor(lsmHarness, opCtx);
ioScheduler.scheduleOperation(new LSMRTreeFlushOperation(accessor, flushingComponent, relFlushFileRefs
.getInsertIndexFileReference(), null, null, callback));
- return true;
}
@Override
@@ -216,14 +165,14 @@
// Renaming order is critical because we use assume ordering when we
// read the file names when we open the tree.
// The RTree should be renamed before the BTree.
- LSMRTreeMutableComponent flushingComponent = (LSMRTreeMutableComponent) flushOp.getFlushingComponent();
+ LSMRTreeMemoryComponent flushingComponent = (LSMRTreeMemoryComponent) flushOp.getFlushingComponent();
ITreeIndexAccessor memRTreeAccessor = flushingComponent.getRTree().createAccessor(
NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor();
SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
- LSMRTreeImmutableComponent component = createDiskComponent(componentFactory, flushOp.getRTreeFlushTarget(),
- null, null, true);
+ LSMRTreeDiskComponent component = createDiskComponent(componentFactory, flushOp.getRTreeFlushTarget(), null,
+ null, true);
RTree diskRTree = component.getRTree();
// scan the memory BTree
@@ -296,13 +245,10 @@
@Override
public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException, IndexException {
- List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
LSMRTreeOpContext rctx = createOpContext(NoOpOperationCallback.INSTANCE);
- rctx.getComponentHolder().addAll(mergingComponents);
- ITreeIndexCursor cursor = new LSMRTreeWithAntiMatterTuplesSearchCursor(ctx);
- ISearchPredicate rtreeSearchPred = new SearchPredicate(null, null);
- search(rctx, cursor, (SearchPredicate) rtreeSearchPred);
rctx.setOperation(IndexOperation.MERGE);
+ List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+ ITreeIndexCursor cursor = new LSMRTreeWithAntiMatterTuplesSearchCursor(rctx);
LSMComponentFileReferences relMergeFileRefs = getMergeTargetFileName(mergingComponents);
ILSMIndexAccessorInternal accessor = new LSMRTreeWithAntiMatterTuplesAccessor(lsmHarness, rctx);
ioScheduler.scheduleOperation(new LSMRTreeMergeOperation(accessor, mergingComponents, cursor, relMergeFileRefs
@@ -310,21 +256,17 @@
}
@Override
- public ILSMComponent merge(List<ILSMComponent> mergedComponents, ILSMIOOperation operation)
- throws HyracksDataException, IndexException {
+ public ILSMComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException {
LSMRTreeMergeOperation mergeOp = (LSMRTreeMergeOperation) operation;
ITreeIndexCursor cursor = mergeOp.getCursor();
- mergedComponents.addAll(mergeOp.getMergingComponents());
-
- // Nothing to merge.
- if (mergedComponents.size() <= 1) {
- cursor.close();
- return null;
- }
+ ISearchPredicate rtreeSearchPred = new SearchPredicate(null, null);
+ ILSMIndexOperationContext opCtx = ((LSMIndexSearchCursor) cursor).getOpCtx();
+ opCtx.getComponentHolder().addAll(mergeOp.getMergingComponents());
+ search(opCtx, cursor, rtreeSearchPred);
// Bulk load the tuples from all on-disk RTrees into the new RTree.
- LSMRTreeImmutableComponent component = createDiskComponent(componentFactory, mergeOp.getRTreeMergeTarget(),
- null, null, true);
+ LSMRTreeDiskComponent component = createDiskComponent(componentFactory, mergeOp.getRTreeMergeTarget(), null,
+ null, true);
RTree mergedRTree = component.getRTree();
IIndexBulkLoader bulkloader = mergedRTree.createBulkLoader(1.0f, false, 0L, false);
try {
@@ -358,7 +300,7 @@
public MultiComparator getMultiComparator() {
LSMRTreeOpContext concreteCtx = (LSMRTreeOpContext) ctx;
- return concreteCtx.rtreeOpContext.cmp;
+ return concreteCtx.currentRTreeOpContext.cmp;
}
}
@@ -397,7 +339,7 @@
} catch (HyracksDataException | IndexException e) {
throw new TreeIndexException(e);
}
- bulkLoader = ((LSMRTreeImmutableComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
+ bulkLoader = ((LSMRTreeDiskComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
numElementsHint, false);
}
@@ -429,8 +371,8 @@
protected void cleanupArtifacts() throws HyracksDataException {
if (!cleanedUpArtifacts) {
cleanedUpArtifacts = true;
- ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
- ((LSMRTreeImmutableComponent) component).getRTree().destroy();
+ ((LSMRTreeDiskComponent) component).getRTree().deactivate();
+ ((LSMRTreeDiskComponent) component).getRTree().destroy();
}
}
@@ -438,7 +380,7 @@
@Override
public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException {
- RTree rtree = ((LSMRTreeImmutableComponent) lsmComponent).getRTree();
+ RTree rtree = ((LSMRTreeDiskComponent) lsmComponent).getRTree();
forceFlushDirtyPages(rtree);
markAsValidInternal(rtree);
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesComponentFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentFactory.java
similarity index 83%
rename from hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesComponentFactory.java
rename to hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentFactory.java
index 39f35b7..fe253cc 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesComponentFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentFactory.java
@@ -23,16 +23,16 @@
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-public class LSMRTreeWithAntiMatterTuplesComponentFactory implements ILSMComponentFactory {
+public class LSMRTreeWithAntiMatterTuplesDiskComponentFactory implements ILSMComponentFactory {
private final TreeIndexFactory<RTree> rtreeFactory;
- public LSMRTreeWithAntiMatterTuplesComponentFactory(TreeIndexFactory<RTree> rtreeFactory) {
+ public LSMRTreeWithAntiMatterTuplesDiskComponentFactory(TreeIndexFactory<RTree> rtreeFactory) {
this.rtreeFactory = rtreeFactory;
}
@Override
public ILSMComponent createLSMComponentInstance(LSMComponentFileReferences cfr) throws IndexException {
- return new LSMRTreeImmutableComponent(rtreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+ return new LSMRTreeDiskComponent(rtreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
null, null);
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
index c937709..cbaf3b3 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
@@ -18,6 +18,7 @@
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.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
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.ICursorInitialState;
@@ -25,74 +26,133 @@
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
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.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
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.RTreeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCursor {
- private RTreeSearchCursor memRTreeCursor;
- private BTreeRangeSearchCursor memBTreeCursor;
+ private ITreeIndexAccessor[] mutableRTreeAccessors;
+ private ITreeIndexAccessor[] btreeAccessors;
+ private RTreeSearchCursor[] mutableRTreeCursors;
+ private ITreeIndexCursor[] btreeCursors;
private RangePredicate btreeRangePredicate;
- private ITreeIndexAccessor memBTreeAccessor;
private boolean foundNext;
private ITupleReference frameTuple;
private int[] comparatorFields;
private MultiComparator btreeCmp;
+ private int currentCursor;
+ private SearchPredicate rtreeSearchPredicate;
+ private int numMutableComponents;
public LSMRTreeWithAntiMatterTuplesSearchCursor(ILSMIndexOperationContext opCtx) {
super(opCtx);
+ currentCursor = 0;
}
@Override
- public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException,
+ IndexException {
LSMRTreeCursorInitialState lsmInitialState = (LSMRTreeCursorInitialState) initialState;
cmp = lsmInitialState.getHilbertCmp();
btreeCmp = lsmInitialState.getBTreeCmp();
- int numDiskRTrees = lsmInitialState.getNumberOfTrees();
- rangeCursors = new RTreeSearchCursor[numDiskRTrees];
- for (int i = 0; i < numDiskRTrees; i++) {
- rangeCursors[i] = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState
- .getRTreeInteriorFrameFactory().createFrame(), (IRTreeLeafFrame) lsmInitialState
- .getRTreeLeafFrameFactory().createFrame());
- }
- includeMemComponent = lsmInitialState.getIncludeMemComponent();
- operationalComponents = lsmInitialState.getOperationalComponents();
- if (includeMemComponent) {
- memRTreeCursor = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState.getRTreeInteriorFrameFactory()
- .createFrame(), (IRTreeLeafFrame) lsmInitialState.getRTreeLeafFrameFactory().createFrame());
- memBTreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) lsmInitialState.getBTreeLeafFrameFactory()
- .createFrame(), false);
- memBTreeAccessor = lsmInitialState.getBTreeAccessors()[0];
- btreeRangePredicate = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp);
- }
lsmHarness = lsmInitialState.getLSMHarness();
comparatorFields = lsmInitialState.getComparatorFields();
+ operationalComponents = lsmInitialState.getOperationalComponents();
+ rtreeSearchPredicate = (SearchPredicate) searchPred;
+
+ includeMutableComponent = false;
+ numMutableComponents = 0;
+ int numImmutableComponents = 0;
+ for (ILSMComponent component : operationalComponents) {
+ if (component.getType() == LSMComponentType.MEMORY) {
+ includeMutableComponent = true;
+ numMutableComponents++;
+ } else {
+ numImmutableComponents++;
+ }
+ }
+ if (includeMutableComponent) {
+ btreeRangePredicate = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp);
+ }
+
+ mutableRTreeCursors = new RTreeSearchCursor[numMutableComponents];
+ mutableRTreeAccessors = new ITreeIndexAccessor[numMutableComponents];
+ btreeCursors = new BTreeRangeSearchCursor[numMutableComponents];
+ btreeAccessors = new ITreeIndexAccessor[numMutableComponents];
+ for (int i = 0; i < numMutableComponents; i++) {
+ ILSMComponent component = operationalComponents.get(i);
+ RTree rtree = (RTree) ((LSMRTreeMemoryComponent) component).getRTree();
+ BTree btree = (BTree) ((LSMRTreeMemoryComponent) component).getBTree();
+ mutableRTreeCursors[i] = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState
+ .getRTreeInteriorFrameFactory().createFrame(), (IRTreeLeafFrame) lsmInitialState
+ .getRTreeLeafFrameFactory().createFrame());
+ btreeCursors[i] = new BTreeRangeSearchCursor((IBTreeLeafFrame) lsmInitialState.getBTreeLeafFrameFactory()
+ .createFrame(), false);
+ btreeAccessors[i] = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ mutableRTreeAccessors[i] = rtree.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ }
+
+ rangeCursors = new RTreeSearchCursor[numImmutableComponents];
+ ITreeIndexAccessor[] immutableRTreeAccessors = new ITreeIndexAccessor[numImmutableComponents];
+ int j = 0;
+ for (int i = numMutableComponents; i < operationalComponents.size(); i++) {
+ ILSMComponent component = operationalComponents.get(i);
+ rangeCursors[j] = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState
+ .getRTreeInteriorFrameFactory().createFrame(), (IRTreeLeafFrame) lsmInitialState
+ .getRTreeLeafFrameFactory().createFrame());
+ RTree rtree = (RTree) ((LSMRTreeDiskComponent) component).getRTree();
+ immutableRTreeAccessors[j] = rtree.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ immutableRTreeAccessors[j].search(rangeCursors[j], searchPred);
+ j++;
+ }
+ searchNextCursor();
setPriorityQueueComparator();
+ initPriorityQueue();
+ }
+
+ private void searchNextCursor() throws HyracksDataException, IndexException {
+ if (currentCursor < numMutableComponents) {
+ mutableRTreeCursors[currentCursor].reset();
+ mutableRTreeAccessors[currentCursor].search(mutableRTreeCursors[currentCursor], rtreeSearchPredicate);
+ }
}
@Override
public boolean hasNext() throws HyracksDataException, IndexException {
- if (includeMemComponent) {
+ if (includeMutableComponent) {
if (foundNext) {
return true;
}
- while (memRTreeCursor.hasNext()) {
- memRTreeCursor.next();
- ITupleReference memRTreeTuple = memRTreeCursor.getTuple();
- if (searchMemBTree(memRTreeTuple)) {
- foundNext = true;
- frameTuple = memRTreeTuple;
- return true;
+
+ while (currentCursor < numMutableComponents) {
+ while (mutableRTreeCursors[currentCursor].hasNext()) {
+ mutableRTreeCursors[currentCursor].next();
+ ITupleReference currentTuple = mutableRTreeCursors[currentCursor].getTuple();
+ if (searchMemBTrees(currentTuple, currentCursor)) {
+ foundNext = true;
+ frameTuple = currentTuple;
+ return true;
+ }
}
+ mutableRTreeCursors[currentCursor].close();
+ currentCursor++;
+ searchNextCursor();
}
while (super.hasNext()) {
super.next();
ITupleReference diskRTreeTuple = super.getTuple();
- if (searchMemBTree(diskRTreeTuple)) {
+ if (searchMemBTrees(diskRTreeTuple, numMutableComponents - 1)) {
foundNext = true;
frameTuple = diskRTreeTuple;
return true;
@@ -107,7 +167,7 @@
@Override
public void next() throws HyracksDataException {
- if (includeMemComponent) {
+ if (includeMutableComponent) {
foundNext = false;
} else {
super.next();
@@ -117,7 +177,7 @@
@Override
public ITupleReference getTuple() {
- if (includeMemComponent) {
+ if (includeMutableComponent) {
return frameTuple;
} else {
return super.getTuple();
@@ -127,48 +187,49 @@
@Override
public void reset() throws HyracksDataException, IndexException {
- if (includeMemComponent) {
- memRTreeCursor.reset();
- memBTreeCursor.reset();
+ if (includeMutableComponent) {
+ for (int i = 0; i < numMutableComponents; i++) {
+ mutableRTreeCursors[i].reset();
+ btreeCursors[i].reset();
+ }
}
+ currentCursor = 0;
super.reset();
}
@Override
public void close() throws HyracksDataException {
- if (includeMemComponent) {
- memRTreeCursor.close();
- memBTreeCursor.close();
+ if (includeMutableComponent) {
+ for (int i = 0; i < numMutableComponents; i++) {
+ mutableRTreeCursors[i].close();
+ btreeCursors[i].close();
+ }
}
+ currentCursor = 0;
super.close();
}
- public ITreeIndexCursor getMemRTreeCursor() {
- return memRTreeCursor;
- }
-
@Override
protected int compare(MultiComparator cmp, ITupleReference tupleA, ITupleReference tupleB) {
return cmp.selectiveFieldCompare(tupleA, tupleB, comparatorFields);
}
- private boolean searchMemBTree(ITupleReference tuple) throws HyracksDataException {
- try {
+ private boolean searchMemBTrees(ITupleReference tuple, int lastBTreeToSearch) throws HyracksDataException,
+ IndexException {
+ for (int i = 0; i <= lastBTreeToSearch; i++) {
+ btreeCursors[i].reset();
btreeRangePredicate.setHighKey(tuple, true);
btreeRangePredicate.setLowKey(tuple, true);
- memBTreeAccessor.search(memBTreeCursor, btreeRangePredicate);
- } catch (IndexException e) {
- throw new HyracksDataException(e);
- }
- try {
- if (memBTreeCursor.hasNext()) {
- return false;
- } else {
- return true;
+ btreeAccessors[i].search(btreeCursors[i], btreeRangePredicate);
+ try {
+ if (btreeCursors[i].hasNext()) {
+ return false;
+ }
+ } finally {
+ btreeCursors[i].close();
}
- } finally {
- memBTreeCursor.close();
}
+ return true;
}
@Override
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 50e5d09..b16d2b6 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
@@ -15,6 +15,8 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.utils;
+import java.util.List;
+
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -58,7 +60,7 @@
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public class LSMRTreeUtils {
- public static LSMRTree createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+ public static LSMRTree createLSMTree(List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
@@ -99,7 +101,7 @@
ILSMIndexFileManager fileNameManager = new LSMRTreeFileManager(diskFileMapProvider, file, diskRTreeFactory,
diskBTreeFactory);
- LSMRTree lsmTree = new LSMRTree(virtualBufferCache, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
+ LSMRTree lsmTree = new LSMRTree(virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory,
bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length,
rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields, linearizerArray,
@@ -108,7 +110,7 @@
}
public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(
- IVirtualBufferCache virtualBufferCache, FileReference file, IBufferCache diskBufferCache,
+ List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
@@ -151,7 +153,7 @@
ILSMIndexFileManager fileNameManager = new LSMRTreeWithAntiMatterTuplesFileManager(diskFileMapProvider, file,
diskRTreeFactory);
- LSMRTreeWithAntiMatterTuples lsmTree = new LSMRTreeWithAntiMatterTuples(virtualBufferCache,
+ LSMRTreeWithAntiMatterTuples lsmTree = new LSMRTreeWithAntiMatterTuples(virtualBufferCaches,
rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
fileNameManager, diskRTreeFactory, bulkLoadRTreeFactory, diskFileMapProvider, typeTraits.length,
rtreeCmpFactories, btreeCmpFactories, linearizerCmpFactory, comparatorFields, linearizerArray,
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
index 528c004..7bb3583 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
@@ -147,7 +147,7 @@
private RTreeOpContext createOpContext(IModificationOperationCallback modificationCallback) {
return new RTreeOpContext((IRTreeLeafFrame) leafFrameFactory.createFrame(),
(IRTreeInteriorFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
- .createFrame(), cmpFactories, 8, modificationCallback);
+ .createFrame(), cmpFactories, modificationCallback);
}
private void insert(ITupleReference tuple, IIndexOperationContext ictx) throws HyracksDataException,
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
index da73f85..a9171c3 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
@@ -31,6 +31,7 @@
public class RTreeOpContext implements IIndexOperationContext {
private static final int INITIAL_TRAVERSE_LIST_SIZE = 100;
+ private static final int INITIAL_HEIGHT = 8;
public final MultiComparator cmp;
public final IRTreeInteriorFrame interiorFrame;
public final IRTreeLeafFrame leafFrame;
@@ -51,20 +52,20 @@
public final IModificationOperationCallback modificationCallback;
public RTreeOpContext(IRTreeLeafFrame leafFrame, IRTreeInteriorFrame interiorFrame,
- ITreeIndexMetaDataFrame metaFrame, IBinaryComparatorFactory[] cmpFactories, int treeHeightHint,
+ ITreeIndexMetaDataFrame metaFrame, IBinaryComparatorFactory[] cmpFactories,
IModificationOperationCallback modificationCallback) {
-
- if (cmpFactories[0] != null) {
+
+ if (cmpFactories[0] != null) {
this.cmp = MultiComparator.create(cmpFactories);
} else {
this.cmp = null;
}
-
+
this.interiorFrame = interiorFrame;
this.leafFrame = leafFrame;
this.metaFrame = metaFrame;
this.modificationCallback = modificationCallback;
- pathList = new PathList(treeHeightHint, treeHeightHint);
+ pathList = new PathList(INITIAL_HEIGHT, INITIAL_HEIGHT);
NSNUpdates = new ArrayList<ICachedPage>();
LSNUpdates = new ArrayList<ICachedPage>();
}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
index 693dc47..6e1995c 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
@@ -29,6 +29,7 @@
public static final int LSM_RTREE_BULKLOAD_ROUNDS = 5;
public static final int LSM_RTREE_MAX_TREES_TO_MERGE = 3;
public static final boolean LSM_RTREE_TEST_RSTAR_POLICY = false;
+ public static final int LSM_RTREE_NUM_MUTABLE_COMPONENTS = 2;
// Test params for BTree, LSMBTree.
public static final int BTREE_NUM_TUPLES_TO_INSERT = 100;
@@ -39,17 +40,18 @@
// Test params for LSMBTree only.
public static final int LSM_BTREE_BULKLOAD_ROUNDS = 5;
public static final int LSM_BTREE_MAX_TREES_TO_MERGE = 10;
+ public static final int LSM_BTREE_NUM_MUTABLE_COMPONENTS = 2;
// Mem configuration for RTree.
public static final int RTREE_PAGE_SIZE = 512;
public static final int RTREE_NUM_PAGES = 1000;
- public static final int RTREE_MAX_OPEN_FILES = 10;
+ public static final int RTREE_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int RTREE_HYRACKS_FRAME_SIZE = 128;
// Mem configuration for LSMRTree and LSMRTreeWithAntiMatterTuples.
public static final int LSM_RTREE_DISK_PAGE_SIZE = 512;
public static final int LSM_RTREE_DISK_NUM_PAGES = 1000;
- public static final int LSM_RTREE_DISK_MAX_OPEN_FILES = 2000;
+ public static final int LSM_RTREE_DISK_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int LSM_RTREE_MEM_PAGE_SIZE = 512;
public static final int LSM_RTREE_MEM_NUM_PAGES = 1000;
public static final int LSM_RTREE_HYRACKS_FRAME_SIZE = 128;
@@ -58,13 +60,13 @@
// Mem configuration for BTree.
public static final int BTREE_PAGE_SIZE = 256;
public static final int BTREE_NUM_PAGES = 100;
- public static final int BTREE_MAX_OPEN_FILES = 10;
+ public static final int BTREE_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int BTREE_HYRACKS_FRAME_SIZE = 128;
// Mem configuration for LSMBTree.
public static final int LSM_BTREE_DISK_PAGE_SIZE = 256;
public static final int LSM_BTREE_DISK_NUM_PAGES = 1000;
- public static final int LSM_BTREE_DISK_MAX_OPEN_FILES = 200;
+ public static final int LSM_BTREE_DISK_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int LSM_BTREE_MEM_PAGE_SIZE = 256;
public static final int LSM_BTREE_MEM_NUM_PAGES = 100;
public static final int LSM_BTREE_HYRACKS_FRAME_SIZE = 128;
@@ -73,11 +75,13 @@
// Mem configuration for Inverted Index.
public static final int LSM_INVINDEX_DISK_PAGE_SIZE = 1024;
public static final int LSM_INVINDEX_DISK_NUM_PAGES = 1000;
- public static final int LSM_INVINDEX_DISK_MAX_OPEN_FILES = 1000;
+ public static final int LSM_INVINDEX_DISK_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int LSM_INVINDEX_MEM_PAGE_SIZE = 1024;
public static final int LSM_INVINDEX_MEM_NUM_PAGES = 100;
public static final int LSM_INVINDEX_HYRACKS_FRAME_SIZE = 32768;
public static final double LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE = 0.01;
+ public static final int LSM_INVINDEX_NUM_MUTABLE_COMPONENTS = 2;
+
// Test parameters.
public static final int LSM_INVINDEX_NUM_DOCS_TO_INSERT = 100;
// Used for full-fledged search test.
@@ -100,7 +104,7 @@
// Mem configuration for BloomFilter.
public static final int BLOOM_FILTER_PAGE_SIZE = 256;
public static final int BLOOM_FILTER_NUM_PAGES = 1000;
- public static final int BLOOM_FILTER_MAX_OPEN_FILES = 10;
+ public static final int BLOOM_FILTER_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int BLOOM_FILTER_HYRACKS_FRAME_SIZE = 128;
}
@@ -111,9 +115,12 @@
public static final int RTREE_NUM_INSERT_ROUNDS = 2;
public static final int RTREE_NUM_DELETE_ROUNDS = 2;
public static final int RTREE_MULTITHREAD_NUM_OPERATIONS = 10000;
+public static final boolean RTREE_TEST_RSTAR_POLICY = true;
// Test params for LSMRTree and LSMRTreeWithAntiMatterTuples.
public static final int LSM_RTREE_BULKLOAD_ROUNDS = 5;
+public static final boolean LSM_RTREE_TEST_RSTAR_POLICY = false;
public static final int LSM_RTREE_MAX_TREES_TO_MERGE = 3;
+public static final int LSM_RTREE_NUM_MUTABLE_COMPONENTS = 2;
// Test params for BTree, LSMBTree.
public static final int BTREE_NUM_TUPLES_TO_INSERT = 10000;
@@ -124,46 +131,51 @@
// Test params for LSMBTree only.
public static final int LSM_BTREE_BULKLOAD_ROUNDS = 5;
public static final int LSM_BTREE_MAX_TREES_TO_MERGE = 10;
+public static final int LSM_BTREE_NUM_MUTABLE_COMPONENTS = 2;
// Mem configuration for RTree.
public static final int RTREE_PAGE_SIZE = 512;
public static final int RTREE_NUM_PAGES = 1000;
-public static final int RTREE_MAX_OPEN_FILES = 10;
+public static final int RTREE_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int RTREE_HYRACKS_FRAME_SIZE = 128;
// Mem configuration for LSMRTree and LSMRTreeWithAntiMatterTuples.
-public static final int LSM_RTREE_DISK_PAGE_SIZE = 256;
-public static final int LSM_RTREE_DISK_NUM_PAGES = 1000;
-public static final int LSM_RTREE_DISK_MAX_OPEN_FILES = 2000;
-public static final int LSM_RTREE_MEM_PAGE_SIZE = 256;
+public static final int LSM_RTREE_DISK_PAGE_SIZE = 512;
+public static final int LSM_RTREE_DISK_NUM_PAGES = 10000;
+public static final int LSM_RTREE_DISK_MAX_OPEN_FILES = Integer.MAX_VALUE;
+public static final int LSM_RTREE_MEM_PAGE_SIZE = 512;
public static final int LSM_RTREE_MEM_NUM_PAGES = 1000;
public static final int LSM_RTREE_HYRACKS_FRAME_SIZE = 128;
+public static final double LSM_RTREE_BLOOMFILTER_FALSE_POSITIVE_RATE = 0.01;
// Mem configuration for BTree.
public static final int BTREE_PAGE_SIZE = 256;
public static final int BTREE_NUM_PAGES = 100;
-public static final int BTREE_MAX_OPEN_FILES = 10;
+public static final int BTREE_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int BTREE_HYRACKS_FRAME_SIZE = 128;
// Mem configuration for LSMBTree.
public static final int LSM_BTREE_DISK_PAGE_SIZE = 256;
-public static final int LSM_BTREE_DISK_NUM_PAGES = 1000;
-public static final int LSM_BTREE_DISK_MAX_OPEN_FILES = 200;
+public static final int LSM_BTREE_DISK_NUM_PAGES = 10000;
+public static final int LSM_BTREE_DISK_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int LSM_BTREE_MEM_PAGE_SIZE = 256;
public static final int LSM_BTREE_MEM_NUM_PAGES = 100;
public static final int LSM_BTREE_HYRACKS_FRAME_SIZE = 128;
+public static final double LSM_BTREE_BLOOMFILTER_FALSE_POSITIVE_RATE = 0.01;
// Mem configuration for Inverted Index.
public static final int INVINDEX_PAGE_SIZE = 32768;
public static final int INVINDEX_NUM_PAGES = 100;
-public static final int INVINDEX_MAX_OPEN_FILES = 10;
+public static final int INVINDEX_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int INVINDEX_HYRACKS_FRAME_SIZE = 32768;
+public static final double LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE = 0.01;
+public static final int LSM_INVINDEX_NUM_MUTABLE_COMPONENTS = 2;
// Mem configuration for Inverted Index.
public static final int LSM_INVINDEX_DISK_PAGE_SIZE = 1024;
-public static final int LSM_INVINDEX_DISK_NUM_PAGES = 1000;
-public static final int LSM_INVINDEX_DISK_MAX_OPEN_FILES = 1000;
+public static final int LSM_INVINDEX_DISK_NUM_PAGES = 10000;
+public static final int LSM_INVINDEX_DISK_MAX_OPEN_FILES = Integer.MAX_VALUE;
public static final int LSM_INVINDEX_MEM_PAGE_SIZE = 1024;
public static final int LSM_INVINDEX_MEM_NUM_PAGES = 100;
public static final int LSM_INVINDEX_HYRACKS_FRAME_SIZE = 32768;
@@ -182,4 +194,13 @@
// Allocate a generous size to make sure we have enough elements for all tests.
public static final int LSM_INVINDEX_SCAN_COUNT_ARRAY_SIZE = 1000000;
public static final int LSM_INVINDEX_MULTITHREAD_NUM_OPERATIONS = 10000;
+
+// Test params for BloomFilter
+public static final int BLOOM_FILTER_NUM_TUPLES_TO_INSERT = 10000;
+
+// Mem configuration for BloomFilter.
+public static final int BLOOM_FILTER_PAGE_SIZE = 256;
+public static final int BLOOM_FILTER_NUM_PAGES = 1000;
+public static final int BLOOM_FILTER_MAX_OPEN_FILES = Integer.MAX_VALUE;
+public static final int BLOOM_FILTER_HYRACKS_FRAME_SIZE = 128;
*/
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index 44ab0d9..705a976 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -134,12 +134,6 @@
TupleUtils.createIntegerTuple(tb, tuple, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
Math.max(p1y, p2y), pk1, pk2);
- if (LOGGER.isLoggable(Level.INFO)) {
- if (i % 1000 == 0) {
- LOGGER.info("Inserting " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
- + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y) + ", " + pk1 + ", " + pk2);
- }
- }
try {
indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
@@ -494,13 +488,6 @@
TupleUtils.createDoubleTuple(tb, tuple, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.min(p1z, p2z),
Math.max(p1x, p2x), Math.max(p1y, p2y), Math.max(p1z, p2z), pk);
- if (LOGGER.isLoggable(Level.INFO)) {
- if (i % 1000 == 0) {
- LOGGER.info("Inserting " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
- + Math.min(p1z, p2z) + " " + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y) + " "
- + Math.max(p1z, p2z) + ", " + pk);
- }
- }
try {
indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
@@ -608,11 +595,6 @@
TupleUtils.createIntegerTuple(tb, tuple, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
Math.max(p1y, p2y), pk);
- if (LOGGER.isLoggable(Level.INFO)) {
- if (i % 1000 == 0) {
- LOGGER.info("Inserting " + i);
- }
- }
try {
indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
@@ -626,11 +608,6 @@
int delDone = 0;
for (int i = 0; i < numInserts; i++) {
TupleUtils.createIntegerTuple(tb, tuple, p1xs[i], p1ys[i], p2xs[i], p2ys[i], pks[i]);
- if (LOGGER.isLoggable(Level.INFO)) {
- if (i % 1000 == 0) {
- LOGGER.info("Deleting " + i);
- }
- }
try {
indexAccessor.delete(tuple);
delDone++;
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
index 9db21cb..afbb86b 100644
--- 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
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.hyracks.test.support;
+import java.util.ArrayList;
+import java.util.List;
+
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;
@@ -33,7 +36,12 @@
}
@Override
- public IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx) {
- return new VirtualBufferCache(new HeapBufferAllocator(), pageSize, numPages);
+ public List<IVirtualBufferCache> getVirtualBufferCaches(IHyracksTaskContext ctx) {
+ List<IVirtualBufferCache> vbcs = new ArrayList<IVirtualBufferCache>();
+ for (int i = 0; i < 2; i++) {
+ IVirtualBufferCache vbc = new VirtualBufferCache(new HeapBufferAllocator(), pageSize, numPages / 2);
+ vbcs.add(vbc);
+ }
+ return vbcs;
}
}
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 e0a2a46..753be5d 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,7 +51,7 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 122b788..646343a 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,7 +51,7 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 60e0492..b0617a1 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,7 +34,7 @@
@Override
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
int[] bloomFilterKeyFields) throws TreeIndexException {
- return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 76a0206..bce69a2 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,7 +51,7 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 639fd47..0f903e5 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
@@ -53,7 +53,7 @@
@Override
public void setup() throws Exception {
harness.setUp();
- testCtx = LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ testCtx = LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 e3c86df..ab46f3f 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,7 +50,7 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 77b65ca..cc3389f 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
@@ -42,7 +42,7 @@
@Override
protected void createIndexInstance() throws Exception {
- index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
SerdeUtils.serdesToTypeTraits(keySerdes),
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
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 a7c8b81..d7d29a8 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,7 +52,7 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 e15e80a..ce9ae0e 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
@@ -49,7 +49,7 @@
@Override
protected void createIndexInstance() throws Exception {
- index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
SerdeUtils.serdesToTypeTraits(keySerdes),
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
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 db221a2..6f8ef84 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,7 +51,7 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 9dfa713..1ae201f 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,7 +50,7 @@
@Override
protected ITreeIndex createIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
int[] bloomFilterKeyFields) throws TreeIndexException {
- return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 fc6642c..3591f78a 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
@@ -17,7 +17,9 @@
import java.io.File;
import java.text.SimpleDateFormat;
+import java.util.ArrayList;
import java.util.Date;
+import java.util.List;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -88,10 +90,15 @@
ioDeviceId = 0;
IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- IVirtualBufferCache virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), inMemPageSize,
- inMemNumPages);
+ List<IVirtualBufferCache> virtualBufferCaches = new ArrayList<IVirtualBufferCache>();
+ for (int i = 0; i < 2; i++) {
+ IVirtualBufferCache virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), inMemPageSize,
+ inMemNumPages);
+ virtualBufferCaches.add(virtualBufferCache);
+ }
+
this.ioScheduler = SynchronousScheduler.INSTANCE;
- lsmtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, bufferCache, fmp, typeTraits, cmpFactories,
+ lsmtree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, bufferCache, fmp, typeTraits, cmpFactories,
bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallback.INSTANCE);
}
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 ab78ee61..39e7184 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
@@ -16,6 +16,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.btree.util;
import java.util.Collection;
+import java.util.List;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -62,7 +63,7 @@
upsertCheckTuple(checkTuple, checkTuples);
}
- public static LSMBTreeTestContext create(IVirtualBufferCache virtualBufferCache, FileReference file,
+ public static LSMBTreeTestContext create(List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
int numKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
@@ -73,7 +74,7 @@
for (int i = 0; i < numKeyFields; ++i) {
bloomFilterKeyFields[i] = i;
}
- LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, diskBufferCache, diskFileMapProvider,
+ LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, diskBufferCache, diskFileMapProvider,
typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
ioScheduler, ioOpCallbackProvider);
LSMBTreeTestContext testCtx = new LSMBTreeTestContext(fieldSerdes, lsmTree);
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 e643cf9..216861f 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
@@ -18,7 +18,9 @@
import java.io.File;
import java.io.FilenameFilter;
import java.text.SimpleDateFormat;
+import java.util.ArrayList;
import java.util.Date;
+import java.util.List;
import java.util.Random;
import java.util.logging.Logger;
@@ -60,12 +62,13 @@
protected final int memNumPages;
protected final int hyracksFrameSize;
protected final double bloomFilterFalsePositiveRate;
+ protected final int numMutableComponents;
protected IOManager ioManager;
protected int ioDeviceId;
protected IBufferCache diskBufferCache;
protected IFileMapProvider diskFileMapProvider;
- protected IVirtualBufferCache virtualBufferCache;
+ protected List<IVirtualBufferCache> virtualBufferCaches;
protected IHyracksTaskContext ctx;
protected ILSMIOOperationScheduler ioScheduler;
protected ILSMMergePolicy mergePolicy;
@@ -90,20 +93,7 @@
this.mergePolicy = NoMergePolicy.INSTANCE;
this.opTracker = new ThreadCountingTracker();
this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
- }
-
- public LSMBTreeTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
- int memNumPages, int hyracksFrameSize, double bloomFilterFalsePositiveRate) {
- this.diskPageSize = diskPageSize;
- this.diskNumPages = diskNumPages;
- this.diskMaxOpenFiles = diskMaxOpenFiles;
- this.memPageSize = memPageSize;
- this.memNumPages = memNumPages;
- this.hyracksFrameSize = hyracksFrameSize;
- this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
- this.ioScheduler = SynchronousScheduler.INSTANCE;
- this.mergePolicy = NoMergePolicy.INSTANCE;
- this.opTracker = new ThreadCountingTracker();
+ this.numMutableComponents = AccessMethodTestsConfig.LSM_BTREE_NUM_MUTABLE_COMPONENTS;
}
public void setUp() throws HyracksException {
@@ -116,7 +106,12 @@
TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
+ virtualBufferCaches = new ArrayList<IVirtualBufferCache>();
+ for (int i = 0; i < numMutableComponents; i++) {
+ IVirtualBufferCache virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize,
+ memNumPages / numMutableComponents);
+ virtualBufferCaches.add(virtualBufferCache);
+ }
rnd.setSeed(RANDOM_SEED);
}
@@ -179,8 +174,8 @@
return diskFileMapProvider;
}
- public IVirtualBufferCache getVirtualBufferCache() {
- return virtualBufferCache;
+ public List<IVirtualBufferCache> getVirtualBufferCaches() {
+ return virtualBufferCaches;
}
public double getBoomFilterFalsePositiveRate() {
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 7276464..48176b7 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
@@ -18,7 +18,9 @@
import java.io.File;
import java.io.FilenameFilter;
import java.text.SimpleDateFormat;
+import java.util.ArrayList;
import java.util.Date;
+import java.util.List;
import java.util.Random;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -27,14 +29,12 @@
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.IVirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
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.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;
@@ -58,13 +58,13 @@
protected final int memNumPages;
protected final int hyracksFrameSize;
protected final double bloomFilterFalsePositiveRate;
+ protected final int numMutableComponents;
protected IOManager ioManager;
protected int ioDeviceId;
protected IBufferCache diskBufferCache;
protected IFileMapProvider diskFileMapProvider;
- protected IVirtualBufferCache virtualBufferCache;
- protected IVirtualFreePageManager virtualFreePageManager;
+ protected List<IVirtualBufferCache> virtualBufferCaches;
protected IHyracksTaskContext ctx;
protected ILSMIOOperationScheduler ioScheduler;
protected ILSMMergePolicy mergePolicy;
@@ -91,20 +91,7 @@
this.mergePolicy = NoMergePolicy.INSTANCE;
this.opTracker = new ThreadCountingTracker();
this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
- }
-
- public LSMInvertedIndexTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
- int memNumPages, int hyracksFrameSize, double bloomFilterFalsePositiveRate) {
- this.diskPageSize = diskPageSize;
- this.diskNumPages = diskNumPages;
- this.diskMaxOpenFiles = diskMaxOpenFiles;
- this.memPageSize = memPageSize;
- this.memNumPages = memNumPages;
- this.hyracksFrameSize = hyracksFrameSize;
- this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
- this.ioScheduler = SynchronousScheduler.INSTANCE;
- this.mergePolicy = NoMergePolicy.INSTANCE;
- this.opTracker = new ThreadCountingTracker();
+ this.numMutableComponents = AccessMethodTestsConfig.LSM_INVINDEX_NUM_MUTABLE_COMPONENTS;
}
public void setUp() throws HyracksException {
@@ -116,10 +103,13 @@
TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- virtualBufferCache = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(),
- memPageSize, memNumPages));
- virtualBufferCache.open();
- virtualFreePageManager = new VirtualFreePageManager(memNumPages);
+ virtualBufferCaches = new ArrayList<IVirtualBufferCache>();
+ for (int i = 0; i < numMutableComponents; i++) {
+ IVirtualBufferCache virtualBufferCache = new MultitenantVirtualBufferCache(new VirtualBufferCache(
+ new HeapBufferAllocator(), memPageSize, memNumPages / numMutableComponents));
+ virtualBufferCaches.add(virtualBufferCache);
+ virtualBufferCache.open();
+ }
rnd.setSeed(RANDOM_SEED);
invIndexFileRef = ioManager.getIODevices().get(0).createFileReference(onDiskDir + invIndexFileName);
}
@@ -141,7 +131,9 @@
}
}
dir.delete();
- virtualBufferCache.close();
+ for (int i = 0; i < numMutableComponents; i++) {
+ virtualBufferCaches.get(i).close();
+ }
}
public FileReference getInvListsFileRef() {
@@ -188,18 +180,14 @@
return diskFileMapProvider;
}
- public IVirtualBufferCache getVirtualBufferCache() {
- return virtualBufferCache;
+ public List<IVirtualBufferCache> getVirtualBufferCaches() {
+ return virtualBufferCaches;
}
public double getBoomFilterFalsePositiveRate() {
return bloomFilterFalsePositiveRate;
}
- public IVirtualFreePageManager getVirtualFreePageManager() {
- return virtualFreePageManager;
- }
-
public IHyracksTaskContext getHyracksTastContext() {
return ctx;
}
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 05dc1b5..5edf74e 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
@@ -36,6 +36,7 @@
import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.InvertedIndexException;
@@ -118,18 +119,20 @@
}
// Create index and test context.
IInvertedIndex invIndex;
+ assert harness.getVirtualBufferCaches().size() > 0;
switch (invIndexType) {
case INMEMORY: {
- invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(harness.getVirtualBufferCache(),
- harness.getVirtualFreePageManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory, new FileReference(new File(harness.getOnDiskDir())));
+ invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(harness.getVirtualBufferCaches().get(0),
+ new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0).getNumPages()),
+ invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+ new FileReference(new File(harness.getOnDiskDir())));
break;
}
case PARTITIONED_INMEMORY: {
invIndex = InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(harness
- .getVirtualBufferCache(), harness.getVirtualFreePageManager(), invListTypeTraits,
- invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, new FileReference(
- new File(harness.getOnDiskDir())));
+ .getVirtualBufferCaches().get(0), new VirtualFreePageManager(harness.getVirtualBufferCaches()
+ .get(0).getNumPages()), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+ tokenCmpFactories, tokenizerFactory, new FileReference(new File(harness.getOnDiskDir())));
break;
}
case ONDISK: {
@@ -145,7 +148,7 @@
break;
}
case LSM: {
- invIndex = InvertedIndexUtils.createLSMInvertedIndex(harness.getVirtualBufferCache(),
+ invIndex = InvertedIndexUtils.createLSMInvertedIndex(harness.getVirtualBufferCaches(),
harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
@@ -154,7 +157,7 @@
break;
}
case PARTITIONED_LSM: {
- invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(harness.getVirtualBufferCache(),
+ invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(harness.getVirtualBufferCaches(),
harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
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 3fcb4e5..09daaab 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,7 +54,7 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 467916e..f5b4ae4 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,7 +54,7 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 a35170c..c94bb48 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,7 +37,7 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
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 44da0c0..858f90a 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,7 +54,7 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 b97bebb..61692b8 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
@@ -63,7 +63,7 @@
@Override
public void setup() throws Exception {
harness.setUp();
- testCtx = LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ testCtx = LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
numKeys, RTreePolicyType.RTREE, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 b13ce4d..436f9b1 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,7 +53,7 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 2baf485..97a3dcc 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,7 +54,7 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 9c440cc..843dcc7 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,7 +54,7 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 909d0cd..6ffa59a 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,7 +37,7 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(),
+ return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
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 6b35192..1e1f46d 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,7 +54,7 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 6dcbdf2..607e980 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
@@ -63,7 +63,7 @@
@Override
public void setup() throws Exception {
harness.setUp();
- testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
+ testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 dc81fc3..bdd80aa 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,7 +53,7 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
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 850d540..0c0d804 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,7 +57,7 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
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 4a6b462..8b54373 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,7 +57,7 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(),
+ return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
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 8c59e31..9536e96 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
@@ -16,6 +16,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.util;
import java.util.Collection;
+import java.util.List;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -66,7 +67,7 @@
return lsmTree.getComparatorFactories();
}
- public static LSMRTreeTestContext create(IVirtualBufferCache virtualBufferCache, FileReference file,
+ public static LSMRTreeTestContext create(List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
@@ -77,7 +78,7 @@
.serdesToComparatorFactories(fieldSerdes, numKeyFields);
IBinaryComparatorFactory[] btreeCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
fieldSerdes.length);
- LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(virtualBufferCache, file, diskBufferCache, diskFileMapProvider,
+ LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(virtualBufferCaches, file, diskBufferCache, diskFileMapProvider,
typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
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 118b1bc..b67274e 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
@@ -18,7 +18,9 @@
import java.io.File;
import java.io.FilenameFilter;
import java.text.SimpleDateFormat;
+import java.util.ArrayList;
import java.util.Date;
+import java.util.List;
import java.util.Random;
import java.util.logging.Logger;
@@ -57,12 +59,13 @@
protected final int memNumPages;
protected final int hyracksFrameSize;
protected final double bloomFilterFalsePositiveRate;
+ protected final int numMutableComponents;
protected IOManager ioManager;
protected int ioDeviceId;
protected IBufferCache diskBufferCache;
protected IFileMapProvider diskFileMapProvider;
- protected IVirtualBufferCache virtualBufferCache;
+ protected List<IVirtualBufferCache> virtualBufferCaches;
protected IHyracksTaskContext ctx;
protected ILSMIOOperationScheduler ioScheduler;
protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
@@ -87,20 +90,7 @@
this.mergePolicy = NoMergePolicy.INSTANCE;
this.opTracker = new ThreadCountingTracker();
this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
- }
-
- public LSMRTreeTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
- int memNumPages, int hyracksFrameSize, double bloomFilterFalsePositiveRate) {
- this.diskPageSize = diskPageSize;
- this.diskNumPages = diskNumPages;
- this.diskMaxOpenFiles = diskMaxOpenFiles;
- this.memPageSize = memPageSize;
- this.memNumPages = memNumPages;
- this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
- this.hyracksFrameSize = hyracksFrameSize;
- this.ioScheduler = SynchronousScheduler.INSTANCE;
- this.mergePolicy = NoMergePolicy.INSTANCE;
- this.opTracker = new ThreadCountingTracker();
+ this.numMutableComponents = AccessMethodTestsConfig.LSM_RTREE_NUM_MUTABLE_COMPONENTS;
}
public void setUp() throws HyracksException {
@@ -113,7 +103,12 @@
TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
+ virtualBufferCaches = new ArrayList<IVirtualBufferCache>();
+ for (int i = 0; i < numMutableComponents; i++) {
+ IVirtualBufferCache virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize,
+ memNumPages / numMutableComponents);
+ virtualBufferCaches.add(virtualBufferCache);
+ }
rnd.setSeed(RANDOM_SEED);
}
@@ -175,8 +170,8 @@
return diskFileMapProvider;
}
- public IVirtualBufferCache getVirtualBufferCache() {
- return virtualBufferCache;
+ public List<IVirtualBufferCache> getVirtualBufferCaches() {
+ return virtualBufferCaches;
}
public double getBoomFilterFalsePositiveRate() {
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 95716f6..069524c 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
@@ -16,6 +16,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.util;
import java.util.Collection;
+import java.util.List;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -67,7 +68,7 @@
return lsmTree.getComparatorFactories();
}
- public static LSMRTreeWithAntiMatterTuplesTestContext create(IVirtualBufferCache virtualBufferCache,
+ public static LSMRTreeWithAntiMatterTuplesTestContext create(List<IVirtualBufferCache> virtualBufferCaches,
FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
int numKeyFields, RTreePolicyType rtreePolicyType, ILSMMergePolicy mergePolicy,
@@ -78,7 +79,7 @@
.serdesToComparatorFactories(fieldSerdes, numKeyFields);
IBinaryComparatorFactory[] btreeCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
fieldSerdes.length);
- LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCache,
+ LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCaches,
file, diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
index 51a9ce3..d68ad2c 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
@@ -538,4 +538,15 @@
public static boolean useLSM(Configuration conf) {
return conf.getBoolean(PregelixJob.UPDATE_INTENSIVE, false);
}
+
+ /***
+ * Get the spilling dir name for global aggregates
+ *
+ * @param conf
+ * @param superStep
+ * @return the spilling dir name
+ */
+ public static String getGlobalAggregateSpillingDirName(Configuration conf, long superStep) {
+ return "/tmp/pregelix/agg/" + conf.get(PregelixJob.JOB_ID) + "/" + superStep;
+ }
}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java
index 24105ae..a0f67e3 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java
@@ -15,6 +15,14 @@
package edu.uci.ics.pregelix.api.util;
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -41,4 +49,19 @@
}
}
+ public static void flushTupleToHDFS(ArrayTupleBuilder atb, Configuration conf, long superStep)
+ throws HyracksDataException {
+ try {
+ if (atb.getSize()>0) {
+ FileSystem dfs = FileSystem.get(conf);
+ String fileName = BspUtils.getGlobalAggregateSpillingDirName(conf, superStep) +"/" + UUID.randomUUID();
+ FSDataOutputStream dos = dfs.create(new Path(fileName), true);
+ dos.write(atb.getByteArray(), 0, atb.getSize());
+ dos.flush();
+ dos.close();
+ }
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
}
diff --git a/pregelix/pregelix-core/pom.xml b/pregelix/pregelix-core/pom.xml
index a3f24f4..3d1699f 100644
--- a/pregelix/pregelix-core/pom.xml
+++ b/pregelix/pregelix-core/pom.xml
@@ -90,21 +90,55 @@
<version>1.3</version>
<executions>
<execution>
+ <id>pregelix</id>
<configuration>
+ <platforms>
+ <platform>unix</platform>
+ </platforms>
<programs>
<program>
<mainClass>org.apache.hadoop.util.RunJar</mainClass>
- <name>pregelix-obselete</name>
+ <name>pregelix</name>
</program>
</programs>
<repositoryLayout>flat</repositoryLayout>
<repositoryName>lib</repositoryName>
+ <configurationDirectory>etc:"$HADOOP_HOME"/conf:/etc/hadoop/conf:"$1"</configurationDirectory>
</configuration>
<phase>package</phase>
<goals>
<goal>assemble</goal>
</goals>
</execution>
+ <execution>
+ <id>cc_nc_drivers</id>
+ <configuration>
+ <platforms>
+ <platform>unix</platform>
+ </platforms>
+ <programs>
+ <program>
+ <mainClass>edu.uci.ics.hyracks.control.cc.CCDriver</mainClass>
+ <name>pregelixcc</name>
+ </program>
+ <program>
+ <mainClass>edu.uci.ics.hyracks.control.nc.NCDriver</mainClass>
+ <name>pregelixnc</name>
+ <commandLineArguments>
+ <commandLineArgument>-app-nc-main-class</commandLineArgument>
+ <commandLineArgument>edu.uci.ics.pregelix.runtime.bootstrap.NCApplicationEntryPoint</commandLineArgument>
+ </commandLineArguments>
+ </program>
+ </programs>
+ <repositoryLayout>flat</repositoryLayout>
+ <repositoryName>lib</repositoryName>
+ <configurationDirectory>etc:"$HADOOP_HOME"/conf:/etc/hadoop/conf</configurationDirectory>
+ </configuration>
+ <phase>package</phase>
+ <goals>
+ <goal>assemble</goal>
+ </goals>
+ </execution>
</executions>
</plugin>
<plugin>
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
index 3e6e9a5..7bd2cf8 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
@@ -99,7 +99,7 @@
IntWritable lastSnapshotSuperstep = new IntWritable(0);
boolean failed = false;
int retryCount = 0;
- int maxRetryCount = 3;
+ int maxRetryCount = 1;
do {
try {
@@ -142,12 +142,11 @@
//restart from snapshot
failed = true;
retryCount++;
- ioe.printStackTrace();
+ throw new HyracksException(ioe);
}
} while (failed && retryCount < maxRetryCount);
LOG.info("job finished");
} catch (Exception e) {
- e.printStackTrace();
throw new HyracksException(e);
}
}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
index d37c6fd..36723a6 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
@@ -108,11 +108,13 @@
import edu.uci.ics.pregelix.dataflow.VertexFileScanOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.VertexWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.bootstrap.IndexLifeCycleManagerProvider;
import edu.uci.ics.pregelix.runtime.bootstrap.StorageManagerInterface;
import edu.uci.ics.pregelix.runtime.bootstrap.VirtualBufferCacheProvider;
+import edu.uci.ics.pregelix.runtime.touchpoint.RecoveryRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.VertexPartitionComputerFactory;
@@ -374,9 +376,10 @@
@Override
public JobSpecification[] generateCheckpointing(int lastSuccessfulIteration) throws HyracksException {
try {
+
PregelixJob tmpJob = this.createCloneJob("Vertex checkpointing for job " + jobId, pregelixJob);
tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
- FileOutputFormat.setOutputPath(tmpJob, new Path(vertexCheckpointPath));
+ FileOutputFormat.setOutputPath(tmpJob, new Path(vertexCheckpointPath + "/" + lastSuccessfulIteration));
tmpJob.setOutputKeyClass(NullWritable.class);
tmpJob.setOutputValueClass(BspUtils.getVertexClass(tmpJob.getConfiguration()));
JobSpecification vertexCkpSpec = scanIndexWriteToHDFS(tmpJob.getConfiguration());
@@ -409,7 +412,7 @@
try {
PregelixJob tmpJob = this.createCloneJob("Vertex checkpoint loading for job " + jobId, pregelixJob);
tmpJob.setVertexInputFormatClass(InternalVertexInputFormat.class);
- FileInputFormat.setInputPaths(tmpJob, new Path(vertexCheckpointPath));
+ FileInputFormat.setInputPaths(tmpJob, new Path(vertexCheckpointPath + "/" + lastCheckpointedIteration));
JobSpecification vertexLoadSpec = loadHDFSData(tmpJob.getConfiguration());
JobSpecification[] stateLoadSpecs = generateStateCheckpointLoading(lastCheckpointedIteration, tmpJob);
JobSpecification[] specs = new JobSpecification[1 + stateLoadSpecs.length];
@@ -637,7 +640,7 @@
MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
ClusterConfig.setLocationConstraint(spec, materializeRead);
- String checkpointPath = "/tmp/ckpoint/" + jobId + "/message";
+ String checkpointPath = "/tmp/ckpoint/" + jobId + "/message/" + lastSuccessfulIteration;
PregelixJob tmpJob = createCloneJob("State checkpointing for job " + jobId, pregelixJob);
tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
FileOutputFormat.setOutputPath(tmpJob, new Path(checkpointPath));
@@ -657,7 +660,7 @@
@SuppressWarnings({ "unchecked", "rawtypes" })
protected JobSpecification[] generateStateCheckpointLoading(int lastCheckpointedIteration, PregelixJob job)
throws HyracksException {
- String checkpointPath = "/tmp/ckpoint/" + jobId + "/message";
+ String checkpointPath = "/tmp/ckpoint/" + jobId + "/message/" + lastCheckpointedIteration;
PregelixJob tmpJob = createCloneJob("State checkpoint loading for job " + jobId, job);
tmpJob.setVertexInputFormatClass(InternalVertexInputFormat.class);
try {
@@ -696,6 +699,12 @@
recordDescriptor);
ClusterConfig.setLocationConstraint(spec, materialize);
+ /** construct runtime hook */
+ RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
+ new RecoveryRuntimeHookFactory(jobId, lastCheckpointedIteration + 1, new ConfigurationFactory(
+ pregelixJob.getConfiguration())));
+ ClusterConfig.setLocationConstraint(spec, postSuperStep);
+
/** construct empty sink operator */
EmptySinkOperatorDescriptor emptySink = new EmptySinkOperatorDescriptor(spec);
ClusterConfig.setLocationConstraint(spec, emptySink);
@@ -706,7 +715,8 @@
ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0,
materialize, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, emptySink, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
spec.setFrameSize(frameSize);
return new JobSpecification[] { spec };
}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
index 58384b2..41887c0 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
@@ -529,7 +529,7 @@
tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
/** generate secondary index checkpoint */
- String checkpointPath = "/tmp/ckpoint/" + jobId + "/secondary";
+ String checkpointPath = "/tmp/ckpoint/" + jobId + "/secondary/" + lastSuccessfulIteration;
FileOutputFormat.setOutputPath(tmpJob, new Path(checkpointPath));
tmpJob.setOutputKeyClass(BspUtils.getVertexIndexClass(tmpJob.getConfiguration()));
tmpJob.setOutputValueClass(MsgList.class);
@@ -569,7 +569,7 @@
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(job.getConfiguration());
JobSpecification spec = new JobSpecification();
- String checkpointPath = "/tmp/ckpoint/" + jobId + "/secondary";
+ String checkpointPath = "/tmp/ckpoint/" + jobId + "/secondary/" + lastSuccessfulIteration;;
PregelixJob tmpJob = createCloneJob("State checkpoint loading for job " + jobId, job);
tmpJob.setVertexInputFormatClass(InternalVertexInputFormat.class);
try {
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
index e8a6b5c..e1795de 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
@@ -61,6 +61,8 @@
ccConfig.defaultMaxJobAttempts = 0;
ccConfig.jobHistorySize = 1;
ccConfig.profileDumpPeriod = -1;
+ //ccConfig.heartbeatPeriod = 5000;
+ //ccConfig.maxHeartbeatLapsePeriods = 1;
// cluster controller
cc = new ClusterControllerService(ccConfig);
@@ -74,7 +76,7 @@
ncConfig1.dataIPAddress = "127.0.0.1";
ncConfig1.datasetIPAddress = "127.0.0.1";
ncConfig1.nodeId = NC1_ID;
- ncConfig1.ioDevices="dev1,dev2";
+ ncConfig1.ioDevices = "dev1,dev2";
ncConfig1.appNCMainClass = NCApplicationEntryPoint.class.getName();
nc1 = new NodeControllerService(ncConfig1);
nc1.start();
@@ -87,7 +89,7 @@
ncConfig2.datasetIPAddress = "127.0.0.1";
ncConfig2.nodeId = NC2_ID;
ncConfig2.appNCMainClass = NCApplicationEntryPoint.class.getName();
- ncConfig2.ioDevices="dev3,dev4";
+ ncConfig2.ioDevices = "dev3,dev4";
nc2 = new NodeControllerService(ncConfig2);
nc2.start();
@@ -96,6 +98,14 @@
ClusterConfig.loadClusterConfig(CC_HOST, TEST_HYRACKS_CC_CLIENT_PORT);
}
+ public static void showDownNC1() throws Exception {
+ nc1.stop();
+ }
+
+ public static void showDownNC2() throws Exception {
+ nc2.stop();
+ }
+
public static void deinit() throws Exception {
nc2.stop();
nc1.stop();
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/pregelix b/pregelix/pregelix-core/src/main/resources/scripts/pregelix
deleted file mode 100644
index 7232ccc..0000000
--- a/pregelix/pregelix-core/src/main/resources/scripts/pregelix
+++ /dev/null
@@ -1,111 +0,0 @@
-#!/bin/sh
-#
-#------------------------------------------------------------------------
-# Copyright 2009-2013 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.
-# ------------------------------------------------------------------------
-#
-
-
-# resolve links - $0 may be a softlink
-PRG="$0"
-
-while [ -h "$PRG" ]; do
- ls=`ls -ld "$PRG"`
- link=`expr "$ls" : '.*-> \(.*\)$'`
- if expr "$link" : '/.*' > /dev/null; then
- PRG="$link"
- else
- PRG=`dirname "$PRG"`/"$link"
- fi
-done
-
-PRGDIR=`dirname "$PRG"`
-BASEDIR=`cd "$PRGDIR/.." >/dev/null; pwd`
-
-
-
-# OS specific support. $var _must_ be set to either true or false.
-cygwin=false;
-darwin=false;
-case "`uname`" in
- CYGWIN*) cygwin=true ;;
- Darwin*) darwin=true
- if [ -z "$JAVA_VERSION" ] ; then
- JAVA_VERSION="CurrentJDK"
- else
- echo "Using Java version: $JAVA_VERSION"
- fi
- if [ -z "$JAVA_HOME" ] ; then
- JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
- fi
- ;;
-esac
-
-if [ -z "$JAVA_HOME" ] ; then
- if [ -r /etc/gentoo-release ] ; then
- JAVA_HOME=`java-config --jre-home`
- fi
-fi
-
-# For Cygwin, ensure paths are in UNIX format before anything is touched
-if $cygwin ; then
- [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
- [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
-fi
-
-# If a specific java binary isn't specified search for the standard 'java' binary
-if [ -z "$JAVACMD" ] ; then
- if [ -n "$JAVA_HOME" ] ; then
- if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
- # IBM's JDK on AIX uses strange locations for the executables
- JAVACMD="$JAVA_HOME/jre/sh/java"
- else
- JAVACMD="$JAVA_HOME/bin/java"
- fi
- else
- JAVACMD=`which java`
- fi
-fi
-
-if [ ! -x "$JAVACMD" ] ; then
- echo "Error: JAVA_HOME is not defined correctly." 1>&2
- echo " We cannot execute $JAVACMD" 1>&2
- exit 1
-fi
-
-if [ -z "$REPO" ]
-then
- REPO="$BASEDIR"/lib
-fi
-
-CLASSPATH=$CLASSPATH_PREFIX:"$HADOOP_HOME"/conf:/etc/hadoop/conf:"$BASEDIR"/etc:$1
-
-# For Cygwin, switch paths to Windows format before running java
-if $cygwin; then
- [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --windows "$CLASSPATH"`
- [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --path --windows "$JAVA_HOME"`
- [ -n "$HOME" ] && HOME=`cygpath --path --windows "$HOME"`
- [ -n "$BASEDIR" ] && BASEDIR=`cygpath --path --windows "$BASEDIR"`
- [ -n "$REPO" ] && REPO=`cygpath --path --windows "$REPO"`
-fi
-
-exec "$JAVACMD" $JAVA_OPTS \
- -classpath "$CLASSPATH" \
- -Dapp.name="pregelix" \
- -Dapp.pid="$$" \
- -Dapp.repo="$REPO" \
- -Dapp.home="$BASEDIR" \
- -Dbasedir="$BASEDIR" \
- org.apache.hadoop.util.RunJar \
- "$@"
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/pregelix.bat b/pregelix/pregelix-core/src/main/resources/scripts/pregelix.bat
deleted file mode 100644
index fe53029..0000000
--- a/pregelix/pregelix-core/src/main/resources/scripts/pregelix.bat
+++ /dev/null
@@ -1,124 +0,0 @@
-@rem /*
-@rem Copyright 2009-2013 by The Regents of the University of California
-@rem Licensed under the Apache License, Version 2.0 (the "License");
-@rem you may not use this file except in compliance with the License.
-@rem you may obtain a copy of the License from
-@rem
-@rem http://www.apache.org/licenses/LICENSE-2.0
-@rem
-@rem Unless required by applicable law or agreed to in writing, software
-@rem distributed under the License is distributed on an "AS IS" BASIS,
-@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-@rem See the License for the specific language governing permissions and
-@rem limitations under the License.
-@rem */
-@REM ----------------------------------------------------------------------------
-@REM Copyright 2001-2006 The Apache Software Foundation.
-@REM
-@REM Licensed under the Apache License, Version 2.0 (the "License");
-@REM you may not use this file except in compliance with the License.
-@REM You may obtain a copy of the License at
-@REM
-@REM http://www.apache.org/licenses/LICENSE-2.0
-@REM
-@REM Unless required by applicable law or agreed to in writing, software
-@REM distributed under the License is distributed on an "AS IS" BASIS,
-@REM WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-@REM See the License for the specific language governing permissions and
-@REM limitations under the License.
-@REM ----------------------------------------------------------------------------
-@REM
-@REM Copyright (c) 2001-2006 The Apache Software Foundation. All rights
-@REM reserved.
-
-@echo off
-
-set ERROR_CODE=0
-
-:init
-@REM Decide how to startup depending on the version of windows
-
-@REM -- Win98ME
-if NOT "%OS%"=="Windows_NT" goto Win9xArg
-
-@REM set local scope for the variables with windows NT shell
-if "%OS%"=="Windows_NT" @setlocal
-
-@REM -- 4NT shell
-if "%eval[2+2]" == "4" goto 4NTArgs
-
-@REM -- Regular WinNT shell
-set CMD_LINE_ARGS=%*
-goto WinNTGetScriptDir
-
-@REM The 4NT Shell from jp software
-:4NTArgs
-set CMD_LINE_ARGS=%$
-goto WinNTGetScriptDir
-
-:Win9xArg
-@REM Slurp the command line arguments. This loop allows for an unlimited number
-@REM of arguments (up to the command line limit, anyway).
-set CMD_LINE_ARGS=
-:Win9xApp
-if %1a==a goto Win9xGetScriptDir
-set CMD_LINE_ARGS=%CMD_LINE_ARGS% %1
-shift
-goto Win9xApp
-
-:Win9xGetScriptDir
-set SAVEDIR=%CD%
-%0\
-cd %0\..\..
-set BASEDIR=%CD%
-cd %SAVEDIR%
-set SAVE_DIR=
-goto repoSetup
-
-:WinNTGetScriptDir
-set BASEDIR=%~dp0\..
-
-:repoSetup
-
-
-if "%JAVACMD%"=="" set JAVACMD=java
-
-if "%REPO%"=="" set REPO=%BASEDIR%\lib
-
-cp $BASEDIR"\..\a-hadoop-patch.jar "$REPO"\
-
-set CLASSPATH="%BASEDIR%"\etc;"%REPO%"\a-hadoop-patch.jar;"%REPO%"\pregelix-api-0.0.1-SNAPSHOT.jar;"%REPO%"\hyracks-dataflow-common-0.2.2-SNAPSHOT.jar;"%REPO%"\hyracks-api-0.2.2-SNAPSHOT.jar;"%REPO%"\json-20090211.jar;"%REPO%"\httpclient-4.1-alpha2.jar;"%REPO%"\httpcore-4.1-beta1.jar;"%REPO%"\commons-logging-1.1.1.jar;"%REPO%"\commons-codec-1.3.jar;"%REPO%"\args4j-2.0.12.jar;"%REPO%"\hyracks-ipc-0.2.2-SNAPSHOT.jar;"%REPO%"\commons-lang3-3.1.jar;"%REPO%"\hyracks-data-std-0.2.2-SNAPSHOT.jar;"%REPO%"\hadoop-core-0.20.2.jar;"%REPO%"\commons-cli-1.2.jar;"%REPO%"\xmlenc-0.52.jar;"%REPO%"\commons-httpclient-3.0.1.jar;"%REPO%"\commons-net-1.4.1.jar;"%REPO%"\oro-2.0.8.jar;"%REPO%"\jetty-6.1.14.jar;"%REPO%"\jetty-util-6.1.14.jar;"%REPO%"\servlet-api-2.5-6.1.14.jar;"%REPO%"\jasper-runtime-5.5.12.jar;"%REPO%"\jasper-compiler-5.5.12.jar;"%REPO%"\jsp-api-2.1-6.1.14.jar;"%REPO%"\jsp-2.1-6.1.14.jar;"%REPO%"\core-3.1.1.jar;"%REPO%"\ant-1.6.5.jar;"%REPO%"\commons-el-1.0.jar;"%REPO%"\jets3t-0.7.1.jar;"%REPO%"\kfs-0.3.jar;"%REPO%"\hsqldb-1.8.0.10.jar;"%REPO%"\pregelix-dataflow-std-0.0.1-SNAPSHOT.jar;"%REPO%"\pregelix-dataflow-std-base-0.0.1-SNAPSHOT.jar;"%REPO%"\hyracks-dataflow-std-0.2.2-SNAPSHOT.jar;"%REPO%"\hyracks-dataflow-hadoop-0.2.2-SNAPSHOT.jar;"%REPO%"\dcache-client-0.0.1.jar;"%REPO%"\jetty-client-8.0.0.M0.jar;"%REPO%"\jetty-http-8.0.0.RC0.jar;"%REPO%"\jetty-io-8.0.0.RC0.jar;"%REPO%"\jetty-util-8.0.0.RC0.jar;"%REPO%"\hyracks-storage-am-common-0.2.2-SNAPSHOT.jar;"%REPO%"\hyracks-storage-common-0.2.2-SNAPSHOT.jar;"%REPO%"\hyracks-storage-am-btree-0.2.2-SNAPSHOT.jar;"%REPO%"\btreehelper-0.2.2-SNAPSHOT.jar;"%REPO%"\hyracks-control-cc-0.2.2-SNAPSHOT.jar;"%REPO%"\hyracks-control-common-0.2.2-SNAPSHOT.jar;"%REPO%"\commons-io-1.3.1.jar;"%REPO%"\jetty-server-8.0.0.RC0.jar;"%REPO%"\servlet-api-3.0.20100224.jar;"%REPO%"\jetty-continuation-8.0.0.RC0.jar;"%REPO%"\jetty-webapp-8.0.0.RC0.jar;"%REPO%"\jetty-xml-8.0.0.RC0.jar;"%REPO%"\jetty-servlet-8.0.0.RC0.jar;"%REPO%"\jetty-security-8.0.0.RC0.jar;"%REPO%"\wicket-core-1.5.2.jar;"%REPO%"\wicket-util-1.5.2.jar;"%REPO%"\slf4j-api-1.6.1.jar;"%REPO%"\wicket-request-1.5.2.jar;"%REPO%"\slf4j-jcl-1.6.3.jar;"%REPO%"\hyracks-control-nc-0.2.2-SNAPSHOT.jar;"%REPO%"\hyracks-net-0.2.2-SNAPSHOT.jar;"%REPO%"\hyracks-hadoop-compat-0.2.2-SNAPSHOT.jar;"%REPO%"\pregelix-dataflow-0.0.1-SNAPSHOT.jar;"%REPO%"\pregelix-runtime-0.0.1-SNAPSHOT.jar;"%REPO%"\hadoop-test-0.20.2.jar;"%REPO%"\ftplet-api-1.0.0.jar;"%REPO%"\mina-core-2.0.0-M5.jar;"%REPO%"\ftpserver-core-1.0.0.jar;"%REPO%"\ftpserver-deprecated-1.0.0-M2.jar;"%REPO%"\javax.servlet-api-3.0.1.jar;"%REPO%"\pregelix-core-0.0.1-SNAPSHOT.jar
-goto endInit
-
-@REM Reaching here means variables are defined and arguments have been captured
-:endInit
-
-%JAVACMD% %JAVA_OPTS% -classpath %CLASSPATH_PREFIX%;%CLASSPATH% -Dapp.name="pregelix" -Dapp.repo="%REPO%" -Dapp.home="%BASEDIR%" -Dbasedir="%BASEDIR%" org.apache.hadoop.util.RunJar %CMD_LINE_ARGS%
-if ERRORLEVEL 1 goto error
-goto end
-
-:error
-if "%OS%"=="Windows_NT" @endlocal
-set ERROR_CODE=%ERRORLEVEL%
-
-:end
-@REM set local scope for the variables with windows NT shell
-if "%OS%"=="Windows_NT" goto endNT
-
-@REM For old DOS remove the set variables from ENV - we assume they were not set
-@REM before we started - at least we don't leave any baggage around
-set CMD_LINE_ARGS=
-goto postExec
-
-:endNT
-@REM If error code is set to 1 then the endlocal was done already in :error.
-if %ERROR_CODE% EQU 0 @endlocal
-
-
-:postExec
-
-if "%FORCE_EXIT_ON_ERROR%" == "on" (
- if %ERROR_CODE% NEQ 0 exit %ERROR_CODE%
-)
-
-exit /B %ERROR_CODE%
\ No newline at end of file
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/pregelixcc b/pregelix/pregelix-core/src/main/resources/scripts/pregelixcc
deleted file mode 100755
index c1ee3f2..0000000
--- a/pregelix/pregelix-core/src/main/resources/scripts/pregelixcc
+++ /dev/null
@@ -1,114 +0,0 @@
-#!/bin/sh
-#
-#------------------------------------------------------------------------
-# Copyright 2009-2013 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.
-# ------------------------------------------------------------------------
-#
-
-# resolve links - $0 may be a softlink
-PRG="$0"
-
-while [ -h "$PRG" ]; do
- ls=`ls -ld "$PRG"`
- link=`expr "$ls" : '.*-> \(.*\)$'`
- if expr "$link" : '/.*' > /dev/null; then
- PRG="$link"
- else
- PRG=`dirname "$PRG"`/"$link"
- fi
-done
-
-PRGDIR=`dirname "$PRG"`
-BASEDIR=`cd "$PRGDIR/.." >/dev/null; pwd`
-
-
-
-# OS specific support. $var _must_ be set to either true or false.
-cygwin=false;
-darwin=false;
-case "`uname`" in
- CYGWIN*) cygwin=true ;;
- Darwin*) darwin=true
- if [ -z "$JAVA_VERSION" ] ; then
- JAVA_VERSION="CurrentJDK"
- else
- echo "Using Java version: $JAVA_VERSION"
- fi
- if [ -z "$JAVA_HOME" ] ; then
- JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
- fi
- ;;
-esac
-
-if [ -z "$JAVA_HOME" ] ; then
- if [ -r /etc/gentoo-release ] ; then
- JAVA_HOME=`java-config --jre-home`
- fi
-fi
-
-# For Cygwin, ensure paths are in UNIX format before anything is touched
-if $cygwin ; then
- [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
- [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
-fi
-
-# If a specific java binary isn't specified search for the standard 'java' binary
-if [ -z "$JAVACMD" ] ; then
- if [ -n "$JAVA_HOME" ] ; then
- if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
- # IBM's JDK on AIX uses strange locations for the executables
- JAVACMD="$JAVA_HOME/jre/sh/java"
- else
- JAVACMD="$JAVA_HOME/bin/java"
- fi
- else
- JAVACMD=`which java`
- fi
-fi
-
-if [ ! -x "$JAVACMD" ] ; then
- echo "Error: JAVA_HOME is not defined correctly." 1>&2
- echo " We cannot execute $JAVACMD" 1>&2
- exit 1
-fi
-
-if [ -z "$REPO" ]
-then
- REPO="$BASEDIR"/lib
-fi
-
-CLASSPATH=$CLASSPATH_PREFIX:"$HADOOP_HOME"/conf:/etc/hadoop/conf:"$BASEDIR"/etc:$1
-
-for f in ${REPO}/*.jar; do
- CLASSPATH=${CLASSPATH}:$f;
-done
-
-# For Cygwin, switch paths to Windows format before running java
-if $cygwin; then
- [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --windows "$CLASSPATH"`
- [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --path --windows "$JAVA_HOME"`
- [ -n "$HOME" ] && HOME=`cygpath --path --windows "$HOME"`
- [ -n "$BASEDIR" ] && BASEDIR=`cygpath --path --windows "$BASEDIR"`
- [ -n "$REPO" ] && REPO=`cygpath --path --windows "$REPO"`
-fi
-
-exec "$JAVACMD" $JAVA_OPTS \
- -classpath "$CLASSPATH" \
- -Dapp.name="pregelixcc" \
- -Dapp.pid="$$" \
- -Dapp.repo="$REPO" \
- -Dapp.home="$BASEDIR" \
- -Dbasedir="$BASEDIR" \
- edu.uci.ics.hyracks.control.cc.CCDriver \
- "$@"
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/pregelixnc b/pregelix/pregelix-core/src/main/resources/scripts/pregelixnc
deleted file mode 100755
index c01b4b4..0000000
--- a/pregelix/pregelix-core/src/main/resources/scripts/pregelixnc
+++ /dev/null
@@ -1,115 +0,0 @@
-#!/bin/sh
-#
-#------------------------------------------------------------------------
-# Copyright 2009-2013 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.
-# ------------------------------------------------------------------------
-#
-
-# resolve links - $0 may be a softlink
-PRG="$0"
-
-while [ -h "$PRG" ]; do
- ls=`ls -ld "$PRG"`
- link=`expr "$ls" : '.*-> \(.*\)$'`
- if expr "$link" : '/.*' > /dev/null; then
- PRG="$link"
- else
- PRG=`dirname "$PRG"`/"$link"
- fi
-done
-
-PRGDIR=`dirname "$PRG"`
-BASEDIR=`cd "$PRGDIR/.." >/dev/null; pwd`
-
-
-
-# OS specific support. $var _must_ be set to either true or false.
-cygwin=false;
-darwin=false;
-case "`uname`" in
- CYGWIN*) cygwin=true ;;
- Darwin*) darwin=true
- if [ -z "$JAVA_VERSION" ] ; then
- JAVA_VERSION="CurrentJDK"
- else
- echo "Using Java version: $JAVA_VERSION"
- fi
- if [ -z "$JAVA_HOME" ] ; then
- JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
- fi
- ;;
-esac
-
-if [ -z "$JAVA_HOME" ] ; then
- if [ -r /etc/gentoo-release ] ; then
- JAVA_HOME=`java-config --jre-home`
- fi
-fi
-
-# For Cygwin, ensure paths are in UNIX format before anything is touched
-if $cygwin ; then
- [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
- [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
-fi
-
-# If a specific java binary isn't specified search for the standard 'java' binary
-if [ -z "$JAVACMD" ] ; then
- if [ -n "$JAVA_HOME" ] ; then
- if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
- # IBM's JDK on AIX uses strange locations for the executables
- JAVACMD="$JAVA_HOME/jre/sh/java"
- else
- JAVACMD="$JAVA_HOME/bin/java"
- fi
- else
- JAVACMD=`which java`
- fi
-fi
-
-if [ ! -x "$JAVACMD" ] ; then
- echo "Error: JAVA_HOME is not defined correctly." 1>&2
- echo " We cannot execute $JAVACMD" 1>&2
- exit 1
-fi
-
-if [ -z "$REPO" ]
-then
- REPO="$BASEDIR"/lib
-fi
-
-CLASSPATH=$CLASSPATH_PREFIX:"$HADOOP_HOME"/conf:/etc/hadoop/conf:"$BASEDIR"/etc:$1
-
-for f in ${REPO}/*.jar; do
- CLASSPATH=${CLASSPATH}:$f;
-done
-
-
-# For Cygwin, switch paths to Windows format before running java
-if $cygwin; then
- [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --windows "$CLASSPATH"`
- [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --path --windows "$JAVA_HOME"`
- [ -n "$HOME" ] && HOME=`cygpath --path --windows "$HOME"`2
- [ -n "$BASEDIR" ] && BASEDIR=`cygpath --path --windows "$BASEDIR"`
- [ -n "$REPO" ] && REPO=`cygpath --path --windows "$REPO"`
-fi
-
-exec "$JAVACMD" $JAVA_OPTS \
- -classpath "$CLASSPATH" \
- -Dapp.name="pregelixnc" \
- -Dapp.pid="$$" \
- -Dapp.repo="$REPO" \
- -Dapp.home="$BASEDIR" \
- -Dbasedir="$BASEDIR" \
- edu.uci.ics.hyracks.control.nc.NCDriver \
- -app-nc-main-class edu.uci.ics.pregelix.runtime.bootstrap.NCApplicationEntryPoint "$@"
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java
index 3fed609..c0be9dd 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java
@@ -17,9 +17,13 @@
import java.io.DataInput;
import java.io.DataInputStream;
+import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Writable;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -32,6 +36,7 @@
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
+import edu.uci.ics.pregelix.api.graph.Vertex;
import edu.uci.ics.pregelix.api.util.BspUtils;
import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
@@ -93,10 +98,28 @@
}
+ @SuppressWarnings("unchecked")
@Override
public void close() throws HyracksDataException {
- Writable finalAggregateValue = aggregator.finishFinal();
- IterationUtils.writeGlobalAggregateValue(conf, jobId, finalAggregateValue);
+ try {
+ // iterate over hdfs spilled aggregates
+ FileSystem dfs = FileSystem.get(conf);
+ String spillingDir = BspUtils.getGlobalAggregateSpillingDirName(conf, Vertex.getSuperstep());
+ FileStatus[] files = dfs.listStatus(new Path(spillingDir));
+ if (files != null) {
+ // goes into this branch only when there are spilled files
+ for (int i = 0; i < files.length; i++) {
+ FileStatus file = files[i];
+ DataInput dis = dfs.open(file.getPath());
+ partialAggregateValue.readFields(dis);
+ aggregator.step(partialAggregateValue);
+ }
+ }
+ Writable finalAggregateValue = aggregator.finishFinal();
+ IterationUtils.writeGlobalAggregateValue(conf, jobId, finalAggregateValue);
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
}
};
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
index b74a5de..9a21680 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
@@ -168,7 +168,8 @@
FileStatus[] results = dfs.listStatus(tempDir, new PathFilter() {
@Override
public boolean accept(Path dir) {
- return dir.getName().indexOf(context.getTaskAttemptID().toString()) >= 0;
+ return dir.getName().indexOf(context.getTaskAttemptID().toString()) >= 0
+ && dir.getName().indexOf(".crc") < 0;
}
});
return results;
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 496d066..bfe89ab 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
@@ -54,14 +54,14 @@
private final ILocalResourceRepository localResourceRepository;
private final ResourceIdFactory resourceIdFactory;
private final IBufferCache bufferCache;
- private final IVirtualBufferCache vBufferCache;
+ private final List<IVirtualBufferCache> vbcs;
private final IFileMapManager fileMapManager;
private final IOManager ioManager;
private final Map<Long, List<FileReference>> iterationToFiles = new ConcurrentHashMap<Long, List<FileReference>>();
private final Map<StateKey, IStateObject> appStateMap = new ConcurrentHashMap<StateKey, IStateObject>();
private final Map<String, Long> jobIdToSuperStep = new ConcurrentHashMap<String, Long>();
private final Map<String, Boolean> jobIdToMove = new ConcurrentHashMap<String, Boolean>();
-
+
private final ThreadFactory threadFactory = new ThreadFactory() {
public Thread newThread(Runnable r) {
return new Thread(r);
@@ -81,8 +81,10 @@
new PreDelayPageCleanerPolicy(Long.MAX_VALUE), fileMapManager, pageSize, numPages, 1000000,
threadFactory);
int numPagesInMemComponents = numPages / 8;
- vBufferCache = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(), pageSize,
- numPagesInMemComponents));
+ vbcs = new ArrayList<IVirtualBufferCache>();
+ IVirtualBufferCache vBufferCache = new MultitenantVirtualBufferCache(new VirtualBufferCache(
+ new HeapBufferAllocator(), pageSize, numPagesInMemComponents));
+ vbcs.add(vBufferCache);
ioManager = (IOManager) appCtx.getRootContext().getIOManager();
lcManager = new NoBudgetIndexLifecycleManager();
localResourceRepository = new TransientLocalResourceRepository();
@@ -129,8 +131,8 @@
return bufferCache;
}
- public IVirtualBufferCache getVirtualBufferCache() {
- return vBufferCache;
+ public List<IVirtualBufferCache> getVirtualBufferCaches() {
+ return vbcs;
}
public IFileMapProvider getFileMapManager() {
@@ -145,7 +147,7 @@
return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
}
- public synchronized void setVertexProperties(String jobId, long numVertices, long numEdges) {
+ public synchronized void setVertexProperties(String jobId, long numVertices, long numEdges, int currentIteration) {
Boolean toMove = jobIdToMove.get(jobId);
if (toMove == null || toMove == true) {
if (jobIdToSuperStep.get(jobId) == null) {
@@ -159,7 +161,11 @@
fileRef.delete();
}
- Vertex.setSuperstep(++superStep);
+ if (currentIteration > 0) {
+ Vertex.setSuperstep(currentIteration);
+ } else {
+ Vertex.setSuperstep(++superStep);
+ }
Vertex.setNumVertices(numVertices);
Vertex.setNumEdges(numEdges);
jobIdToSuperStep.put(jobId, superStep);
@@ -169,8 +175,8 @@
System.gc();
}
- public synchronized void endSuperStep(String giraphJobId) {
- jobIdToMove.put(giraphJobId, true);
+ public synchronized void endSuperStep(String pregelixJobId) {
+ jobIdToMove.put(pregelixJobId, true);
LOGGER.info("end iteration " + Vertex.getSuperstep());
}
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
index 603a464..1cf81ac 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
@@ -75,11 +75,11 @@
context.endSuperStep(giraphJobId);
}
- public static void setProperties(String giraphJobId, IHyracksTaskContext ctx, Configuration conf) {
+ public static void setProperties(String jobId, IHyracksTaskContext ctx, Configuration conf, int currentIteration) {
INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
- context.setVertexProperties(giraphJobId, conf.getLong(PregelixJob.NUM_VERTICE, -1),
- conf.getLong(PregelixJob.NUM_EDGES, -1));
+ context.setVertexProperties(jobId, conf.getLong(PregelixJob.NUM_VERTICE, -1),
+ conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration);
}
public static void writeTerminationState(Configuration conf, String jobId, boolean terminate)
diff --git a/pregelix/pregelix-example/pom.xml b/pregelix/pregelix-example/pom.xml
index 1066e3b..9994c0e 100644
--- a/pregelix/pregelix-example/pom.xml
+++ b/pregelix/pregelix-example/pom.xml
@@ -94,7 +94,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-clean-plugin</artifactId>
- <version>2.4.1</version>
+ <version>2.5</version>
<configuration>
<filesets>
<fileset>
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
new file mode 100644
index 0000000..dac087f
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2013 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.pregelix.example;
+
+import java.io.File;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.Test;
+
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+import edu.uci.ics.pregelix.example.util.TestUtils;
+
+/**
+ * @author yingyib
+ */
+public class FailureRecoveryTest {
+ private static String INPUTPATH = "data/webmap";
+ private static String OUTPUTPAH = "actual/result";
+ private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal";
+
+ @Test
+ public void test() throws Exception {
+ TestCluster testCluster = new TestCluster();
+
+ try {
+ PregelixJob job = new PregelixJob(PageRankVertex.class.getName());
+ job.setVertexClass(PageRankVertex.class);
+ job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+ job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+ job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+ job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+ FileInputFormat.setInputPaths(job, INPUTPATH);
+ FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+ job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+ job.setCheckpointHook(ConservativeCheckpointHook.class);
+
+ testCluster.setUp();
+ Driver driver = new Driver(PageRankVertex.class);
+ // Thread thread = new Thread(new Runnable() {
+ //
+ // @Override
+ // public void run() {
+ // try {
+ // synchronized (this) {
+ // this.wait(10000);
+ // PregelixHyracksIntegrationUtil.showDownNC1();
+ // }
+ // } catch (Exception e) {
+ // throw new IllegalStateException(e);
+ // }
+ // }
+ //
+ // });
+ //thread.start();
+ driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+
+ TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
+ } catch (Exception e) {
+ throw e;
+ } finally {
+ testCluster.tearDown();
+ }
+ }
+
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/FailureVertexTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureVertexTest.java
similarity index 96%
rename from pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/FailureVertexTest.java
rename to pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureVertexTest.java
index 5a2636a..a2d32c0 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/FailureVertexTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureVertexTest.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.pregelix.example.test;
+package edu.uci.ics.pregelix.example;
import junit.framework.Assert;
@@ -29,6 +29,7 @@
import edu.uci.ics.pregelix.example.FailureVertex;
import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
/**
* This test case tests the error message propagation.
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/JobConcatenationTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java
similarity index 91%
rename from pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/JobConcatenationTest.java
rename to pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java
index d2995f1..5a485ba 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/JobConcatenationTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.pregelix.example.test;
+package edu.uci.ics.pregelix.example;
import java.io.File;
import java.util.ArrayList;
@@ -24,13 +24,12 @@
import org.junit.Test;
import edu.uci.ics.pregelix.api.job.PregelixJob;
-import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
import edu.uci.ics.pregelix.core.driver.Driver;
import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
-import edu.uci.ics.pregelix.example.PageRankVertex;
import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
import edu.uci.ics.pregelix.example.util.TestUtils;
/**
@@ -56,7 +55,7 @@
job1.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
FileInputFormat.setInputPaths(job1, INPUTPATH);
job1.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
- job1.setCheckpointHook(ConservativeCheckpointHook.class);
+ //job1.setCheckpointHook(ConservativeCheckpointHook.class);
PregelixJob job2 = new PregelixJob(PageRankVertex.class.getName());
job2.setVertexClass(PageRankVertex.class);
@@ -66,7 +65,7 @@
job2.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
FileOutputFormat.setOutputPath(job2, new Path(OUTPUTPAH));
job2.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
- job2.setCheckpointHook(ConservativeCheckpointHook.class);
+ //job2.setCheckpointHook(ConservativeCheckpointHook.class);
jobs.add(job1);
jobs.add(job2);
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/OverflowAggregatorTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/OverflowAggregatorTest.java
new file mode 100644
index 0000000..474d0a6
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/OverflowAggregatorTest.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2013 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.pregelix.example;
+
+import java.io.File;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.Test;
+
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
+import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
+import edu.uci.ics.pregelix.example.aggregator.OverflowAggregator;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+import edu.uci.ics.pregelix.example.util.TestUtils;
+
+/**
+ * @author yingyib
+ */
+public class OverflowAggregatorTest {
+
+ private static String INPUTPATH = "data/webmap";
+ private static String OUTPUTPAH = "actual/result";
+ private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal";
+
+ @Test
+ public void test() throws Exception {
+ TestCluster testCluster = new TestCluster();
+
+ try {
+ PregelixJob job = new PregelixJob(PageRankVertex.class.getName());
+ job.setVertexClass(PageRankVertex.class);
+ job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+ job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+ job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+ job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+ FileInputFormat.setInputPaths(job, INPUTPATH);
+ FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+ job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+ job.setGlobalAggregatorClass(OverflowAggregator.class);
+
+ testCluster.setUp();
+ Driver driver = new Driver(PageRankVertex.class);
+ driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+
+ TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
+ Text text = (Text) IterationUtils.readGlobalAggregateValue(job.getConfiguration(),
+ BspUtils.getJobId(job.getConfiguration()));
+ Assert.assertEquals(text.getLength(), 20 * 32767);
+ } catch (Exception e) {
+ throw e;
+ } finally {
+ testCluster.tearDown();
+ }
+ }
+
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/aggregator/OverflowAggregator.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/aggregator/OverflowAggregator.java
new file mode 100644
index 0000000..34b8b51
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/aggregator/OverflowAggregator.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2009-2013 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.pregelix.example.aggregator;
+
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.example.io.DoubleWritable;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * Test the case where the global aggregate's state is bloated
+ *
+ * @author yingyib
+ */
+public class OverflowAggregator extends
+ GlobalAggregator<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable, Text, Text> {
+
+ private int textLength = 0;
+ private int inc = 32767;
+
+ @Override
+ public void init() {
+ textLength = 0;
+ }
+
+ @Override
+ public void step(Vertex<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> v)
+ throws HyracksDataException {
+ textLength += inc;
+ }
+
+ @Override
+ public void step(Text partialResult) {
+ textLength += partialResult.getLength();
+ }
+
+ @Override
+ public Text finishPartial() {
+ byte[] partialResult = new byte[textLength];
+ for (int i = 0; i < partialResult.length; i++) {
+ partialResult[i] = 'a';
+ }
+ Text text = new Text();
+ text.set(partialResult);
+ return text;
+ }
+
+ @Override
+ public Text finishFinal() {
+ byte[] result = new byte[textLength];
+ for (int i = 0; i < result.length; i++) {
+ result[i] = 'a';
+ }
+ Text text = new Text();
+ text.set(result);
+ return text;
+ }
+
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/TestCluster.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java
similarity index 98%
rename from pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/TestCluster.java
rename to pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java
index d0cf654..40ea690 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/TestCluster.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.pregelix.example.test;
+package edu.uci.ics.pregelix.example.util;
import java.io.BufferedReader;
import java.io.DataOutputStream;
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/VirtualBufferCacheProvider.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/VirtualBufferCacheProvider.java
index ec51047..f15b1c2 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/VirtualBufferCacheProvider.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/VirtualBufferCacheProvider.java
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.pregelix.runtime.bootstrap;
+import java.util.List;
+
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;
@@ -33,7 +35,7 @@
}
@Override
- public synchronized IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx) {
- return RuntimeContext.get(ctx).getVirtualBufferCache();
+ public synchronized List<IVirtualBufferCache> getVirtualBufferCaches(IHyracksTaskContext ctx) {
+ return RuntimeContext.get(ctx).getVirtualBufferCaches();
}
}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
index ab564fa..f3a0bb4 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
@@ -237,8 +237,12 @@
Writable agg = aggregator.finishPartial();
agg.write(tbGlobalAggregate.getDataOutput());
tbGlobalAggregate.addFieldEndOffset();
- appenderGlobalAggregate.append(tbGlobalAggregate.getFieldEndOffsets(),
- tbGlobalAggregate.getByteArray(), 0, tbGlobalAggregate.getSize());
+ if (!appenderGlobalAggregate.append(tbGlobalAggregate.getFieldEndOffsets(),
+ tbGlobalAggregate.getByteArray(), 0, tbGlobalAggregate.getSize())) {
+ // aggregate state exceed the page size, write to HDFS
+ FrameTupleUtils.flushTupleToHDFS(tbGlobalAggregate, conf, Vertex.getSuperstep());
+ appenderGlobalAggregate.reset(bufferGlobalAggregate, true);
+ }
FrameTupleUtils.flushTuplesFinal(appenderGlobalAggregate, writerGlobalAggregate);
} catch (IOException e) {
throw new HyracksDataException(e);
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
index b4e1dd8..ca8ec01 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
@@ -179,7 +179,7 @@
vertex.setOutputWriters(writers);
vertex.setOutputAppenders(appenders);
vertex.setOutputTupleBuilders(tbs);
-
+
if (vertex.isHalted()) {
vertex.activate();
}
@@ -230,8 +230,12 @@
Writable agg = aggregator.finishPartial();
agg.write(tbGlobalAggregate.getDataOutput());
tbGlobalAggregate.addFieldEndOffset();
- appenderGlobalAggregate.append(tbGlobalAggregate.getFieldEndOffsets(),
- tbGlobalAggregate.getByteArray(), 0, tbGlobalAggregate.getSize());
+ if (!appenderGlobalAggregate.append(tbGlobalAggregate.getFieldEndOffsets(),
+ tbGlobalAggregate.getByteArray(), 0, tbGlobalAggregate.getSize())) {
+ // aggregate state exceed the page size, write to HDFS
+ FrameTupleUtils.flushTupleToHDFS(tbGlobalAggregate, conf, Vertex.getSuperstep());
+ appenderGlobalAggregate.reset(bufferGlobalAggregate, true);
+ }
FrameTupleUtils.flushTuplesFinal(appenderGlobalAggregate, writerGlobalAggregate);
} catch (IOException e) {
throw new HyracksDataException(e);
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/PreSuperStepRuntimeHookFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/PreSuperStepRuntimeHookFactory.java
index cd2012a..3dcdad2 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/PreSuperStepRuntimeHookFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/PreSuperStepRuntimeHookFactory.java
@@ -40,7 +40,7 @@
@Override
public void configure(IHyracksTaskContext ctx) throws HyracksDataException {
Configuration conf = confFactory.createConfiguration(ctx);
- IterationUtils.setProperties(jobId, ctx, conf);
+ IterationUtils.setProperties(jobId, ctx, conf, -1);
}
};
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java
new file mode 100644
index 0000000..35e7cd8
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2013 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.pregelix.runtime.touchpoint;
+
+import org.apache.hadoop.conf.Configuration;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHook;
+import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
+
+/**
+ * Recover the pregelix job state in a NC
+ *
+ * @author yingyib
+ */
+public class RecoveryRuntimeHookFactory implements IRuntimeHookFactory {
+ private static final long serialVersionUID = 1L;
+ private final int currentSuperStep;
+ private String jobId;
+ private IConfigurationFactory confFactory;
+
+ public RecoveryRuntimeHookFactory(String jobId, int currentSuperStep, IConfigurationFactory confFactory) {
+ this.currentSuperStep = currentSuperStep;
+ this.jobId = jobId;
+ this.confFactory = confFactory;
+ }
+
+ @Override
+ public IRuntimeHook createRuntimeHook() {
+ return new IRuntimeHook() {
+
+ @Override
+ public void configure(IHyracksTaskContext ctx) throws HyracksDataException {
+ IterationUtils.endSuperStep(jobId, ctx);
+ Configuration conf = confFactory.createConfiguration(ctx);
+ IterationUtils.setProperties(jobId, ctx, conf, currentSuperStep);
+ }
+
+ };
+ }
+
+}