Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 6539F200C33 for ; Fri, 3 Feb 2017 16:57:17 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 63EBA160B3F; Fri, 3 Feb 2017 15:57:17 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 3AEF8160B64 for ; Fri, 3 Feb 2017 16:57:15 +0100 (CET) Received: (qmail 28689 invoked by uid 500); 3 Feb 2017 15:57:14 -0000 Mailing-List: contact commits-help@asterixdb.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@asterixdb.apache.org Delivered-To: mailing list commits@asterixdb.apache.org Received: (qmail 28617 invoked by uid 99); 3 Feb 2017 15:57:14 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 03 Feb 2017 15:57:14 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 16FE8E0A74; Fri, 3 Feb 2017 15:57:14 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: amoudi@apache.org To: commits@asterixdb.apache.org Date: Fri, 03 Feb 2017 15:57:18 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [6/8] asterixdb git commit: Improve reading from and writing to Metadata pages archived-at: Fri, 03 Feb 2017 15:57:17 -0000 http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d718dc4a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java index c79205e..cc4bfb9 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java @@ -49,11 +49,12 @@ import org.apache.hyracks.storage.am.common.api.TreeIndexException; import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback; import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation; import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeRefrencingTupleWriterFactory; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; -import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext; import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy; @@ -81,7 +82,7 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { private final LSMBTreeDiskComponentFactory transactionComponentFactory; // A second disk component list that will be used when a transaction is // committed and will be seen by subsequent accessors - private final List secondDiskComponents; + private final List secondDiskComponents; // A pointer that points to the current most recent list (either // diskComponents = 0, or secondDiskComponents = 1). It starts with -1 to // indicate first time activation @@ -91,45 +92,30 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { //TODO remove BloomFilter from external dataset's secondary LSMBTree index public ExternalBTree(IIOManager ioManager, ITreeIndexFrameFactory interiorFrameFactory, - ITreeIndexFrameFactory insertLeafFrameFactory, - ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager, - TreeIndexFactory diskBTreeFactory, TreeIndexFactory bulkLoadBTreeFactory, - BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate, - IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] cmpFactories, - ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, - ILSMIOOperationCallback ioOpCallback, TreeIndexFactory transactionBTreeFactory, int version, - boolean durable) { - super(ioManager, insertLeafFrameFactory, deleteLeafFrameFactory, fileManager, - diskBTreeFactory, - bulkLoadBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider, fieldCount, - cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallback, false, durable); - this.transactionComponentFactory = new LSMBTreeDiskComponentFactory(transactionBTreeFactory, bloomFilterFactory, - null); + ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory, + ILSMIndexFileManager fileManager, TreeIndexFactory diskBTreeFactory, + TreeIndexFactory bulkLoadBTreeFactory, BloomFilterFactory bloomFilterFactory, + double bloomFilterFalsePositiveRate, IFileMapProvider diskFileMapProvider, int fieldCount, + IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, + ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, + TreeIndexFactory transactionBTreeFactory, int version, boolean durable) { + super(ioManager, insertLeafFrameFactory, deleteLeafFrameFactory, fileManager, diskBTreeFactory, + bulkLoadBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider, + fieldCount, cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallback, false, durable); + this.transactionComponentFactory = + new LSMBTreeDiskComponentFactory(transactionBTreeFactory, bloomFilterFactory, null); this.secondDiskComponents = new LinkedList<>(); this.interiorFrameFactory = interiorFrameFactory; this.version = version; } - // This method is used to create a target for a bulk modify operation. This - // component must then be either committed or deleted - private ILSMComponent createTransactionTarget() throws HyracksDataException, IndexException { - LSMComponentFileReferences componentFileRefs; - try { - componentFileRefs = fileManager.getNewTransactionFileReference(); - } catch (IOException e) { - throw new HyracksDataException("Failed to create transaction components", e); - } - return createDiskComponent(transactionComponentFactory, componentFileRefs.getInsertIndexFileReference(), - componentFileRefs.getBloomFilterFileReference(), true); - } - // The subsume merged components is overridden to account for: // Maintaining two versions of the index @Override - public void subsumeMergedComponents(ILSMComponent newComponent, List mergedComponents) + public void subsumeMergedComponents(ILSMDiskComponent newComponent, List mergedComponents) throws HyracksDataException { - List newerList; - List olderList; + List newerList; + List olderList; if (version == 0) { newerList = diskComponents; olderList = secondDiskComponents; @@ -155,7 +141,7 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { // is needed. // It only needs to return the newer list @Override - public List getImmutableComponents() { + public List getImmutableComponents() { if (version == 0) { return diskComponents; } else { @@ -204,9 +190,9 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { BTree lastBTree = ((LSMBTreeDiskComponent) mergingComponents.get(mergingComponents.size() - 1)).getBTree(); FileReference firstFile = firstBTree.getFileReference(); FileReference lastFile = lastBTree.getFileReference(); - LSMComponentFileReferences relMergeFileRefs = fileManager - .getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName()); - ILSMIndexAccessorInternal accessor = new LSMBTreeAccessor(lsmHarness, opCtx); + LSMComponentFileReferences relMergeFileRefs = + fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName()); + ILSMIndexAccessor accessor = new LSMBTreeAccessor(lsmHarness, opCtx); ioScheduler.scheduleOperation(new LSMBTreeMergeOperation(accessor, mergingComponents, cursor, relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir())); @@ -222,7 +208,7 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { // This function in an instance of this index is only used after a bulk load // is successful @Override - public void addComponent(ILSMComponent c) throws HyracksDataException { + public void addDiskComponent(ILSMDiskComponent c) throws HyracksDataException { if (version == 0) { diskComponents.add(0, c); } else if (version == 1) { @@ -232,10 +218,10 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { // This function is used when a new component is to be committed. @Override - public void commitTransactionDiskComponent(ILSMComponent newComponent) throws HyracksDataException { + public void commitTransactionDiskComponent(ILSMDiskComponent newComponent) throws HyracksDataException { // determine which list is the new one and flip the pointer - List newerList; - List olderList; + List newerList; + List olderList; if (version == 0) { newerList = diskComponents; olderList = secondDiskComponents; @@ -409,7 +395,8 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { // Not supported @Override - public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException { + public void modify(IIndexOperationContext ictx, ITupleReference tuple) + throws HyracksDataException, IndexException { throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-BTree"); } @@ -422,7 +409,7 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { // Not supported @Override - public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException { + public ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException { throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree"); } @@ -430,7 +417,7 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { @Override public void getOperationalComponents(ILSMIndexOperationContext ctx) { List operationalComponents = ctx.getComponentHolder(); - List immutableComponents; + List immutableComponents; // Identify current list in case of a merge if (version == 0) { immutableComponents = diskComponents; @@ -486,7 +473,7 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { // The bulk loader used for both initial loading and transaction // modifications public class LSMTwoPCBTreeBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader { - private final ILSMComponent component; + private final ILSMDiskComponent component; private final BTreeBulkLoader bulkLoader; private final IIndexBulkLoader builder; private boolean cleanedUpArtifacts = false; @@ -516,14 +503,14 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { } } - frameTupleWriterFactory = ((LSMBTreeDiskComponent) component).getBTree().getLeafFrameFactory() - .getTupleWriterFactory(); + frameTupleWriterFactory = + ((LSMBTreeDiskComponent) component).getBTree().getLeafFrameFactory().getTupleWriterFactory(); bulkLoader = (BTreeBulkLoader) ((LSMBTreeDiskComponent) component).getBTree().createBulkLoader(fillFactor, verifyInput, numElementsHint, false); int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint); - BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement, - bloomFilterFalsePositiveRate); + BloomFilterSpecification bloomFilterSpec = + BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate); builder = ((LSMBTreeDiskComponent) component).getBloomFilter().createBuilder(numElementsHint, bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements()); } @@ -619,6 +606,19 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { // Do nothing } } + + // This method is used to create a target for a bulk modify operation. This + // component must then be either committed or deleted + private ILSMDiskComponent createTransactionTarget() throws HyracksDataException, IndexException { + LSMComponentFileReferences componentFileRefs; + try { + componentFileRefs = fileManager.getNewTransactionFileReference(); + } catch (IOException e) { + throw new HyracksDataException("Failed to create transaction components", e); + } + return createDiskComponent(transactionComponentFactory, componentFileRefs.getInsertIndexFileReference(), + componentFileRefs.getBloomFilterFileReference(), true); + } } @Override @@ -628,13 +628,13 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { // The accessor for disk only indexes don't use modification callback and always carry the target index version with them @Override - public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback, + public ILSMIndexAccessor createAccessor(IModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback) { return new LSMBTreeAccessor(lsmHarness, createOpContext(searchCallback, version)); } @Override - public ILSMIndexAccessorInternal createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion) + public ILSMIndexAccessor createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion) throws HyracksDataException { return new LSMBTreeAccessor(lsmHarness, createOpContext(searchCallback, targetIndexVersion)); } @@ -670,12 +670,12 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex { } @Override - public List getFirstComponentList() { + public List getFirstComponentList() { return diskComponents; } @Override - public List getSecondComponentList() { + public List getSecondComponentList() { return secondDiskComponents; } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d718dc4a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java index 29fedef..5f867db 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java @@ -29,6 +29,7 @@ import org.apache.hyracks.storage.am.common.api.ISearchPredicate; import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory; import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation; import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness; import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext; @@ -43,8 +44,8 @@ public class ExternalBTreeOpContext extends AbstractLSMIndexOperationContext { public final MultiComparator bloomFilterCmp; public final ISearchOperationCallback searchCallback; private final List componentHolder; - private final List componentsToBeMerged; - private final List componentsToBeReplicated; + private final List componentsToBeMerged; + private final List componentsToBeReplicated; private final int targetIndexVersion; public ISearchPredicate searchPredicate; public LSMBTreeCursorInitialState searchInitialState; @@ -69,9 +70,9 @@ public class ExternalBTreeOpContext extends AbstractLSMIndexOperationContext { if (deleteLeafFrame != null && this.cmp != null) { deleteLeafFrame.setMultiComparator(cmp); } - this.componentHolder = new LinkedList(); - this.componentsToBeMerged = new LinkedList(); - this.componentsToBeReplicated = new LinkedList(); + this.componentHolder = new LinkedList<>(); + this.componentsToBeMerged = new LinkedList<>(); + this.componentsToBeReplicated = new LinkedList<>(); this.searchCallback = searchCallback; this.targetIndexVersion = targetIndexVersion; searchInitialState = new LSMBTreeCursorInitialState(insertLeafFrameFactory, cmp, bloomFilterCmp, lsmHarness, @@ -119,7 +120,7 @@ public class ExternalBTreeOpContext extends AbstractLSMIndexOperationContext { } @Override - public List getComponentsToBeMerged() { + public List getComponentsToBeMerged() { return componentsToBeMerged; } @@ -139,7 +140,7 @@ public class ExternalBTreeOpContext extends AbstractLSMIndexOperationContext { } @Override - public List getComponentsToBeReplicated() { + public List getComponentsToBeReplicated() { return componentsToBeReplicated; } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d718dc4a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java index 8e29ef2..dc28db4 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java @@ -52,13 +52,14 @@ import org.apache.hyracks.storage.am.common.api.TreeIndexException; import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback; import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation; import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent; -import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; -import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext; import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy; @@ -92,7 +93,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd // A second disk component list that will be used when a transaction is // committed and will be seen by subsequent accessors - private final List secondDiskComponents; + private final List secondDiskComponents; private int version = -1; public ExternalBTreeWithBuddy(IIOManager ioManager, ITreeIndexFrameFactory btreeInteriorFrameFactory, @@ -105,18 +106,17 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd IBinaryComparatorFactory[] btreeCmpFactories, IBinaryComparatorFactory[] buddyBtreeCmpFactories, int[] buddyBTreeFields, int version, boolean durable) { super(ioManager, diskBufferCache, fileManager, diskFileMapProvider, bloomFilterFalsePositiveRate, mergePolicy, - opTracker, - ioScheduler, ioOpCallback, durable); + opTracker, ioScheduler, ioOpCallback, durable); this.btreeCmpFactories = btreeCmpFactories; this.buddyBtreeCmpFactories = buddyBtreeCmpFactories; this.buddyBTreeFields = buddyBTreeFields; this.btreeInteriorFrameFactory = btreeInteriorFrameFactory; this.btreeLeafFrameFactory = btreeLeafFrameFactory; this.buddyBtreeLeafFrameFactory = buddyBtreeLeafFrameFactory; - this.componentFactory = new LSMBTreeWithBuddyDiskComponentFactory(copyBtreeFactory, buddyBtreeFactory, - bloomFilterFactory); - this.bulkComponentFactory = new LSMBTreeWithBuddyDiskComponentFactory(bulkLoadBTreeFactory, buddyBtreeFactory, - bloomFilterFactory); + this.componentFactory = + new LSMBTreeWithBuddyDiskComponentFactory(copyBtreeFactory, buddyBtreeFactory, bloomFilterFactory); + this.bulkComponentFactory = + new LSMBTreeWithBuddyDiskComponentFactory(bulkLoadBTreeFactory, buddyBtreeFactory, bloomFilterFactory); this.secondDiskComponents = new LinkedList<>(); this.version = version; } @@ -253,7 +253,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd } @Override - public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback, + public ILSMIndexAccessor createAccessor(IModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback) throws HyracksDataException { return new LSMBTreeWithBuddyAccessor(lsmHarness, createOpContext(searchCallback, version)); } @@ -271,10 +271,10 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd // The subsume merged components is overridden to account for: // Maintaining two versions of the index @Override - public void subsumeMergedComponents(ILSMComponent newComponent, List mergedComponents) + public void subsumeMergedComponents(ILSMDiskComponent newComponent, List mergedComponents) throws HyracksDataException { - List newerList; - List olderList; + List newerList; + List olderList; if (version == 0) { newerList = diskComponents; olderList = secondDiskComponents; @@ -320,7 +320,8 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd } @Override - public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException { + public void modify(IIndexOperationContext ictx, ITupleReference tuple) + throws HyracksDataException, IndexException { throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-BTree"); } @@ -340,7 +341,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd } @Override - public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException { + public ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException { throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree"); } @@ -351,8 +352,8 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd .getBTree(); FileReference firstFile = firstTree.getFileReference(); FileReference lastFile = lastTree.getFileReference(); - LSMComponentFileReferences fileRefs = fileManager.getRelMergeFileReference(firstFile.getFile().getName(), - lastFile.getFile().getName()); + LSMComponentFileReferences fileRefs = + fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName()); return fileRefs; } @@ -364,7 +365,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd List mergingComponents = ctx.getComponentHolder(); ITreeIndexCursor cursor = new LSMBTreeWithBuddySortedCursor(bctx, buddyBTreeFields); LSMComponentFileReferences relMergeFileRefs = getMergeTargetFileName(mergingComponents); - ILSMIndexAccessorInternal accessor = new LSMBTreeWithBuddyAccessor(lsmHarness, bctx); + ILSMIndexAccessor accessor = new LSMBTreeWithBuddyAccessor(lsmHarness, bctx); // Since we have two lists of components, to tell whether we need to // keep deleted tuples, we need to know @@ -384,14 +385,15 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd } // This method creates the appropriate opContext for the targeted version - public ExternalBTreeWithBuddyOpContext createOpContext(ISearchOperationCallback searchCallback, int targetVersion) { + public ExternalBTreeWithBuddyOpContext createOpContext(ISearchOperationCallback searchCallback, + int targetVersion) { return new ExternalBTreeWithBuddyOpContext(btreeCmpFactories, buddyBtreeCmpFactories, searchCallback, targetVersion, lsmHarness, btreeInteriorFrameFactory, btreeLeafFrameFactory, buddyBtreeLeafFrameFactory); } @Override - public ILSMComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException { + public ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException { LSMBTreeWithBuddyMergeOperation mergeOp = (LSMBTreeWithBuddyMergeOperation) operation; ITreeIndexCursor cursor = mergeOp.getCursor(); ISearchPredicate btreeSearchPred = new RangePredicate(null, null, true, true, null, null); @@ -399,9 +401,9 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd opCtx.getComponentHolder().addAll(mergeOp.getMergingComponents()); search(opCtx, cursor, btreeSearchPred); - LSMBTreeWithBuddyDiskComponent mergedComponent = createDiskComponent(componentFactory, - mergeOp.getBTreeMergeTarget(), mergeOp.getBuddyBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), - true); + LSMBTreeWithBuddyDiskComponent mergedComponent = + createDiskComponent(componentFactory, mergeOp.getBTreeMergeTarget(), + mergeOp.getBuddyBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true); // In case we must keep the deleted-keys BuddyBTrees, then they must be // merged *before* merging the b-trees so that @@ -419,13 +421,13 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd long numElements = 0L; for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) { - numElements += ((LSMBTreeWithBuddyDiskComponent) mergeOp.getMergingComponents().get(i)).getBloomFilter() - .getNumElements(); + numElements += ((LSMBTreeWithBuddyDiskComponent) mergeOp.getMergingComponents().get(i)) + .getBloomFilter().getNumElements(); } int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements); - BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement, - bloomFilterFalsePositiveRate); + BloomFilterSpecification bloomFilterSpec = + BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate); IIndexBulkLoader builder = mergedComponent.getBloomFilter().createBuilder(numElements, bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements()); @@ -460,7 +462,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd @Override public void getOperationalComponents(ILSMIndexOperationContext ctx) { List operationalComponents = ctx.getComponentHolder(); - List immutableComponents; + List immutableComponents; // Identify current list in case of a merge if (version == 0) { immutableComponents = diskComponents; @@ -498,7 +500,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd } @Override - public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException { + public void markAsValid(ILSMDiskComponent lsmComponent) throws HyracksDataException { LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) lsmComponent; // Flush the bloom filter first. markAsValidInternal(component.getBTree().getBufferCache(), component.getBloomFilter()); @@ -509,11 +511,11 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd // This function is used when a new component is to be committed -- is // called by the harness. @Override - public void commitTransactionDiskComponent(ILSMComponent newComponent) throws HyracksDataException { + public void commitTransactionDiskComponent(ILSMDiskComponent newComponent) throws HyracksDataException { // determine which list is the new one and flip the pointer - List newerList; - List olderList; + List newerList; + List olderList; if (version == 0) { newerList = diskComponents; olderList = secondDiskComponents; @@ -609,12 +611,12 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd return btreeCmpFactories; } - private LSMBTreeWithBuddyDiskComponent createDiskComponent(ILSMComponentFactory factory, + private LSMBTreeWithBuddyDiskComponent createDiskComponent(ILSMDiskComponentFactory factory, FileReference insertFileRef, FileReference deleteFileRef, FileReference bloomFilterFileRef, boolean createComponent) throws HyracksDataException, IndexException { // Create new instance. - LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) factory.createLSMComponentInstance( - new LSMComponentFileReferences(insertFileRef, deleteFileRef, bloomFilterFileRef)); + LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) factory + .createComponent(new LSMComponentFileReferences(insertFileRef, deleteFileRef, bloomFilterFileRef)); if (createComponent) { component.getBTree().create(); component.getBuddyBTree().create(); @@ -654,7 +656,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd // The bulk loader used for both initial loading and transaction // modifications public class LSMTwoPCBTreeWithBuddyBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader { - private final ILSMComponent component; + private final ILSMDiskComponent component; private final BTreeBulkLoader btreeBulkLoader; private final BTreeBulkLoader buddyBtreeBulkLoader; private final IIndexBulkLoader builder; @@ -690,8 +692,8 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd buddyBtreeBulkLoader = (BTreeBulkLoader) ((LSMBTreeWithBuddyDiskComponent) component).getBuddyBTree() .createBulkLoader(fillFactor, verifyInput, numElementsHint, false); int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint); - BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement, - bloomFilterFalsePositiveRate); + BloomFilterSpecification bloomFilterSpec = + BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate); builder = ((LSMBTreeWithBuddyDiskComponent) component).getBloomFilter().createBuilder(numElementsHint, bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements()); } @@ -783,29 +785,31 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd } catch (Exception e) { } } - } - protected ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException { - LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference(); - return createDiskComponent(bulkComponentFactory, componentFileRefs.getInsertIndexFileReference(), - componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true); + // This method is used to create a target for a bulk modify operation. This + // component must then eventually be either committed or deleted + private ILSMDiskComponent createTransactionTarget() throws HyracksDataException, IndexException { + LSMComponentFileReferences componentFileRefs; + try { + componentFileRefs = fileManager.getNewTransactionFileReference(); + } catch (IOException e) { + throw new HyracksDataException("Failed to create transaction components", e); + } + return createDiskComponent(bulkComponentFactory, componentFileRefs.getInsertIndexFileReference(), + componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), + true); + } } - // This method is used to create a target for a bulk modify operation. This - // component must then eventually be either committed or deleted - private ILSMComponent createTransactionTarget() throws HyracksDataException, IndexException { - LSMComponentFileReferences componentFileRefs; - try { - componentFileRefs = fileManager.getNewTransactionFileReference(); - } catch (IOException e) { - throw new HyracksDataException("Failed to create transaction components", e); - } + protected ILSMDiskComponent createBulkLoadTarget() throws HyracksDataException, IndexException { + LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference(); return createDiskComponent(bulkComponentFactory, componentFileRefs.getInsertIndexFileReference(), - componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true); + componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), + true); } @Override - public ILSMIndexAccessorInternal createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion) + public ILSMIndexAccessor createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion) throws HyracksDataException { return new LSMBTreeWithBuddyAccessor(lsmHarness, createOpContext(searchCallback, targetIndexVersion)); } @@ -814,7 +818,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd // is successful // it will therefore add the component to the first list and enter it. @Override - public void addComponent(ILSMComponent c) throws HyracksDataException { + public void addDiskComponent(ILSMDiskComponent c) throws HyracksDataException { if (version == 0) { diskComponents.add(0, c); } else if (version == 1) { @@ -828,12 +832,12 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd } @Override - public List getFirstComponentList() { + public List getFirstComponentList() { return diskComponents; } @Override - public List getSecondComponentList() { + public List getSecondComponentList() { return secondDiskComponents; } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d718dc4a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java index c44f529..051b655 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java @@ -29,6 +29,7 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory; import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback; import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation; import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness; import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext; @@ -38,8 +39,8 @@ public class ExternalBTreeWithBuddyOpContext extends AbstractLSMIndexOperationCo private MultiComparator bTreeCmp; private MultiComparator buddyBTreeCmp; public final List componentHolder; - private final List componentsToBeMerged; - private final List componentsToBeReplicated; + private final List componentsToBeMerged; + private final List componentsToBeReplicated; public final ISearchOperationCallback searchCallback; private final int targetIndexVersion; public ISearchPredicate searchPredicate; @@ -49,9 +50,9 @@ public class ExternalBTreeWithBuddyOpContext extends AbstractLSMIndexOperationCo IBinaryComparatorFactory[] buddyBtreeCmpFactories, ISearchOperationCallback searchCallback, int targetIndexVersion, ILSMHarness lsmHarness, ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory, ITreeIndexFrameFactory buddyBtreeLeafFrameFactory) { - this.componentHolder = new LinkedList(); - this.componentsToBeMerged = new LinkedList(); - this.componentsToBeReplicated = new LinkedList(); + this.componentHolder = new LinkedList<>(); + this.componentsToBeMerged = new LinkedList<>(); + this.componentsToBeReplicated = new LinkedList<>(); this.searchCallback = searchCallback; this.targetIndexVersion = targetIndexVersion; this.bTreeCmp = MultiComparator.create(btreeCmpFactories); @@ -110,7 +111,7 @@ public class ExternalBTreeWithBuddyOpContext extends AbstractLSMIndexOperationCo } @Override - public List getComponentsToBeMerged() { + public List getComponentsToBeMerged() { return componentsToBeMerged; } @@ -129,7 +130,7 @@ public class ExternalBTreeWithBuddyOpContext extends AbstractLSMIndexOperationCo } @Override - public List getComponentsToBeReplicated() { + public List getComponentsToBeReplicated() { return componentsToBeReplicated; } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d718dc4a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java index b77cc15..53b8405 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java @@ -57,6 +57,7 @@ import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation; import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator; import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference; import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory; @@ -65,7 +66,6 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor; -import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext; import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy; @@ -102,16 +102,15 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { private final boolean hasBloomFilter; public LSMBTree(IIOManager ioManager, List virtualBufferCaches, - ITreeIndexFrameFactory interiorFrameFactory, - ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory, - ILSMIndexFileManager fileManager, TreeIndexFactory diskBTreeFactory, - TreeIndexFactory bulkLoadBTreeFactory, BloomFilterFactory bloomFilterFactory, - ILSMComponentFilterFactory filterFactory, ILSMComponentFilterFrameFactory filterFrameFactory, - LSMComponentFilterManager filterManager, double bloomFilterFalsePositiveRate, - IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] cmpFactories, - ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, - ILSMIOOperationCallback ioOpCallback, boolean needKeyDupCheck, int[] btreeFields, int[] filterFields, - boolean durable) throws HyracksDataException { + ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory, + ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager, + TreeIndexFactory diskBTreeFactory, TreeIndexFactory bulkLoadBTreeFactory, + BloomFilterFactory bloomFilterFactory, ILSMComponentFilterFactory filterFactory, + ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager, + double bloomFilterFalsePositiveRate, IFileMapProvider diskFileMapProvider, int fieldCount, + IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, + ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, boolean needKeyDupCheck, + int[] btreeFields, int[] filterFields, boolean durable) throws HyracksDataException { super(ioManager, virtualBufferCaches, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback, filterFrameFactory, filterManager, filterFields, durable); @@ -126,21 +125,20 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { insertLeafFrameFactory, cmpFactories, fieldCount, ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_" + i)), virtualBufferCache, i == 0 ? true : false, - filterFactory == null ? null : filterFactory.createLSMComponentFilter()); + filterFactory == null ? null : filterFactory.createFilter()); memoryComponents.add(mutableComponent); ++i; } componentFactory = new LSMBTreeDiskComponentFactory(diskBTreeFactory, bloomFilterFactory, filterFactory); - bulkLoadComponentFactory = new LSMBTreeDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory, - filterFactory); + bulkLoadComponentFactory = + new LSMBTreeDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory, filterFactory); this.needKeyDupCheck = needKeyDupCheck; this.btreeFields = btreeFields; this.hasBloomFilter = needKeyDupCheck; } // Without memory components - public LSMBTree(IIOManager ioManager, - ITreeIndexFrameFactory insertLeafFrameFactory, + public LSMBTree(IIOManager ioManager, ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager, TreeIndexFactory diskBTreeFactory, TreeIndexFactory bulkLoadBTreeFactory, BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate, @@ -148,8 +146,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, boolean needKeyDupCheck, boolean durable) { super(ioManager, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider, - bloomFilterFalsePositiveRate, - mergePolicy, opTracker, ioScheduler, ioOpCallback, durable); + bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback, durable); this.insertLeafFrameFactory = insertLeafFrameFactory; this.deleteLeafFrameFactory = deleteLeafFrameFactory; this.cmpFactories = cmpFactories; @@ -177,7 +174,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { if (isActivated) { throw new HyracksDataException("Failed to activate the index since it is already activated."); } - List immutableComponents = diskComponents; + List immutableComponents = diskComponents; immutableComponents.clear(); List validFileReferences; try { @@ -188,9 +185,9 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) { LSMBTreeDiskComponent component; try { - component = createDiskComponent(componentFactory, lsmComonentFileReference - .getInsertIndexFileReference(), - lsmComonentFileReference.getBloomFilterFileReference(), false); + component = + createDiskComponent(componentFactory, lsmComonentFileReference.getInsertIndexFileReference(), + lsmComonentFileReference.getBloomFilterFileReference(), false); } catch (IndexException e) { throw new HyracksDataException(e); } @@ -207,7 +204,8 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { if (flushOnExit) { BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(ioOpCallback); - ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE); + ILSMIndexAccessor accessor = + createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE); accessor.scheduleFlush(cb); try { cb.waitForIO(); @@ -216,8 +214,8 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { } } - List immutableComponents = diskComponents; - for (ILSMComponent c : immutableComponents) { + List immutableComponents = diskComponents; + for (ILSMDiskComponent c : immutableComponents) { LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c; component.getBTree().deactivateCloseHandle(); if (hasBloomFilter) { @@ -239,8 +237,8 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { throw new HyracksDataException("Failed to destroy the index since it is activated."); } - List immutableComponents = diskComponents; - for (ILSMComponent c : immutableComponents) { + List immutableComponents = diskComponents; + for (ILSMDiskComponent c : immutableComponents) { LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c; component.getBTree().destroy(); if (hasBloomFilter) { @@ -261,7 +259,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { } clearMemoryComponents(); - List immutableComponents = diskComponents; + List immutableComponents = diskComponents; for (ILSMComponent c : immutableComponents) { LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c; if (hasBloomFilter) { @@ -278,7 +276,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { @Override public void getOperationalComponents(ILSMIndexOperationContext ctx) throws HyracksDataException { - List immutableComponents = diskComponents; + List immutableComponents = diskComponents; List operationalComponents = ctx.getComponentHolder(); int cmc = currentMutableComponentId.get(); ctx.setCurrentMutableComponentId(cmc); @@ -328,7 +326,8 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { } @Override - public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException { + public void modify(IIndexOperationContext ictx, ITupleReference tuple) + throws HyracksDataException, IndexException { LSMBTreeOpContext ctx = (LSMBTreeOpContext) ictx; ITupleReference indexTuple; @@ -425,14 +424,14 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { assert ctx.getComponentHolder().size() == 1; opCtx.setOperation(IndexOperation.FLUSH); opCtx.getComponentHolder().add(flushingComponent); - ILSMIndexAccessorInternal flushAccessor = new LSMBTreeAccessor(lsmHarness, opCtx); + ILSMIndexAccessor flushAccessor = new LSMBTreeAccessor(lsmHarness, opCtx); ioScheduler.scheduleOperation(new LSMBTreeFlushOperation(flushAccessor, flushingComponent, componentFileRefs.getInsertIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), - callback, fileManager.getBaseDir(), flushingComponent.getMostRecentMarkerLSN())); + callback, fileManager.getBaseDir())); } @Override - public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException { + public ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException { LSMBTreeFlushOperation flushOp = (LSMBTreeFlushOperation) operation; LSMBTreeMemoryComponent flushingComponent = (LSMBTreeMemoryComponent) flushOp.getFlushingComponent(); IIndexAccessor accessor = flushingComponent.getBTree().createAccessor(NoOpOperationCallback.INSTANCE, @@ -490,10 +489,17 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { List filterTuples = new ArrayList<>(); filterTuples.add(flushingComponent.getLSMComponentFilter().getMinTuple()); filterTuples.add(flushingComponent.getLSMComponentFilter().getMaxTuple()); - filterManager.updateFilterInfo(component.getLSMComponentFilter(), filterTuples); - filterManager.writeFilterInfo(component.getLSMComponentFilter(), component.getBTree()); + filterManager.updateFilter(component.getLSMComponentFilter(), filterTuples); + filterManager.writeFilter(component.getLSMComponentFilter(), component.getBTree()); } - component.setMostRecentMarkerLSN(flushOp.getPrevMarkerLSN()); + // Write metadata from memory component to disk + // Q. what about the merge operation? how do we resolve conflicts + // A. Through providing an appropriate ILSMIOOperationCallback + // Must not reset the metadata before the flush is completed + // Use the copy of the metadata in the opContext + // TODO This code should be in the callback and not in the index + flushingComponent.getMetadata().copy(component.getMetadata()); + bulkLoader.end(); return component; } @@ -514,16 +520,16 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { BTree lastBTree = ((LSMBTreeDiskComponent) mergingComponents.get(mergingComponents.size() - 1)).getBTree(); FileReference firstFile = firstBTree.getFileReference(); FileReference lastFile = lastBTree.getFileReference(); - LSMComponentFileReferences relMergeFileRefs = fileManager.getRelMergeFileReference(firstFile.getFile() - .getName(), lastFile.getFile().getName()); - ILSMIndexAccessorInternal accessor = new LSMBTreeAccessor(lsmHarness, opCtx); + LSMComponentFileReferences relMergeFileRefs = + fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName()); + ILSMIndexAccessor accessor = new LSMBTreeAccessor(lsmHarness, opCtx); ioScheduler.scheduleOperation(new LSMBTreeMergeOperation(accessor, mergingComponents, cursor, relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir())); } @Override - public ILSMComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException { + public ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException { LSMBTreeMergeOperation mergeOp = (LSMBTreeMergeOperation) operation; ITreeIndexCursor cursor = mergeOp.getCursor(); RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null); @@ -572,14 +578,10 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple()); filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple()); } - filterManager.updateFilterInfo(mergedComponent.getLSMComponentFilter(), filterTuples); - filterManager.writeFilterInfo(mergedComponent.getLSMComponentFilter(), mergedComponent.getBTree()); + filterManager.updateFilter(mergedComponent.getLSMComponentFilter(), filterTuples); + filterManager.writeFilter(mergedComponent.getLSMComponentFilter(), mergedComponent.getBTree()); } - - mergedComponent - .setMostRecentMarkerLSN(mergedComponents.get(mergedComponents.size() - 1).getMostRecentMarkerLSN()); bulkLoader.end(); - return mergedComponent; } @@ -587,8 +589,8 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { FileReference btreeFileRef, FileReference bloomFilterFileRef, boolean createComponent) throws HyracksDataException, IndexException { // Create new BTree instance. - LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) factory - .createLSMComponentInstance(new LSMComponentFileReferences(btreeFileRef, null, bloomFilterFileRef)); + LSMBTreeDiskComponent component = + factory.createComponent(new LSMComponentFileReferences(btreeFileRef, null, bloomFilterFileRef)); // BTree will be closed during cleanup of merge(). if (createComponent) { component.getBTree().create(); @@ -601,10 +603,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { component.getBloomFilter().activate(); } if (component.getLSMComponentFilter() != null && !createComponent) { - filterManager.readFilterInfo(component.getLSMComponentFilter(), component.getBTree()); - } - if (!createComponent) { - component.readMostRecentMarkerLSN(component.getBTree()); + filterManager.readFilter(component.getLSMComponentFilter(), component.getBTree()); } return component; } @@ -619,14 +618,14 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { } } - protected ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException { + protected ILSMDiskComponent createBulkLoadTarget() throws HyracksDataException, IndexException { LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference(); return createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true); } @Override - public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException { + public void markAsValid(ILSMDiskComponent lsmComponent) throws HyracksDataException { // The order of forcing the dirty page to be flushed is critical. The // bloom filter must be always done first. LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) lsmComponent; @@ -637,7 +636,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { } public class LSMBTreeBulkLoader implements IIndexBulkLoader { - private final ILSMComponent component; + private final ILSMDiskComponent component; private final BTreeBulkLoader bulkLoader; private final IIndexBulkLoader builder; private boolean cleanedUpArtifacts = false; @@ -662,8 +661,8 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { if (hasBloomFilter) { int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint); - BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement, - bloomFilterFalsePositiveRate); + BloomFilterSpecification bloomFilterSpec = + BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate); builder = ((LSMBTreeDiskComponent) component).getBloomFilter().createBuilder(numElementsHint, bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements()); } else { @@ -735,10 +734,9 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { } if (component.getLSMComponentFilter() != null) { - filterManager.writeFilterInfo(component.getLSMComponentFilter(), + filterManager.writeFilter(component.getLSMComponentFilter(), ((LSMBTreeDiskComponent) component).getBTree()); } - component.setMostRecentMarkerLSN(-1L); bulkLoader.end(); if (isEmptyComponent) { @@ -774,7 +772,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { } @Override - public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback, + public ILSMIndexAccessor createAccessor(IModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback) { return new LSMBTreeAccessor(lsmHarness, createOpContext(modificationCallback, searchCallback)); } @@ -807,36 +805,36 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { @Override public ITreeIndexFrameFactory getInteriorFrameFactory() { - LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents.get( - currentMutableComponentId.get()); + LSMBTreeMemoryComponent mutableComponent = + (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get()); return mutableComponent.getBTree().getInteriorFrameFactory(); } @Override public int getFieldCount() { - LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents.get( - currentMutableComponentId.get()); + LSMBTreeMemoryComponent mutableComponent = + (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get()); return mutableComponent.getBTree().getFieldCount(); } @Override public int getFileId() { - LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents.get( - currentMutableComponentId.get()); + LSMBTreeMemoryComponent mutableComponent = + (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get()); return mutableComponent.getBTree().getFileId(); } @Override public IPageManager getPageManager() { - LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents.get( - currentMutableComponentId.get()); + LSMBTreeMemoryComponent mutableComponent = + (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get()); return mutableComponent.getBTree().getPageManager(); } @Override public ITreeIndexFrameFactory getLeafFrameFactory() { - LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents.get( - currentMutableComponentId.get()); + LSMBTreeMemoryComponent mutableComponent = + (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get()); return mutableComponent.getBTree().getLeafFrameFactory(); } @@ -853,16 +851,16 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { @Override public int getRootPageId() { - LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) memoryComponents.get( - currentMutableComponentId.get()); + LSMBTreeMemoryComponent mutableComponent = + (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get()); return mutableComponent.getBTree().getRootPageId(); } @Override public void validate() throws HyracksDataException { validateMemoryComponents(); - List immutableComponents = diskComponents; - for (ILSMComponent c : immutableComponents) { + List immutableComponents = diskComponents; + for (ILSMDiskComponent c : immutableComponents) { BTree btree = ((LSMBTreeDiskComponent) c).getBTree(); btree.validate(); } @@ -897,21 +895,11 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { if (memoryComponentsAllocated) { return; } - long markerLSN = -1L; - if (!diskComponents.isEmpty()) { - markerLSN = diskComponents.get(diskComponents.size() - 1).getMostRecentMarkerLSN(); - } else { - // Needed in case a marker was added before any record - if (memoryComponents != null && !memoryComponents.isEmpty()) { - markerLSN = memoryComponents.get(0).getMostRecentMarkerLSN(); - } - } for (ILSMComponent c : memoryComponents) { LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c; ((IVirtualBufferCache) mutableComponent.getBTree().getBufferCache()).open(); mutableComponent.getBTree().create(); mutableComponent.getBTree().activate(); - mutableComponent.setMostRecentMarkerLSN(markerLSN); } memoryComponentsAllocated = true; } @@ -961,14 +949,4 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex { memoryComponentsAllocated = false; } } - - public synchronized long getMostRecentMarkerLSN() throws HyracksDataException { - if (!isPrimaryIndex()) { - throw new HyracksDataException("Markers are only supported for primary indexes"); - } - LSMBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE); - opCtx.setOperation(IndexOperation.SEARCH); - getOperationalComponents(opCtx); - return !opCtx.getComponentHolder().isEmpty() ? opCtx.getComponentHolder().get(0).getMostRecentMarkerLSN() : -1L; - } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d718dc4a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java index 244f7a9..eb8ec92 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java @@ -23,14 +23,14 @@ import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter; import org.apache.hyracks.storage.am.btree.impls.BTree; import org.apache.hyracks.storage.am.common.api.IMetadataPageManager; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter; -import org.apache.hyracks.storage.am.lsm.common.impls.AbstractDiskLSMComponent; +import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponent; -public class LSMBTreeDiskComponent extends AbstractDiskLSMComponent { +public class LSMBTreeDiskComponent extends AbstractLSMDiskComponent { private final BTree btree; private final BloomFilter bloomFilter; public LSMBTreeDiskComponent(BTree btree, BloomFilter bloomFilter, ILSMComponentFilter filter) { - super(filter); + super((IMetadataPageManager) btree.getPageManager(), filter); this.btree = btree; this.bloomFilter = bloomFilter; } @@ -63,10 +63,4 @@ public class LSMBTreeDiskComponent extends AbstractDiskLSMComponent { public int getFileReferenceCount() { return btree.getBufferCache().getFileReferenceCount(btree.getFileId()); } - - public void readMostRecentMarkerLSN(BTree treeIndex) throws HyracksDataException { - IMetadataPageManager treeMetaManager = (IMetadataPageManager) treeIndex.getPageManager(); - treeMetaManager.get(treeMetaManager.createMetadataFrame(), MARKER_LSN_KEY, pointable); - mostRecentMarkerLSN = pointable.getByteArray() == null ? -1L : pointable.longValue(); - } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d718dc4a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java index c46119e..a8845cc 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java @@ -22,15 +22,12 @@ package org.apache.hyracks.storage.am.lsm.btree.impls; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory; import org.apache.hyracks.storage.am.btree.impls.BTree; -import org.apache.hyracks.storage.am.common.api.IndexException; -import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent; -import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFactory; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory; import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences; import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory; -import org.apache.hyracks.storage.common.buffercache.IBufferCache; -public class LSMBTreeDiskComponentFactory implements ILSMComponentFactory { +public class LSMBTreeDiskComponentFactory implements ILSMDiskComponentFactory { private final TreeIndexFactory btreeFactory; private final BloomFilterFactory bloomFilterFactory; private final ILSMComponentFilterFactory filterFactory; @@ -43,17 +40,11 @@ public class LSMBTreeDiskComponentFactory implements ILSMComponentFactory { } @Override - public ILSMComponent createLSMComponentInstance(LSMComponentFileReferences cfr) - throws IndexException, HyracksDataException { + public LSMBTreeDiskComponent createComponent(LSMComponentFileReferences cfr) throws HyracksDataException { return new LSMBTreeDiskComponent(btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()), bloomFilterFactory == null ? null : bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()), - filterFactory == null ? null : filterFactory.createLSMComponentFilter()); - } - - @Override - public IBufferCache getBufferCache() { - return btreeFactory.getBufferCache(); + filterFactory == null ? null : filterFactory.createFilter()); } public int[] getBloomFilterKeyFields() { http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d718dc4a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java index f82a1b0..a2ed40b 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java @@ -29,28 +29,26 @@ import org.apache.hyracks.storage.am.common.api.IndexException; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback; -import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor; public class LSMBTreeFlushOperation implements ILSMIOOperation, Comparable { - private final ILSMIndexAccessorInternal accessor; + private final ILSMIndexAccessor accessor; private final ILSMComponent flushingComponent; private final FileReference btreeFlushTarget; private final FileReference bloomFilterFlushTarget; private final ILSMIOOperationCallback callback; private final String indexIdentifier; - private final long prevMarkerLSN; - public LSMBTreeFlushOperation(ILSMIndexAccessorInternal accessor, ILSMComponent flushingComponent, + public LSMBTreeFlushOperation(ILSMIndexAccessor accessor, ILSMComponent flushingComponent, FileReference btreeFlushTarget, FileReference bloomFilterFlushTarget, ILSMIOOperationCallback callback, - String indexIdentifier, long prevMarkerLSN) { + String indexIdentifier) { this.accessor = accessor; this.flushingComponent = flushingComponent; this.btreeFlushTarget = btreeFlushTarget; this.bloomFilterFlushTarget = bloomFilterFlushTarget; this.callback = callback; this.indexIdentifier = indexIdentifier; - this.prevMarkerLSN = prevMarkerLSN; } @Override @@ -60,7 +58,7 @@ public class LSMBTreeFlushOperation implements ILSMIOOperation, Comparable getWriteDevices() { - Set devs = new HashSet(); + Set devs = new HashSet<>(); devs.add(btreeFlushTarget.getDeviceHandle()); if (bloomFilterFlushTarget != null) { devs.add(bloomFilterFlushTarget.getDeviceHandle()); @@ -87,7 +85,7 @@ public class LSMBTreeFlushOperation implements ILSMIOOperation, Comparable mergingComponents; private final ITreeIndexCursor cursor; private final FileReference btreeMergeTarget; @@ -43,7 +43,7 @@ public class LSMBTreeMergeOperation implements ILSMIOOperation { private final ILSMIOOperationCallback callback; private final String indexIdentifier; - public LSMBTreeMergeOperation(ILSMIndexAccessorInternal accessor, List mergingComponents, + public LSMBTreeMergeOperation(ILSMIndexAccessor accessor, List mergingComponents, ITreeIndexCursor cursor, FileReference btreeMergeTarget, FileReference bloomFilterMergeTarget, ILSMIOOperationCallback callback, String indexIdentifier) { this.accessor = accessor; @@ -57,7 +57,7 @@ public class LSMBTreeMergeOperation implements ILSMIOOperation { @Override public Set getReadDevices() { - Set devs = new HashSet(); + Set devs = new HashSet<>(); for (ILSMComponent o : mergingComponents) { LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) o; devs.add(component.getBTree().getFileReference().getDeviceHandle()); @@ -70,7 +70,7 @@ public class LSMBTreeMergeOperation implements ILSMIOOperation { @Override public Set getWriteDevices() { - Set devs = new HashSet(); + Set devs = new HashSet<>(); devs.add(btreeMergeTarget.getDeviceHandle()); if (bloomFilterMergeTarget != null) { devs.add(bloomFilterMergeTarget.getDeviceHandle()); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d718dc4a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java index fc8f9b2..6dafe43 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java @@ -36,8 +36,10 @@ import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback; import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation; import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator; import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent; import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext; public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext { @@ -57,8 +59,8 @@ public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext { public IModificationOperationCallback modificationCallback; public ISearchOperationCallback searchCallback; private final List componentHolder; - private final List componentsToBeMerged; - private final List componentsToBeReplicated; + private final List componentsToBeMerged; + private final List componentsToBeReplicated; public final PermutingTupleReference indexTuple; public final MultiComparator filterCmp; public final PermutingTupleReference filterTuple; @@ -67,10 +69,10 @@ public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext { public LSMBTreeCursorInitialState searchInitialState; public LSMBTreePointSearchCursor insertSearchCursor; - public LSMBTreeOpContext(List mutableComponents, ITreeIndexFrameFactory insertLeafFrameFactory, + public LSMBTreeOpContext(List mutableComponents, ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory, IModificationOperationCallback modificationCallback, - ISearchOperationCallback searchCallback, int numBloomFilterKeyFields, int[] btreeFields, int[] filterFields, - ILSMHarness lsmHarness) { + ISearchOperationCallback searchCallback, int numBloomFilterKeyFields, int[] btreeFields, + int[] filterFields, ILSMHarness lsmHarness) { LSMBTreeMemoryComponent c = (LSMBTreeMemoryComponent) mutableComponents.get(0); IBinaryComparatorFactory cmpFactories[] = c.getBTree().getComparatorFactories(); if (cmpFactories[0] != null) { @@ -103,9 +105,9 @@ public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext { if (deleteLeafFrame != null && this.cmp != null) { deleteLeafFrame.setMultiComparator(cmp); } - this.componentHolder = new LinkedList(); - this.componentsToBeMerged = new LinkedList(); - this.componentsToBeReplicated = new LinkedList(); + this.componentHolder = new LinkedList<>(); + this.componentsToBeMerged = new LinkedList<>(); + this.componentsToBeReplicated = new LinkedList<>(); this.modificationCallback = modificationCallback; this.searchCallback = searchCallback; @@ -199,7 +201,7 @@ public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext { } @Override - public List getComponentsToBeMerged() { + public List getComponentsToBeMerged() { return componentsToBeMerged; } @@ -214,7 +216,7 @@ public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext { } @Override - public List getComponentsToBeReplicated() { + public List getComponentsToBeReplicated() { return componentsToBeReplicated; } }