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 587C0200C66 for ; Sat, 15 Apr 2017 23:44:50 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 56FA6160BA0; Sat, 15 Apr 2017 21:44:50 +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 067FE160B8B for ; Sat, 15 Apr 2017 23:44:47 +0200 (CEST) Received: (qmail 41023 invoked by uid 500); 15 Apr 2017 21:44:47 -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 40911 invoked by uid 99); 15 Apr 2017 21:44:45 -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; Sat, 15 Apr 2017 21:44:45 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 1F21BE382D; Sat, 15 Apr 2017 21:44:45 +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: Sat, 15 Apr 2017 21:44:49 -0000 Message-Id: <08d3742c54d54044abc3671685eb28de@git.apache.org> In-Reply-To: <04cfd80b44fd4ddea8b1c9219b10a5e9@git.apache.org> References: <04cfd80b44fd4ddea8b1c9219b10a5e9@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [06/13] asterixdb git commit: Cleanup storage exceptions archived-at: Sat, 15 Apr 2017 21:44:50 -0000 http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java index 9299620..60f8e21 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java @@ -28,13 +28,10 @@ import org.apache.hyracks.data.std.primitive.IntegerPointable; import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder; import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference; import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference; -import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer; import org.apache.hyracks.dataflow.common.utils.TupleUtils; import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor; import org.apache.hyracks.storage.am.btree.impls.RangePredicate; import org.apache.hyracks.storage.am.common.api.IIndexCursor; -import org.apache.hyracks.storage.am.common.api.IndexException; -import org.apache.hyracks.storage.am.common.api.TreeIndexException; import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator; import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference; import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference; @@ -68,7 +65,7 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor { } public void prepare(BTreeAccessor btreeAccessor, RangePredicate btreePred, MultiComparator tokenFieldsCmp, - MultiComparator btreeCmp) throws HyracksDataException, IndexException { + MultiComparator btreeCmp) throws HyracksDataException { // Avoid object creation if this.btreeAccessor == btreeAccessor. if (this.btreeAccessor != btreeAccessor) { this.btreeAccessor = btreeAccessor; @@ -87,7 +84,7 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor { return size() - cursor.size(); } - public void reset(ITupleReference tuple) throws HyracksDataException, IndexException { + public void reset(ITupleReference tuple) throws HyracksDataException { numElements = -1; // Copy the tokens tuple for later use in btree probes. TupleUtils.copyTuple(tokenTupleBuilder, tuple, tuple.getFieldCount()); @@ -104,7 +101,7 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor { } @Override - public void pinPages() throws HyracksDataException, IndexException { + public void pinPages() throws HyracksDataException { btreePred.setLowKeyComparator(tokenFieldsCmp); btreePred.setHighKeyComparator(tokenFieldsCmp); btreePred.setLowKey(tokenTuple, true); @@ -122,7 +119,7 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor { } @Override - public boolean hasNext() throws HyracksDataException, IndexException { + public boolean hasNext() throws HyracksDataException { return btreeCursor.hasNext(); } @@ -155,8 +152,6 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor { } } catch (HyracksDataException e) { e.printStackTrace(); - } catch (IndexException e) { - e.printStackTrace(); } finally { try { countingCursor.close(); @@ -184,8 +179,7 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor { } @Override - public boolean containsKey(ITupleReference searchTuple, MultiComparator invListCmp) throws HyracksDataException, - IndexException { + public boolean containsKey(ITupleReference searchTuple, MultiComparator invListCmp) throws HyracksDataException { // Close cursor if necessary. unpinPages(); btreeSearchTuple.addTuple(searchTuple); @@ -195,9 +189,9 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor { btreePred.setHighKey(btreeSearchTuple, true); try { btreeAccessor.search(btreeCursor, btreePred); - } catch (TreeIndexException e) { + } catch (Exception e) { btreeSearchTuple.removeLastTuple(); - throw new HyracksDataException(e); + throw HyracksDataException.create(e); } boolean containsKey = false; try { @@ -212,7 +206,7 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor { @SuppressWarnings("rawtypes") @Override - public String printInvList(ISerializerDeserializer[] serdes) throws HyracksDataException, IndexException { + public String printInvList(ISerializerDeserializer[] serdes) throws HyracksDataException { StringBuilder strBuilder = new StringBuilder(); try { while (btreeCursor.hasNext()) { @@ -228,11 +222,7 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor { btreeCursor.close(); btreeCursor.reset(); } - try { - btreeAccessor.search(btreeCursor, btreePred); - } catch (TreeIndexException e) { - throw new HyracksDataException(e); - } + btreeAccessor.search(btreeCursor, btreePred); return strBuilder.toString(); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java index 0bea41f..24d1cdd 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java @@ -18,7 +18,7 @@ */ package org.apache.hyracks.storage.am.lsm.invertedindex.inmemory; -import java.util.ArrayList; +import java.util.List; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; @@ -32,7 +32,6 @@ import org.apache.hyracks.storage.am.common.api.IIndexOperationContext; import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback; import org.apache.hyracks.storage.am.common.api.IPageManager; import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback; -import org.apache.hyracks.storage.am.common.api.IndexException; import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor; @@ -59,7 +58,7 @@ public class PartitionedInMemoryInvertedIndex extends InMemoryInvertedIndex impl @Override public void insert(ITupleReference tuple, BTreeAccessor btreeAccessor, IIndexOperationContext ictx) - throws HyracksDataException, IndexException { + throws HyracksDataException { super.insert(tuple, btreeAccessor, ictx); PartitionedInMemoryInvertedIndexOpContext ctx = (PartitionedInMemoryInvertedIndexOpContext) ictx; PartitionedInvertedIndexTokenizingTupleIterator tupleIter = @@ -98,7 +97,7 @@ public class PartitionedInMemoryInvertedIndex extends InMemoryInvertedIndex impl @Override public boolean openInvertedListPartitionCursors(IInvertedIndexSearcher searcher, IIndexOperationContext ictx, short numTokensLowerBound, short numTokensUpperBound, InvertedListPartitions invListPartitions, - ArrayList cursorsOrderedByTokens) throws HyracksDataException, IndexException { + List cursorsOrderedByTokens) throws HyracksDataException { short minPartitionIndex; short maxPartitionIndex; partitionIndexLock.readLock().lock(); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java index 70a5024..1dcd18e 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java @@ -46,9 +46,6 @@ import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback; import org.apache.hyracks.storage.am.common.api.IPageManagerFactory; import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback; import org.apache.hyracks.storage.am.common.api.ISearchPredicate; -import org.apache.hyracks.storage.am.common.api.IndexException; -import org.apache.hyracks.storage.am.common.api.TreeIndexException; -import org.apache.hyracks.storage.am.common.api.UnsortedInputException; import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback; import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator; import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference; @@ -57,7 +54,6 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccesso import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor; -import org.apache.hyracks.storage.am.lsm.invertedindex.exceptions.InvertedIndexException; import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate; import org.apache.hyracks.storage.am.lsm.invertedindex.search.TOccurrenceSearcher; import org.apache.hyracks.storage.common.buffercache.IBufferCache; @@ -258,7 +254,7 @@ public class OnDiskInvertedIndex implements IInvertedIndex { @Override public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey, - IIndexOperationContext ictx) throws HyracksDataException, IndexException { + IIndexOperationContext ictx) throws HyracksDataException { OnDiskInvertedIndexOpContext ctx = (OnDiskInvertedIndexOpContext) ictx; ctx.btreePred.setLowKeyComparator(ctx.searchCmp); ctx.btreePred.setHighKeyComparator(ctx.searchCmp); @@ -311,7 +307,7 @@ public class OnDiskInvertedIndex implements IInvertedIndex { private IFIFOPageQueue queue; public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint, - boolean checkIfEmptyIndex, int startPageId) throws IndexException, HyracksDataException { + boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException { this.verifyInput = verifyInput; this.tokenCmp = MultiComparator.create(btree.getComparatorFactories()); this.invListCmp = MultiComparator.create(invListCmpFactories); @@ -338,7 +334,7 @@ public class OnDiskInvertedIndex implements IInvertedIndex { currentPage = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, currentPageId)); } - private void createAndInsertBTreeTuple() throws IndexException, HyracksDataException { + private void createAndInsertBTreeTuple() throws HyracksDataException { // Build tuple. btreeTupleBuilder.reset(); DataOutput output = btreeTupleBuilder.getDataOutput(); @@ -375,7 +371,7 @@ public class OnDiskInvertedIndex implements IInvertedIndex { * Key fields of inverted list are fixed size. */ @Override - public void add(ITupleReference tuple) throws IndexException, HyracksDataException { + public void add(ITupleReference tuple) throws HyracksDataException { boolean firstElement = lastTupleBuilder.getSize() == 0; boolean startNewList = firstElement; if (!firstElement) { @@ -416,7 +412,7 @@ public class OnDiskInvertedIndex implements IInvertedIndex { if (verifyInput && lastTupleBuilder.getSize() != 0) { if (allCmp.compare(tuple, lastTuple) <= 0) { - throw new UnsortedInputException( + throw new HyracksDataException( "Input stream given to OnDiskInvertedIndex bulk load is not sorted."); } } @@ -430,7 +426,7 @@ public class OnDiskInvertedIndex implements IInvertedIndex { } @Override - public void end() throws IndexException, HyracksDataException { + public void end() throws HyracksDataException { // The last tuple builder is empty if add() was never called. if (lastTupleBuilder.getSize() != 0) { createAndInsertBTreeTuple(); @@ -505,10 +501,8 @@ public class OnDiskInvertedIndex implements IInvertedIndex { } @Override - public void search(IIndexCursor cursor, ISearchPredicate searchPred) - throws HyracksDataException, IndexException { - searcher.search((OnDiskInvertedIndexSearchCursor) cursor, (InvertedIndexSearchPredicate) searchPred, - opCtx); + public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException { + searcher.search((OnDiskInvertedIndexSearchCursor) cursor, (InvertedIndexSearchPredicate) searchPred, opCtx); } @Override @@ -518,7 +512,7 @@ public class OnDiskInvertedIndex implements IInvertedIndex { @Override public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey) - throws HyracksDataException, IndexException { + throws HyracksDataException { index.openInvertedListCursor(listCursor, searchKey, opCtx); } @@ -528,29 +522,28 @@ public class OnDiskInvertedIndex implements IInvertedIndex { } @Override - public void rangeSearch(IIndexCursor cursor, ISearchPredicate searchPred) - throws HyracksDataException, IndexException { + public void rangeSearch(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException { OnDiskInvertedIndexRangeSearchCursor rangeSearchCursor = (OnDiskInvertedIndexRangeSearchCursor) cursor; rangeSearchCursor.open(null, searchPred); } @Override - public void insert(ITupleReference tuple) throws HyracksDataException, IndexException { + public void insert(ITupleReference tuple) throws HyracksDataException { throw new UnsupportedOperationException("Insert not supported by inverted index."); } @Override - public void update(ITupleReference tuple) throws HyracksDataException, IndexException { + public void update(ITupleReference tuple) throws HyracksDataException { throw new UnsupportedOperationException("Update not supported by inverted index."); } @Override - public void delete(ITupleReference tuple) throws HyracksDataException, IndexException { + public void delete(ITupleReference tuple) throws HyracksDataException { throw new UnsupportedOperationException("Delete not supported by inverted index."); } @Override - public void upsert(ITupleReference tuple) throws HyracksDataException, TreeIndexException { + public void upsert(ITupleReference tuple) throws HyracksDataException { throw new UnsupportedOperationException("Upsert not supported by inverted index."); } } @@ -603,13 +596,9 @@ public class OnDiskInvertedIndex implements IInvertedIndex { @Override public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint, - boolean checkIfEmptyIndex) throws IndexException { - try { - return new OnDiskInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, - rootPageId); - } catch (HyracksDataException e) { - throw new InvertedIndexException(e); - } + boolean checkIfEmptyIndex) throws HyracksDataException { + return new OnDiskInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, + rootPageId); } @Override @@ -628,8 +617,7 @@ public class OnDiskInvertedIndex implements IInvertedIndex { PermutingTupleReference tokenTuple = new PermutingTupleReference(fieldPermutation); IInvertedIndexAccessor invIndexAccessor = - (IInvertedIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE, - NoOpOperationCallback.INSTANCE); + (IInvertedIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE); IInvertedListCursor invListCursor = invIndexAccessor.createInvertedListCursor(); MultiComparator invListCmp = MultiComparator.create(invListCmpFactories); @@ -667,8 +655,6 @@ public class OnDiskInvertedIndex implements IInvertedIndex { invListCursor.unpinPages(); } } - } catch (IndexException e) { - throw new HyracksDataException(e); } finally { btreeCursor.close(); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java index bd6ce9b..4a74833 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java @@ -27,7 +27,6 @@ import org.apache.hyracks.storage.am.common.api.IIndexAccessor; import org.apache.hyracks.storage.am.common.api.IIndexCursor; import org.apache.hyracks.storage.am.common.api.IIndexOperationContext; import org.apache.hyracks.storage.am.common.api.ISearchPredicate; -import org.apache.hyracks.storage.am.common.api.IndexException; import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback; import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference; import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference; @@ -70,19 +69,15 @@ public class OnDiskInvertedIndexRangeSearchCursor implements IIndexCursor { } @Override - public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException, IndexException { + public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { this.btreePred = (RangePredicate) searchPred; - try { - btreeAccessor.search(btreeCursor, btreePred); - } catch (IndexException e) { - throw new HyracksDataException(e); - } + btreeAccessor.search(btreeCursor, btreePred); invListCursor.pinPages(); unpinNeeded = true; } @Override - public boolean hasNext() throws HyracksDataException, IndexException { + public boolean hasNext() throws HyracksDataException { if (invListCursor.hasNext()) { return true; } @@ -95,11 +90,7 @@ public class OnDiskInvertedIndexRangeSearchCursor implements IIndexCursor { } btreeCursor.next(); tokenTuple.reset(btreeCursor.getTuple()); - try { - invIndex.openInvertedListCursor(invListCursor, tokenTuple, opCtx); - } catch (IndexException e) { - throw new HyracksDataException(e); - } + invIndex.openInvertedListCursor(invListCursor, tokenTuple, opCtx); invListCursor.pinPages(); invListCursor.hasNext(); unpinNeeded = true; @@ -127,7 +118,7 @@ public class OnDiskInvertedIndexRangeSearchCursor implements IIndexCursor { } @Override - public void reset() throws HyracksDataException, IndexException { + public void reset() throws HyracksDataException { if (unpinNeeded) { invListCursor.unpinPages(); unpinNeeded = false; http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java index a3a4de4..697d217 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java @@ -19,7 +19,7 @@ package org.apache.hyracks.storage.am.lsm.invertedindex.ondisk; -import java.util.ArrayList; +import java.util.List; import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; import org.apache.hyracks.api.dataflow.value.ITypeTraits; @@ -32,7 +32,6 @@ import org.apache.hyracks.storage.am.common.api.IIndexOperationContext; import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback; import org.apache.hyracks.storage.am.common.api.IPageManagerFactory; import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback; -import org.apache.hyracks.storage.am.common.api.IndexException; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor; @@ -70,7 +69,7 @@ public class PartitionedOnDiskInvertedIndex extends OnDiskInvertedIndex implemen @Override public boolean openInvertedListPartitionCursors(IInvertedIndexSearcher searcher, IIndexOperationContext ictx, short numTokensLowerBound, short numTokensUpperBound, InvertedListPartitions invListPartitions, - ArrayList cursorsOrderedByTokens) throws HyracksDataException, IndexException { + List cursorsOrderedByTokens) throws HyracksDataException { PartitionedTOccurrenceSearcher partSearcher = (PartitionedTOccurrenceSearcher) searcher; OnDiskInvertedIndexOpContext ctx = (OnDiskInvertedIndexOpContext) ictx; ITupleReference lowSearchKey = null; http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java index e4b220b..764d9a5 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java @@ -42,7 +42,6 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IObjectFactory; -import org.apache.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException; import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeFrameTupleAccessor; import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeTupleReference; import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer; @@ -52,8 +51,8 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo. import org.apache.hyracks.storage.am.lsm.invertedindex.util.ObjectCache; public abstract class AbstractTOccurrenceSearcher implements IInvertedIndexSearcher { - protected static final RecordDescriptor QUERY_TOKEN_REC_DESC = new RecordDescriptor( - new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer() }); + protected static final RecordDescriptor QUERY_TOKEN_REC_DESC = + new RecordDescriptor(new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer() }); protected final int OBJECT_CACHE_INIT_SIZE = 10; protected final int OBJECT_CACHE_EXPAND_SIZE = 10; @@ -82,20 +81,20 @@ public abstract class AbstractTOccurrenceSearcher implements IInvertedIndexSearc this.invIndex = invIndex; this.invListCmp = MultiComparator.create(invIndex.getInvListCmpFactories()); this.invListCursorFactory = new InvertedListCursorFactory(invIndex); - this.invListCursorCache = new ObjectCache(invListCursorFactory, OBJECT_CACHE_INIT_SIZE, + this.invListCursorCache = new ObjectCache<>(invListCursorFactory, OBJECT_CACHE_INIT_SIZE, OBJECT_CACHE_EXPAND_SIZE); - this.queryTokenFrame = new VSizeFrame(ctx); + this.queryTokenFrame = new VSizeFrame(ctx); this.queryTokenAppender = new FrameTupleAppenderAccessor(QUERY_TOKEN_REC_DESC); this.queryTokenAppender.reset(queryTokenFrame, true); } + @Override public void reset() { searchResult.clear(); invListMerger.reset(); } - protected void tokenizeQuery(InvertedIndexSearchPredicate searchPred) throws HyracksDataException, - OccurrenceThresholdPanicException { + protected void tokenizeQuery(InvertedIndexSearchPredicate searchPred) throws HyracksDataException { ITupleReference queryTuple = searchPred.getQueryTuple(); int queryFieldIndex = searchPred.getQueryFieldIndex(); IBinaryTokenizer queryTokenizer = searchPred.getQueryTokenizer(); @@ -144,10 +143,12 @@ public abstract class AbstractTOccurrenceSearcher implements IInvertedIndexSearc } } + @Override public IFrameTupleAccessor createResultFrameTupleAccessor() { return new FixedSizeFrameTupleAccessor(ctx.getInitialFrameSize(), searchResult.getTypeTraits()); } + @Override public ITupleReference createResultFrameTupleReference() { return new FixedSizeTupleReference(searchResult.getTypeTraits()); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java index 5c916f2..55aa159 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java @@ -27,8 +27,6 @@ import org.apache.hyracks.api.context.IHyracksCommonContext; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.data.std.primitive.IntegerPointable; import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference; -import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer; -import org.apache.hyracks.storage.am.common.api.IndexException; import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor; @@ -50,7 +48,7 @@ public class InvertedListMerger { } public void merge(ArrayList invListCursors, int occurrenceThreshold, int numPrefixLists, - SearchResult searchResult) throws HyracksDataException, IndexException { + SearchResult searchResult) throws HyracksDataException { Collections.sort(invListCursors); int numInvLists = invListCursors.size(); SearchResult result = null; @@ -88,7 +86,7 @@ public class InvertedListMerger { protected void mergeSuffixListProbe(IInvertedListCursor invListCursor, SearchResult prevSearchResult, SearchResult newSearchResult, int invListIx, int numInvLists, int occurrenceThreshold) - throws HyracksDataException, IndexException { + throws HyracksDataException { int prevBufIdx = 0; int maxPrevBufIdx = prevSearchResult.getCurrentBufferIndex(); @@ -104,7 +102,8 @@ public class InvertedListMerger { while (resultTidx < resultFrameTupleAcc.getTupleCount()) { resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx)); - int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)); + int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), + resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)); if (invListCursor.containsKey(resultTuple, invListCmp)) { count++; @@ -129,7 +128,7 @@ public class InvertedListMerger { protected void mergeSuffixListScan(IInvertedListCursor invListCursor, SearchResult prevSearchResult, SearchResult newSearchResult, int invListIx, int numInvLists, int occurrenceThreshold) - throws HyracksDataException, IndexException { + throws HyracksDataException { int prevBufIdx = 0; int maxPrevBufIdx = prevSearchResult.getCurrentBufferIndex(); @@ -147,8 +146,9 @@ public class InvertedListMerger { int invListTidx = 0; int invListNumTuples = invListCursor.size(); - if (invListCursor.hasNext()) + if (invListCursor.hasNext()) { invListCursor.next(); + } while (invListTidx < invListNumTuples && resultTidx < resultFrameTupleAcc.getTupleCount()) { @@ -158,7 +158,8 @@ public class InvertedListMerger { int cmp = invListCmp.compare(invListTuple, resultTuple); if (cmp == 0) { - int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)) + 1; + int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), + resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)) + 1; newSearchResult.append(resultTuple, count); advanceCursor = true; advancePrevResult = true; @@ -167,7 +168,8 @@ public class InvertedListMerger { advanceCursor = true; advancePrevResult = false; } else { - int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)); + int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), + resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)); if (count + numInvLists - invListIx > occurrenceThreshold) { newSearchResult.append(resultTuple, count); } @@ -201,7 +203,8 @@ public class InvertedListMerger { resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx)); - int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)); + int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), + resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)); if (count + numInvLists - invListIx > occurrenceThreshold) { newSearchResult.append(resultTuple, count); } @@ -219,7 +222,7 @@ public class InvertedListMerger { } protected void mergePrefixList(IInvertedListCursor invListCursor, SearchResult prevSearchResult, - SearchResult newSearchResult) throws HyracksDataException, IndexException { + SearchResult newSearchResult) throws HyracksDataException { int prevBufIdx = 0; int maxPrevBufIdx = prevSearchResult.getCurrentBufferIndex(); @@ -237,8 +240,9 @@ public class InvertedListMerger { int invListTidx = 0; int invListNumTuples = invListCursor.size(); - if (invListCursor.hasNext()) + if (invListCursor.hasNext()) { invListCursor.next(); + } while (invListTidx < invListNumTuples && resultTidx < resultFrameTupleAcc.getTupleCount()) { @@ -247,7 +251,8 @@ public class InvertedListMerger { int cmp = invListCmp.compare(invListTuple, resultTuple); if (cmp == 0) { - int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)) + 1; + int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), + resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)) + 1; newSearchResult.append(resultTuple, count); advanceCursor = true; advancePrevResult = true; @@ -258,7 +263,8 @@ public class InvertedListMerger { advanceCursor = true; advancePrevResult = false; } else { - int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)); + int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), + resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)); newSearchResult.append(resultTuple, count); advanceCursor = false; advancePrevResult = true; @@ -300,7 +306,8 @@ public class InvertedListMerger { resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx)); - int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)); + int count = IntegerPointable.getInteger(resultTuple.getFieldData(0), + resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)); newSearchResult.append(resultTuple, count); resultTidx++; http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java index 7c7e781..9221e1f 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java @@ -23,19 +23,18 @@ import java.io.IOException; import java.util.ArrayList; import org.apache.hyracks.api.context.IHyracksCommonContext; +import org.apache.hyracks.api.exceptions.ErrorCode; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.data.std.primitive.ShortPointable; import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder; import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference; import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference; import org.apache.hyracks.storage.am.common.api.IIndexOperationContext; -import org.apache.hyracks.storage.am.common.api.IndexException; import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IPartitionedInvertedIndex; -import org.apache.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException; import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexSearchCursor; public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher { @@ -50,7 +49,7 @@ public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher // Inverted list cursors ordered by token. Used to read relevant inverted-list partitions of one token one after // the other for better I/O performance (because the partitions of one inverted list are stored contiguously in a file). // The above implies that we currently require holding all inverted list for a query in memory. - protected final ArrayList cursorsOrderedByTokens = new ArrayList(); + protected final ArrayList cursorsOrderedByTokens = new ArrayList<>(); protected final InvertedListPartitions partitions = new InvertedListPartitions(); public PartitionedTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) @@ -87,8 +86,9 @@ public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher } } + @Override public void search(OnDiskInvertedIndexSearchCursor resultCursor, InvertedIndexSearchPredicate searchPred, - IIndexOperationContext ictx) throws HyracksDataException, IndexException { + IIndexOperationContext ictx) throws HyracksDataException { IPartitionedInvertedIndex partInvIndex = (IPartitionedInvertedIndex) invIndex; searchResult.reset(); if (partInvIndex.isEmpty()) { @@ -104,7 +104,7 @@ public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher occurrenceThreshold = searchModifier.getOccurrenceThreshold(numQueryTokens); if (occurrenceThreshold <= 0) { - throw new OccurrenceThresholdPanicException("Merge Threshold is <= 0. Failing Search."); + throw HyracksDataException.create(ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION); } short maxCountPossible = numQueryTokens; http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java index 39418f3..508a51d 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java @@ -22,25 +22,25 @@ package org.apache.hyracks.storage.am.lsm.invertedindex.search; import java.util.ArrayList; import org.apache.hyracks.api.context.IHyracksCommonContext; +import org.apache.hyracks.api.exceptions.ErrorCode; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.storage.am.common.api.IIndexOperationContext; -import org.apache.hyracks.storage.am.common.api.IndexException; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier; import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor; -import org.apache.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException; import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexSearchCursor; public class TOccurrenceSearcher extends AbstractTOccurrenceSearcher { - protected final ArrayList invListCursors = new ArrayList(); + protected final ArrayList invListCursors = new ArrayList<>(); public TOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) throws HyracksDataException { super(ctx, invIndex); } + @Override public void search(OnDiskInvertedIndexSearchCursor resultCursor, InvertedIndexSearchPredicate searchPred, - IIndexOperationContext ictx) throws HyracksDataException, IndexException { + IIndexOperationContext ictx) throws HyracksDataException { tokenizeQuery(searchPred); int numQueryTokens = queryTokenAppender.getTupleCount(); @@ -56,7 +56,7 @@ public class TOccurrenceSearcher extends AbstractTOccurrenceSearcher { IInvertedIndexSearchModifier searchModifier = searchPred.getSearchModifier(); occurrenceThreshold = searchModifier.getOccurrenceThreshold(numQueryTokens); if (occurrenceThreshold <= 0) { - throw new OccurrenceThresholdPanicException("Merge threshold is <= 0. Failing Search."); + throw HyracksDataException.create(ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION); } int numPrefixLists = searchModifier.getNumPrefixLists(occurrenceThreshold, invListCursors.size()); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java index c6a552c..075cded 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java @@ -34,7 +34,6 @@ import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory; import org.apache.hyracks.storage.am.common.api.IPageManager; import org.apache.hyracks.storage.am.common.api.IPageManagerFactory; import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory; -import org.apache.hyracks.storage.am.common.api.IndexException; import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; @@ -69,8 +68,8 @@ public class InvertedIndexUtils { IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory, FileReference btreeFileRef) throws HyracksDataException { - return new InMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits, - invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, btreeFileRef); + return new InMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits, invListCmpFactories, + tokenTypeTraits, tokenCmpFactories, tokenizerFactory, btreeFileRef); } public static InMemoryInvertedIndex createPartitionedInMemoryBTreeInvertedindex(IBufferCache memBufferCache, @@ -86,7 +85,7 @@ public class InvertedIndexUtils { IFileMapProvider fileMapProvider, ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference invListsFile, - IPageManagerFactory pageManagerFactory) throws IndexException, HyracksDataException { + IPageManagerFactory pageManagerFactory) throws HyracksDataException { IInvertedListBuilder builder = new FixedSizeElementInvertedListBuilder(invListTypeTraits); FileReference btreeFile = getBTreeFile(ioManager, invListsFile); return new OnDiskInvertedIndex(bufferCache, fileMapProvider, builder, invListTypeTraits, invListCmpFactories, @@ -97,7 +96,7 @@ public class InvertedIndexUtils { IBufferCache bufferCache, IFileMapProvider fileMapProvider, ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference invListsFile, - IPageManagerFactory pageManagerFactory) throws IndexException, HyracksDataException { + IPageManagerFactory pageManagerFactory) throws HyracksDataException { IInvertedListBuilder builder = new FixedSizeElementInvertedListBuilder(invListTypeTraits); FileReference btreeFile = getBTreeFile(ioManager, invListsFile); return new PartitionedOnDiskInvertedIndex(bufferCache, fileMapProvider, builder, invListTypeTraits, @@ -109,10 +108,9 @@ public class InvertedIndexUtils { return ioManager.resolveAbsolutePath(invListsFile.getFile().getPath() + "_btree"); } - public static BTreeFactory createDeletedKeysBTreeFactory(IIOManager ioManager, - IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits, - IBinaryComparatorFactory[] invListCmpFactories, IBufferCache diskBufferCache, - IPageManagerFactory freePageManagerFactory) throws HyracksDataException { + public static BTreeFactory createDeletedKeysBTreeFactory(IIOManager ioManager, IFileMapProvider diskFileMapProvider, + ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories, + IBufferCache diskBufferCache, IPageManagerFactory freePageManagerFactory) throws HyracksDataException { TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(invListTypeTraits); ITreeIndexFrameFactory leafFrameFactory = BTreeUtils.getLeafFrameFactory(tupleWriterFactory, BTreeLeafFrameType.REGULAR_NSM); @@ -132,7 +130,7 @@ public class InvertedIndexUtils { ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, - IMetadataPageManagerFactory pageManagerFactory) throws IndexException, HyracksDataException { + IMetadataPageManagerFactory pageManagerFactory) throws HyracksDataException { BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories, diskBufferCache, pageManagerFactory); @@ -166,9 +164,9 @@ public class InvertedIndexUtils { LSMInvertedIndex invIndex = new LSMInvertedIndex(ioManager, virtualBufferCaches, invIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory, filterFactory, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories, - tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler, - ioOpCallback, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps, - invertedIndexFieldsForNonBulkLoadOps, durable); + tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallback, + invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, + durable); return invIndex; } @@ -181,7 +179,7 @@ public class InvertedIndexUtils { ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, - IPageManagerFactory pageManagerFactory) throws IndexException, HyracksDataException { + IPageManagerFactory pageManagerFactory) throws HyracksDataException { BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories, diskBufferCache, pageManagerFactory); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java index 1b2ff6f..ba61d6c 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java @@ -30,7 +30,6 @@ import org.apache.hyracks.storage.am.common.api.IIndex; import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory; import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory; import org.apache.hyracks.storage.am.common.api.ITreeIndex; -import org.apache.hyracks.storage.am.common.api.TreeIndexException; import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; @@ -95,16 +94,11 @@ public class ExternalRTreeDataflowHelper extends LSMRTreeDataflowHelper { RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) throws HyracksDataException { - try { - return LSMRTreeUtils.createExternalRTree(ctx.getIOManager(), file, diskBufferCache, diskFileMapProvider, - typeTraits, - rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType, - bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, - ioOpCallbackFactory.createIoOpCallback(), linearizeCmpFactory, btreeFields, version, durable, - isPointMBR, (IMetadataPageManagerFactory) opDesc.getPageManagerFactory()); - } catch (TreeIndexException e) { - throw new HyracksDataException(e); - } + return LSMRTreeUtils.createExternalRTree(ctx.getIOManager(), file, diskBufferCache, diskFileMapProvider, + typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType, + bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, + ioOpCallbackFactory.createIoOpCallback(), linearizeCmpFactory, btreeFields, version, durable, + isPointMBR, (IMetadataPageManagerFactory) opDesc.getPageManagerFactory()); } public int getTargetVersion() { http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java index 63633aa..d580756 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java @@ -30,7 +30,6 @@ import org.apache.hyracks.api.io.FileReference; import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory; import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory; import org.apache.hyracks.storage.am.common.api.ITreeIndex; -import org.apache.hyracks.storage.am.common.api.TreeIndexException; import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; @@ -84,16 +83,11 @@ public class LSMRTreeDataflowHelper extends AbstractLSMRTreeDataflowHelper { RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) throws HyracksDataException { - try { - return LSMRTreeUtils.createLSMTree(ctx.getIOManager(), virtualBufferCaches, file, diskBufferCache, - diskFileMapProvider, - typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType, - bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, - ioOpCallbackFactory.createIoOpCallback(), linearizeCmpFactory, rtreeFields, btreeFields, - filterTypeTraits, filterCmpFactories, filterFields, durable, isPointMBR, - (IMetadataPageManagerFactory) opDesc.getPageManagerFactory()); - } catch (TreeIndexException e) { - throw new HyracksDataException(e); - } + return LSMRTreeUtils.createLSMTree(ctx.getIOManager(), virtualBufferCaches, file, diskBufferCache, + diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, + rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, + ioOpCallbackFactory.createIoOpCallback(), linearizeCmpFactory, rtreeFields, btreeFields, + filterTypeTraits, filterCmpFactories, filterFields, durable, isPointMBR, + (IMetadataPageManagerFactory) opDesc.getPageManagerFactory()); } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java index 6e0ffaf..81fdbb8 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java @@ -30,7 +30,6 @@ import org.apache.hyracks.api.io.FileReference; import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory; import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory; import org.apache.hyracks.storage.am.common.api.ITreeIndex; -import org.apache.hyracks.storage.am.common.api.TreeIndexException; import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; @@ -66,15 +65,11 @@ public class LSMRTreeWithAntiMatterTuplesDataflowHelper extends AbstractLSMRTree RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) throws HyracksDataException { - try { - return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(ctx.getIOManager(), virtualBufferCaches, file, - diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, - valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory - .createIoOpCallback(), linearizeCmpFactory, rtreeFields, filterTypeTraits, - filterCmpFactories, filterFields, durable, isPointMBR, (IMetadataPageManagerFactory) opDesc - .getPageManagerFactory()); - } catch (TreeIndexException e) { - throw new HyracksDataException(e); - } + return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(ctx.getIOManager(), virtualBufferCaches, file, + diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, + valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, + ioOpCallbackFactory.createIoOpCallback(), linearizeCmpFactory, rtreeFields, filterTypeTraits, + filterCmpFactories, filterFields, durable, isPointMBR, + (IMetadataPageManagerFactory) opDesc.getPageManagerFactory()); } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java index 480e1e2..646dbef 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java @@ -23,6 +23,7 @@ import java.util.List; import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory; +import org.apache.hyracks.api.exceptions.ErrorCode; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.api.io.FileReference; import org.apache.hyracks.api.io.IIOManager; @@ -36,16 +37,14 @@ import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback; import org.apache.hyracks.storage.am.common.api.ISearchPredicate; import org.apache.hyracks.storage.am.common.api.ITreeIndex; import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory; -import org.apache.hyracks.storage.am.common.api.IndexException; -import org.apache.hyracks.storage.am.common.exceptions.TreeIndexDuplicateKeyException; import org.apache.hyracks.storage.am.common.impls.AbstractSearchPredicate; 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.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.ILSMComponentFilterFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory; +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.ILSMIOOperationCallback; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor; @@ -184,8 +183,7 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree 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(); @@ -258,7 +256,7 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree @Override public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred) - throws HyracksDataException, IndexException { + throws HyracksDataException { LSMRTreeOpContext ctx = (LSMRTreeOpContext) ictx; cursor.open(ctx.searchInitialState, pred); } @@ -277,7 +275,7 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree protected LSMRTreeDiskComponent createDiskComponent(ILSMDiskComponentFactory factory, FileReference insertFileRef, FileReference deleteFileRef, FileReference bloomFilterFileRef, boolean createComponent) - throws HyracksDataException, IndexException { + throws HyracksDataException { // Create new tree instance. LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) factory .createComponent(new LSMComponentFileReferences(insertFileRef, deleteFileRef, bloomFilterFileRef)); @@ -343,8 +341,7 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree } @Override - public void modify(IIndexOperationContext ictx, ITupleReference tuple) - throws HyracksDataException, IndexException { + public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException { LSMRTreeOpContext ctx = (LSMRTreeOpContext) ictx; if (ctx.getOperation() == IndexOperation.PHYSICALDELETE) { throw new UnsupportedOperationException("Physical delete not supported in the LSM-RTree"); @@ -368,9 +365,12 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree // Insert key into the deleted-keys BTree. try { ctx.currentMutableBTreeAccessor.insert(indexTuple); - } catch (TreeIndexDuplicateKeyException e) { - // Do nothing, because one delete tuple is enough to indicate - // that all the corresponding insert tuples are deleted + } catch (HyracksDataException e) { + if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) { + // Do nothing, because one delete tuple is enough to indicate + // that all the corresponding insert tuples are deleted + throw e; + } } } if (ctx.filterTuple != null) { http://git-wip-us.apache.org/repos/asf/asterixdb/blob/3b3319a9/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java index f846c6c..dce7102 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory; +import org.apache.hyracks.api.exceptions.ErrorCode; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.api.io.IIOManager; import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference; @@ -42,12 +43,10 @@ import org.apache.hyracks.storage.am.common.api.ISearchPredicate; import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor; import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory; import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader; -import org.apache.hyracks.storage.am.common.api.IndexException; -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.common.api.ILSMDiskComponent; import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent; +import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent; 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; @@ -198,21 +197,13 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { if (diskComponents.size() == 0 && secondDiskComponents.size() == 0) { //First time activation List validFileReferences; - try { - validFileReferences = fileManager.cleanupAndGetValidFiles(); - } catch (IndexException e) { - throw new HyracksDataException(e); - } + validFileReferences = fileManager.cleanupAndGetValidFiles(); for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) { LSMRTreeDiskComponent component; - try { - component = createDiskComponent(componentFactory, - lsmComonentFileReference.getInsertIndexFileReference(), - lsmComonentFileReference.getDeleteIndexFileReference(), - lsmComonentFileReference.getBloomFilterFileReference(), false); - } catch (IndexException e) { - throw new HyracksDataException(e); - } + component = + createDiskComponent(componentFactory, lsmComonentFileReference.getInsertIndexFileReference(), + lsmComonentFileReference.getDeleteIndexFileReference(), + lsmComonentFileReference.getBloomFilterFileReference(), false); diskComponents.add(component); secondDiskComponents.add(component); } @@ -254,7 +245,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { // we override this method because this index uses a different opcontext @Override public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred) - throws HyracksDataException, IndexException { + throws HyracksDataException { ExternalRTreeOpContext ctx = (ExternalRTreeOpContext) ictx; List operationalComponents = ictx.getComponentHolder(); ctx.initialState.setOperationalComponents(operationalComponents); @@ -267,7 +258,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { // This can be done in a better way by creating a method boolean // keepDeletedTuples(mergedComponents); @Override - public ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException { + public ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException { LSMRTreeMergeOperation mergeOp = (LSMRTreeMergeOperation) operation; ITreeIndexCursor cursor = mergeOp.getCursor(); ISearchPredicate rtreeSearchPred = new SearchPredicate(null, null); @@ -430,8 +421,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { // Not supported @Override - public void modify(IIndexOperationContext ictx, ITupleReference tuple) - throws HyracksDataException, IndexException { + public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException { throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-RTree"); } @@ -444,7 +434,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { // Not supported @Override - public ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException { + public ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException { throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-RTree"); } @@ -492,23 +482,15 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { // For initial load @Override public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint, - boolean checkIfEmptyIndex) throws TreeIndexException { - try { - return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, 0, checkIfEmptyIndex, false); - } catch (HyracksDataException e) { - throw new TreeIndexException(e); - } + boolean checkIfEmptyIndex) throws HyracksDataException { + return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, 0, checkIfEmptyIndex, false); } // For transaction bulk load <- could consolidate with the above method -> @Override public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint, - boolean checkIfEmptyIndex) throws TreeIndexException { - try { - return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex, true); - } catch (HyracksDataException e) { - throw new TreeIndexException(e); - } + boolean checkIfEmptyIndex) throws HyracksDataException { + return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex, true); } // The bulk loader used for both initial loading and transaction @@ -524,24 +506,16 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { private final boolean isTransaction; public LSMTwoPCRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint, - boolean checkIfEmptyIndex, boolean isTransaction) throws TreeIndexException, HyracksDataException { + boolean checkIfEmptyIndex, boolean isTransaction) throws HyracksDataException { this.isTransaction = isTransaction; // Create the appropriate target if (isTransaction) { - try { - component = createTransactionTarget(); - } catch (HyracksDataException | IndexException e) { - throw new TreeIndexException(e); - } + component = createTransactionTarget(); } else { if (checkIfEmptyIndex && !isEmptyIndex()) { - throw new TreeIndexException("Cannot load an index that is not empty"); - } - try { - component = createBulkLoadTarget(); - } catch (HyracksDataException | IndexException e) { - throw new TreeIndexException(e); + throw HyracksDataException.create(ErrorCode.LOAD_NON_EMPTY_INDEX); } + component = createBulkLoadTarget(); } // Create the three loaders @@ -557,10 +531,10 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { } @Override - public void add(ITupleReference tuple) throws IndexException, HyracksDataException { + public void add(ITupleReference tuple) throws HyracksDataException { try { rtreeBulkLoader.add(tuple); - } catch (IndexException | HyracksDataException | RuntimeException e) { + } catch (Exception e) { cleanupArtifacts(); throw e; } @@ -596,7 +570,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { } @Override - public void end() throws HyracksDataException, IndexException { + public void end() throws HyracksDataException { if (!cleanedUpArtifacts) { if (!endedBloomFilterLoad) { builder.end(); @@ -623,11 +597,11 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { } @Override - public void delete(ITupleReference tuple) throws IndexException, HyracksDataException { + public void delete(ITupleReference tuple) throws HyracksDataException { try { btreeBulkLoader.add(tuple); builder.add(tuple); - } catch (IndexException | HyracksDataException | RuntimeException e) { + } catch (Exception e) { cleanupArtifacts(); throw e; } @@ -647,12 +621,12 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { // 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 { + private ILSMDiskComponent createTransactionTarget() throws HyracksDataException { LSMComponentFileReferences componentFileRefs; try { componentFileRefs = fileManager.getNewTransactionFileReference(); } catch (IOException e) { - throw new HyracksDataException("Failed to create transaction components", e); + throw HyracksDataException.create(e); } return createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), @@ -669,7 +643,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { // opCtx. first line <- in schedule merge, we-> @Override public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) - throws HyracksDataException, IndexException { + throws HyracksDataException { ILSMIndexOperationContext rctx = createOpContext(NoOpOperationCallback.INSTANCE, -1); rctx.setOperation(IndexOperation.MERGE); List mergingComponents = ctx.getComponentHolder(); @@ -732,7 +706,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { } @Override - public void commitTransaction() throws TreeIndexException, HyracksDataException, IndexException { + public void commitTransaction() throws HyracksDataException { LSMComponentFileReferences componentFileRefrences = fileManager.getTransactionFileReferenceForCommit(); LSMRTreeDiskComponent component = null; if (componentFileRefrences != null) { @@ -744,21 +718,13 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex { } @Override - public void abortTransaction() throws TreeIndexException { - try { - fileManager.deleteTransactionFiles(); - } catch (HyracksDataException e) { - throw new TreeIndexException(e); - } + public void abortTransaction() throws HyracksDataException { + fileManager.deleteTransactionFiles(); } @Override - public void recoverTransaction() throws TreeIndexException { - try { - fileManager.recoverTransaction(); - } catch (HyracksDataException e) { - throw new TreeIndexException(e); - } + public void recoverTransaction() throws HyracksDataException { + fileManager.recoverTransaction(); } @Override