From commits-return-5042-archive-asf-public=cust-asf.ponee.io@asterixdb.apache.org Tue Feb 13 01:09:55 2018 Return-Path: X-Original-To: archive-asf-public@eu.ponee.io Delivered-To: archive-asf-public@eu.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by mx-eu-01.ponee.io (Postfix) with ESMTP id C0AAE180652 for ; Tue, 13 Feb 2018 01:09:54 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id AF752160C3F; Tue, 13 Feb 2018 00:09:54 +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 15DF0160C66 for ; Tue, 13 Feb 2018 01:09:51 +0100 (CET) Received: (qmail 19290 invoked by uid 500); 13 Feb 2018 00:09:51 -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 19156 invoked by uid 99); 13 Feb 2018 00:09:51 -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; Tue, 13 Feb 2018 00:09:51 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 6BB9EE9640; Tue, 13 Feb 2018 00:09:50 +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: Tue, 13 Feb 2018 00:09:55 -0000 Message-Id: In-Reply-To: <4186517b3d434a10b816dcc7e9637c08@git.apache.org> References: <4186517b3d434a10b816dcc7e9637c08@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [6/7] asterixdb git commit: [ASTERIXDB-2204][STO] Fix implementations and usages of IIndexCursor http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java deleted file mode 100644 index 4d0f287..0000000 --- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java +++ /dev/null @@ -1,263 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.hyracks.tests.unit; - -import org.apache.hyracks.storage.common.ICursorInitialState; -import org.apache.hyracks.storage.common.IIndexCursor; -import org.apache.hyracks.storage.common.ISearchPredicate; -import org.junit.Assert; -import org.junit.Test; - -import java.util.List; - -/** - * This is a test class that forms the basis for unit tests of different implementations of the IIndexCursor interface - */ -public abstract class IIndexCursorTest { - @Test - public void testNormalLifeCycle() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - for (ISearchPredicate predicate : predicates) { - cursor.open(initialState, predicate); - while (cursor.hasNext()) { - cursor.next(); - } - cursor.close(); - } - cursor.destroy(); - } - - @Test - public void testCreateDestroySucceed() throws Exception { - IIndexCursor cursor = createCursor(); - cursor.destroy(); - } - - @Test - public void testDoubleOpenFails() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - cursor.open(initialState, predicates.get(0)); - boolean expectedExceptionThrown = false; - try { - cursor.open(initialState, predicates.get(0)); - } catch (Exception e) { - expectedExceptionThrown = true; - } - cursor.close(); - cursor.destroy(); - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testCloseWithoutOpenFails() throws Exception { - IIndexCursor cursor = createCursor(); - boolean expectedExceptionThrown = false; - try { - cursor.close(); - } catch (Exception e) { - expectedExceptionThrown = true; - } - cursor.destroy(); - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testDoubleCloseFails() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - cursor.open(initialState, predicates.get(0)); - cursor.close(); - boolean expectedExceptionThrown = false; - try { - cursor.close(); - } catch (Exception e) { - expectedExceptionThrown = true; - } - cursor.destroy(); - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testHasNextBeforeOpenFails() throws Exception { - IIndexCursor cursor = createCursor(); - boolean expectedExceptionThrown = false; - try { - cursor.hasNext(); - } catch (Exception e) { - expectedExceptionThrown = true; - } - cursor.destroy(); - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testHasNextAfterCloseFails() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - cursor.open(initialState, predicates.get(0)); - cursor.close(); - boolean expectedExceptionThrown = false; - try { - cursor.hasNext(); - } catch (Exception e) { - expectedExceptionThrown = true; - } - cursor.destroy(); - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testNextBeforeOpenFails() throws Exception { - IIndexCursor cursor = createCursor(); - boolean expectedExceptionThrown = false; - try { - cursor.next(); - } catch (Exception e) { - expectedExceptionThrown = true; - } - cursor.destroy(); - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testNextAfterCloseFails() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - cursor.open(initialState, predicates.get(0)); - cursor.close(); - boolean expectedExceptionThrown = false; - try { - cursor.next(); - } catch (Exception e) { - expectedExceptionThrown = true; - } - cursor.destroy(); - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testDestroyWhileOpenFails() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - cursor.open(initialState, predicates.get(0)); - boolean expectedExceptionThrown = false; - try { - cursor.destroy(); - } catch (Exception e) { - expectedExceptionThrown = true; - } - cursor.close(); - cursor.destroy(); - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testOpenAfterDestroyFails() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - cursor.open(initialState, predicates.get(0)); - cursor.close(); - cursor.destroy(); - boolean expectedExceptionThrown = false; - try { - cursor.open(initialState, predicates.get(0)); - } catch (Exception e) { - expectedExceptionThrown = true; - } - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testCloseAfterDestroyFails() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - cursor.open(initialState, predicates.get(0)); - cursor.close(); - cursor.destroy(); - boolean expectedExceptionThrown = false; - try { - cursor.close(); - } catch (Exception e) { - expectedExceptionThrown = true; - } - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testNextAfterDestroyFails() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - cursor.open(initialState, predicates.get(0)); - cursor.close(); - cursor.destroy(); - boolean expectedExceptionThrown = false; - try { - cursor.next(); - } catch (Exception e) { - expectedExceptionThrown = true; - } - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testHasNextAfterDestroyFails() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - cursor.open(initialState, predicates.get(0)); - cursor.close(); - cursor.destroy(); - boolean expectedExceptionThrown = false; - try { - cursor.hasNext(); - } catch (Exception e) { - expectedExceptionThrown = true; - } - Assert.assertTrue(expectedExceptionThrown); - } - - @Test - public void testGetTupleReturnsNullAfterDestroy() throws Exception { - IIndexCursor cursor = createCursor(); - ICursorInitialState initialState = createCursorInitialState(); - List predicates = createSearchPredicates(); - cursor.open(initialState, predicates.get(0)); - cursor.close(); - cursor.destroy(); - Assert.assertNull(cursor.getTuple()); - } - - protected abstract List createSearchPredicates(); - - protected abstract ICursorInitialState createCursorInitialState(); - - protected abstract IIndexCursor createCursor(); -} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java index 3c045cb..a296672 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java @@ -29,10 +29,10 @@ import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame; import org.apache.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor; import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory; import org.apache.hyracks.storage.am.common.api.ITreeIndex; -import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor; import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame; import org.apache.hyracks.storage.am.common.api.ITupleUpdater; import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory; +import org.apache.hyracks.storage.common.IIndexCursor; public class BTreeUpdateSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable { private final ITupleUpdater tupleUpdater; @@ -49,7 +49,7 @@ public class BTreeUpdateSearchOperatorNodePushable extends BTreeSearchOperatorNo } @Override - protected ITreeIndexCursor createCursor() { + protected IIndexCursor createCursor() { ITreeIndex treeIndex = (ITreeIndex) index; ITreeIndexFrame cursorFrame = treeIndex.getLeafFrameFactory().createFrame(); return new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java index 6e2d694..077a006 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java @@ -841,6 +841,7 @@ public class BTree extends AbstractTreeIndex { public class BTreeAccessor implements ITreeIndexAccessor { protected BTree btree; protected BTreeOpContext ctx; + private boolean destroyed = false; public BTreeAccessor(BTree btree, IModificationOperationCallback modificationCalback, ISearchOperationCallback searchCallback) { @@ -995,6 +996,15 @@ public class BTree extends AbstractTreeIndex { throw HyracksDataException.create(ErrorCode.OPERATION_EXCEEDED_MAX_RESTARTS, MAX_RESTARTS); } } + + @Override + public void destroy() throws HyracksDataException { + if (destroyed) { + return; + } + destroyed = true; + ctx.destroy(); + } } @Override http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java index 10f79a2..f7e0ce0 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java @@ -29,6 +29,7 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor; import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference; import org.apache.hyracks.storage.am.common.ophelpers.FindTupleMode; import org.apache.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy; +import org.apache.hyracks.storage.common.EnforcedIndexCursor; import org.apache.hyracks.storage.common.ICursorInitialState; import org.apache.hyracks.storage.common.ISearchPredicate; import org.apache.hyracks.storage.common.MultiComparator; @@ -36,7 +37,7 @@ import org.apache.hyracks.storage.common.buffercache.IBufferCache; import org.apache.hyracks.storage.common.buffercache.ICachedPage; import org.apache.hyracks.storage.common.file.BufferedFileHandle; -public class BTreeCountingSearchCursor implements ITreeIndexCursor { +public class BTreeCountingSearchCursor extends EnforcedIndexCursor implements ITreeIndexCursor { private int fileId = -1; private ICachedPage page = null; @@ -75,11 +76,7 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor { } @Override - public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { - // in case open is called multiple times without closing - if (page != null) { - releasePage(); - } + public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { page = ((BTreeCursorInitialState) initialState).getPage(); isPageDirty = false; @@ -169,7 +166,7 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor { } @Override - public boolean hasNext() throws HyracksDataException { + public boolean doHasNext() throws HyracksDataException { // get the count for the current page // follow the sibling pointer until last page // if no more tuples on a page, then done @@ -199,7 +196,7 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor { } @Override - public void next() throws HyracksDataException { + public void doNext() throws HyracksDataException { // Do nothing. Count is performed just once! IntegerPointable.setInteger(countBuf, 0, count); tupleBuilder.addField(countBuf, 0, 4); @@ -207,7 +204,7 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor { } @Override - public void destroy() throws HyracksDataException { + public void doDestroy() throws HyracksDataException { if (page != null) { releasePage(); } @@ -220,16 +217,12 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor { } @Override - public void close() { - try { - destroy(); - } catch (Exception e) { - e.printStackTrace(); - } + public void doClose() throws HyracksDataException { + doDestroy(); } @Override - public ITupleReference getTuple() { + public ITupleReference doGetTuple() { return countTuple; } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java index a46ae9a..60fa145 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java @@ -24,6 +24,7 @@ import java.util.Deque; import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.api.util.DestroyUtils; 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; @@ -73,6 +74,7 @@ public class BTreeOpContext implements IIndexOperationContext, IExtraPageBlockHe private ISearchOperationCallback searchCallback; private ITupleAcceptor acceptor; private int smoCount; + private boolean destroyed = false; // Debug private final Deque validationInfos; @@ -383,4 +385,16 @@ public class BTreeOpContext implements IIndexOperationContext, IExtraPageBlockHe public void resetNonIndexFieldsTuple(ITupleReference newValue) { tupleWithNonIndexFields.reset(newValue); } + + @Override + public void destroy() throws HyracksDataException { + if (destroyed) { + return; + } + destroyed = true; + Throwable failure = DestroyUtils.destroy(null, accessor, cursor); + if (failure != null) { + throw HyracksDataException.create(failure); + } + } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java index e903180..bff1bcb 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java @@ -29,6 +29,7 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor; import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference; import org.apache.hyracks.storage.am.common.ophelpers.FindTupleMode; import org.apache.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy; +import org.apache.hyracks.storage.common.EnforcedIndexCursor; import org.apache.hyracks.storage.common.ICursorInitialState; import org.apache.hyracks.storage.common.IIndexAccessor; import org.apache.hyracks.storage.common.ISearchOperationCallback; @@ -38,7 +39,7 @@ import org.apache.hyracks.storage.common.buffercache.IBufferCache; import org.apache.hyracks.storage.common.buffercache.ICachedPage; import org.apache.hyracks.storage.common.file.BufferedFileHandle; -public class BTreeRangeSearchCursor implements ITreeIndexCursor { +public class BTreeRangeSearchCursor extends EnforcedIndexCursor implements ITreeIndexCursor { protected final IBTreeLeafFrame frame; protected final ITreeIndexTupleReference frameTuple; @@ -81,19 +82,12 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor { } @Override - public void destroy() throws HyracksDataException { - if (page != null) { - releasePage(); - } - - tupleIndex = 0; - page = null; - isPageDirty = false; - pred = null; + public void doDestroy() throws HyracksDataException { + // No Op all resources are released in the close call } @Override - public ITupleReference getTuple() { + public ITupleReference doGetTuple() { return frameTuple; } @@ -118,7 +112,7 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor { } @Override - public boolean hasNext() throws HyracksDataException { + public boolean doHasNext() throws HyracksDataException { int nextLeafPage; if (tupleIndex >= frame.getTupleCount()) { nextLeafPage = frame.getNextLeaf(); @@ -159,8 +153,10 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor { // retraverse the index looking for the reconciled key reusablePredicate.setLowKey(reconciliationTuple, true); + // before re-using the cursor, we must close it + close(); + // this search call will re-open the cursor accessor.search(this, reusablePredicate); - if (stopTupleIndex < 0 || tupleIndex > stopTupleIndex) { return false; } @@ -177,7 +173,7 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor { } @Override - public void next() throws HyracksDataException { + public void doNext() throws HyracksDataException { tupleIndex++; } @@ -216,7 +212,7 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor { } @Override - public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { + public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { // in case open is called multiple times without closing if (page != null) { resetBeforeOpen(); @@ -262,8 +258,15 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor { } @Override - public void close() throws HyracksDataException { - destroy(); + public void doClose() throws HyracksDataException { + if (page != null) { + releasePage(); + } + + tupleIndex = 0; + page = null; + isPageDirty = false; + pred = null; } @Override http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java index 32fa9df..eee43b5 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java @@ -275,7 +275,10 @@ public class DiskBTree extends BTree { @Override protected void releasePage() throws HyracksDataException { - bufferCache.unpin(page); + if (page != null) { + bufferCache.unpin(page); + page = null; + } } @Override http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java index 5839d0e..7814e60 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java @@ -35,17 +35,17 @@ public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor { } @Override - public boolean hasNext() throws HyracksDataException { + public boolean doHasNext() throws HyracksDataException { return tupleIndex >= 0 && !nextHasBeenCalled; } @Override - public void next() throws HyracksDataException { + public void doNext() throws HyracksDataException { nextHasBeenCalled = true; } @Override - public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { + public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { // in case open is called multiple times without closing if (page != null) { resetBeforeOpen(); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreeRangeSearchCursor.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreeRangeSearchCursor.java index 7d4ee0d..0e82088 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreeRangeSearchCursor.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreeRangeSearchCursor.java @@ -37,7 +37,7 @@ public class DiskBTreeRangeSearchCursor extends BTreeRangeSearchCursor { } @Override - public boolean hasNext() throws HyracksDataException { + public boolean doHasNext() throws HyracksDataException { int nextLeafPage; if (tupleIndex >= frame.getTupleCount()) { nextLeafPage = frame.getNextLeaf(); @@ -99,8 +99,8 @@ public class DiskBTreeRangeSearchCursor extends BTreeRangeSearchCursor { } @Override - public void close() throws HyracksDataException { - super.close(); + public void doClose() throws HyracksDataException { + super.doClose(); searchPages.clear(); } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java index 4ff0656..12dc310 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java @@ -67,10 +67,12 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; @RunWith(PowerMockRunner.class) +@PowerMockIgnore("javax.management.*") @PrepareForTest({ BTreeUtils.class, FrameTupleAccessor.class, ArrayTupleBuilder.class, IndexSearchOperatorNodePushable.class, FrameUtils.class, FrameTupleAppender.class }) public class FramewriterTest { http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml index 94fe2c2..5f05ce9 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml @@ -16,18 +16,16 @@ ! specific language governing permissions and limitations ! under the License. !--> - - + 4.0.0 hyracks-storage-am-common hyracks-storage-am-common - org.apache.hyracks hyracks 0.3.4-SNAPSHOT - Apache License, Version 2.0 @@ -36,11 +34,24 @@ A business-friendly OSS license - ${basedir}/../.. - + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + org.apache.hyracks @@ -94,5 +105,10 @@ org.apache.logging.log4j log4j-api + + org.mockito + mockito-all + test + - + \ No newline at end of file http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexOperationContext.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexOperationContext.java index a8bdf32..01c98f1 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexOperationContext.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexOperationContext.java @@ -18,13 +18,26 @@ */ package org.apache.hyracks.storage.am.common.api; +import org.apache.hyracks.api.dataflow.IDestroyable; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation; -public interface IIndexOperationContext { +public interface IIndexOperationContext extends IDestroyable { + /** + * Set the next intended operation for this context + * + * @param newOp + * @throws HyracksDataException + */ void setOperation(IndexOperation newOp) throws HyracksDataException; + /** + * @return the intended operation + */ IndexOperation getOperation(); + /** + * Release resources associated with the context and prepare it for the next use + */ void reset(); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexAccessor.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexAccessor.java index 90963bf..0d7fd5f 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexAccessor.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexAccessor.java @@ -39,6 +39,8 @@ public interface ITreeIndexAccessor extends IIndexAccessor { /** * Open the given cursor for a disk-order scan, positioning the cursor to * the first leaf tuple. + * If this method returns successfully, the cursor is open. + * Otherwise, it was not open * * @param icursor * Cursor to be opened for disk-order scanning. http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java index aae830d..8490c6a 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java @@ -21,7 +21,6 @@ package org.apache.hyracks.storage.am.common.dataflow; import static org.apache.hyracks.api.exceptions.ErrorCode.CANNOT_DROP_IN_USE_INDEX; import static org.apache.hyracks.api.exceptions.ErrorCode.INDEX_DOES_NOT_EXIST; -import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption; import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption.IF_EXISTS; import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption.WAIT_ON_IN_USE; @@ -34,6 +33,7 @@ import org.apache.hyracks.api.dataflow.value.RecordDescriptor; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.dataflow.std.base.AbstractOperatorNodePushable; import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper; +import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -109,7 +109,6 @@ public class IndexDropOperatorNodePushable extends AbstractOperatorNodePushable maxWaitTimeMillis -= DROP_ATTEMPT_WAIT_TIME_MILLIS; return true; } catch (InterruptedException e1) { - Thread.currentThread().interrupt(); throw HyracksDataException.create(e1); } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java index d55962a..41fdc41 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java @@ -29,6 +29,8 @@ import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory; import org.apache.hyracks.api.dataflow.value.RecordDescriptor; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.api.job.profiling.IOperatorStats; +import org.apache.hyracks.api.util.DestroyUtils; +import org.apache.hyracks.api.util.ExceptionUtils; import org.apache.hyracks.control.common.job.profiling.OperatorStats; import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder; import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor; @@ -43,6 +45,7 @@ import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory; import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters; import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback; import org.apache.hyracks.storage.am.common.tuples.PermutingFrameTupleReference; +import org.apache.hyracks.storage.am.common.util.ResourceReleaseUtils; import org.apache.hyracks.storage.common.IIndex; import org.apache.hyracks.storage.common.IIndexAccessParameters; import org.apache.hyracks.storage.common.IIndexAccessor; @@ -218,7 +221,15 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput @Override public void close() throws HyracksDataException { - HyracksDataException closeException = null; + Throwable failure = releaseResources(); + failure = ResourceReleaseUtils.close(writer, failure); + if (failure != null) { + throw HyracksDataException.create(failure); + } + } + + private Throwable releaseResources() { + Throwable failure = null; if (index != null) { // if index == null, then the index open was not successful if (!failed) { @@ -226,44 +237,24 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput if (appender.getTupleCount() > 0) { appender.write(writer, true); } - } catch (Throwable th) { - writer.fail(); - closeException = HyracksDataException.create(th); - } - } - - try { - cursor.destroy(); - } catch (Throwable th) { - if (closeException == null) { - closeException = HyracksDataException.create(th); - } else { - closeException.addSuppressed(th); + } catch (Throwable th) { // NOSONAR Must ensure writer.fail is called. + // subsequently, the failure will be thrown + failure = th; } - } - try { - indexHelper.close(); - } catch (Throwable th) { - if (closeException == null) { - closeException = new HyracksDataException(th); - } else { - closeException.addSuppressed(th); + if (failure != null) { + try { + writer.fail(); + } catch (Throwable th) {// NOSONAR Must cursor.close is called. + // subsequently, the failure will be thrown + failure = ExceptionUtils.suppress(failure, th); + } } } + failure = ResourceReleaseUtils.close(cursor, failure); + failure = DestroyUtils.destroy(failure, cursor, indexAccessor); + failure = ResourceReleaseUtils.close(indexHelper, failure); } - try { - // will definitely be called regardless of exceptions - writer.close(); - } catch (Throwable th) { - if (closeException == null) { - closeException = new HyracksDataException(th); - } else { - closeException.addSuppressed(th); - } - } - if (closeException != null) { - throw closeException; - } + return failure; } @Override http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java index 7626815..90b50c6 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java @@ -19,10 +19,12 @@ package org.apache.hyracks.storage.am.common.dataflow; import java.io.DataOutput; +import java.io.IOException; import org.apache.hyracks.api.comm.VSizeFrame; import org.apache.hyracks.api.context.IHyracksTaskContext; import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.api.util.ExceptionUtils; import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder; import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender; import org.apache.hyracks.dataflow.common.comm.util.FrameUtils; @@ -37,6 +39,7 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame; import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters; import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback; import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor; +import org.apache.hyracks.storage.am.common.util.ResourceReleaseUtils; import org.apache.hyracks.storage.common.IIndexAccessParameters; import org.apache.hyracks.storage.common.ISearchOperationCallback; import org.apache.hyracks.storage.common.LocalResource; @@ -56,55 +59,69 @@ public class TreeIndexDiskOrderScanOperatorNodePushable extends AbstractUnaryOut @Override public void initialize() throws HyracksDataException { + Throwable failure = null; treeIndexHelper.open(); - ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndexInstance(); try { - ITreeIndexFrame cursorFrame = treeIndex.getLeafFrameFactory().createFrame(); - ITreeIndexCursor cursor = new TreeIndexDiskOrderScanCursor(cursorFrame); - LocalResource resource = treeIndexHelper.getResource(); - ISearchOperationCallback searchCallback = - searchCallbackFactory.createSearchOperationCallback(resource.getId(), ctx, null); - IIndexAccessParameters iap = new IndexAccessParameters(NoOpOperationCallback.INSTANCE, searchCallback); - ITreeIndexAccessor indexAccessor = (ITreeIndexAccessor) treeIndex.createAccessor(iap); + writer.open(); + FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx)); + scan(appender); + appender.write(writer, true); + } catch (Throwable th) { // NOSONAR: Must call writer.fail + failure = th; try { - writer.open(); - indexAccessor.diskOrderScan(cursor); - int fieldCount = treeIndex.getFieldCount(); - FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx)); - ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount); - DataOutput dos = tb.getDataOutput(); + writer.fail(); + } catch (Throwable failFailure) {// NOSONAR: Must maintain all stacks + failure = ExceptionUtils.suppress(failure, failFailure); + } + } finally { + failure = ResourceReleaseUtils.close(writer, failure); + } + if (failure != null) { + throw HyracksDataException.create(failure); + } + } + + private void scan(FrameTupleAppender appender) throws IOException { + ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndexInstance(); + LocalResource resource = treeIndexHelper.getResource(); + ISearchOperationCallback searchCallback = + searchCallbackFactory.createSearchOperationCallback(resource.getId(), ctx, null); + IIndexAccessParameters iap = new IndexAccessParameters(NoOpOperationCallback.INSTANCE, searchCallback); + ITreeIndexAccessor indexAccessor = (ITreeIndexAccessor) treeIndex.createAccessor(iap); + try { + doScan(treeIndex, indexAccessor, appender); + } finally { + indexAccessor.destroy(); + } + } + private void doScan(ITreeIndex treeIndex, ITreeIndexAccessor indexAccessor, FrameTupleAppender appender) + throws IOException { + int fieldCount = treeIndex.getFieldCount(); + ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount); + DataOutput dos = tb.getDataOutput(); + ITreeIndexFrame cursorFrame = treeIndex.getLeafFrameFactory().createFrame(); + ITreeIndexCursor cursor = new TreeIndexDiskOrderScanCursor(cursorFrame); + try { + indexAccessor.diskOrderScan(cursor); + try { while (cursor.hasNext()) { tb.reset(); cursor.next(); - ITupleReference frameTuple = cursor.getTuple(); for (int i = 0; i < frameTuple.getFieldCount(); i++) { dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i)); tb.addFieldEndOffset(); } - FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize()); - } - appender.write(writer, true); - } catch (Throwable th) { - writer.fail(); - throw new HyracksDataException(th); } finally { - try { - cursor.destroy(); - } catch (Exception cursorCloseException) { - throw new IllegalStateException(cursorCloseException); - } finally { - writer.close(); - } + cursor.close(); } - } catch (Throwable th) { - treeIndexHelper.close(); - throw new HyracksDataException(th); + } finally { + cursor.destroy(); } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java index b2c26db..36fba76 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java @@ -20,17 +20,17 @@ package org.apache.hyracks.storage.am.common.impls; import org.apache.hyracks.api.exceptions.HyracksDataException; -import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference; import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor; import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame; import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference; +import org.apache.hyracks.storage.common.EnforcedIndexCursor; import org.apache.hyracks.storage.common.ICursorInitialState; import org.apache.hyracks.storage.common.ISearchPredicate; import org.apache.hyracks.storage.common.buffercache.IBufferCache; import org.apache.hyracks.storage.common.buffercache.ICachedPage; import org.apache.hyracks.storage.common.file.BufferedFileHandle; -public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor { +public class TreeIndexDiskOrderScanCursor extends EnforcedIndexCursor implements ITreeIndexCursor { protected int tupleIndex = 0; protected int fileId = -1; @@ -48,13 +48,15 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor { } @Override - public void destroy() throws HyracksDataException { + public void doDestroy() throws HyracksDataException { + tupleIndex = 0; + currentPageId = -1; + maxPageId = -1; releasePage(); - page = null; } @Override - public ITreeIndexTupleReference getTuple() { + public ITreeIndexTupleReference doGetTuple() { return frameTuple; } @@ -65,7 +67,6 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor { } releasePage(); - ICachedPage nextPage = acquireNextPage(); page = nextPage; frame.setPage(page); @@ -80,7 +81,7 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor { } @Override - public boolean hasNext() throws HyracksDataException { + public boolean doHasNext() throws HyracksDataException { if (currentPageId > maxPageId) { return false; } @@ -98,16 +99,12 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor { } @Override - public void next() throws HyracksDataException { + public void doNext() throws HyracksDataException { tupleIndex++; } @Override - public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { - // in case open is called multiple times without closing - if (page != null) { - releasePage(); - } + public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { page = initialState.getPage(); tupleIndex = 0; frame.setPage(page); @@ -115,11 +112,11 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor { } @Override - public void close() { + public void doClose() throws HyracksDataException { tupleIndex = 0; currentPageId = -1; maxPageId = -1; - page = null; + releasePage(); } @Override @@ -146,8 +143,11 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor { } protected void releasePage() throws HyracksDataException { - page.releaseReadLatch(); - bufferCache.unpin(page); + if (page != null) { + page.releaseReadLatch(); + bufferCache.unpin(page); + page = null; + } } protected ICachedPage acquireNextPage() throws HyracksDataException { http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/ResourceReleaseUtils.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/ResourceReleaseUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/ResourceReleaseUtils.java new file mode 100644 index 0000000..50b6e59 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/ResourceReleaseUtils.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.hyracks.storage.am.common.util; + +import java.util.List; + +import org.apache.hyracks.api.comm.IFrameWriter; +import org.apache.hyracks.api.util.ExceptionUtils; +import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper; +import org.apache.hyracks.storage.common.IIndexCursor; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +public class ResourceReleaseUtils { + + private static final Logger LOGGER = LogManager.getLogger(); + + private ResourceReleaseUtils() { + } + + /** + * Close the cursor and suppress any Throwable thrown by the close call. + * This method must NEVER throw any Throwable + * + * @param cursor + * the cursor to close + * @param root + * the first exception encountered during release of resources + * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null + */ + public static Throwable close(IIndexCursor cursor, Throwable root) { + if (cursor != null) { + try { + cursor.close(); + } catch (Throwable th) { // NOSONAR Will be re-thrown + try { + LOGGER.log(Level.WARN, "Failure closing a closeable resource", th); + } catch (Throwable loggingFailure) { + // Do nothing + } + root = ExceptionUtils.suppress(root, th); + } + } + return root; + } + + /** + * Close the AutoCloseable and suppress any Throwable thrown by the close call. + * This method must NEVER throw any Throwable + * + * @param closable + * the resource to close + * @param root + * the first exception encountered during release of resources + * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null + */ + public static Throwable close(AutoCloseable closable, Throwable root) { + if (closable != null) { + try { + closable.close(); + } catch (Throwable th) { // NOSONAR Will be re-thrown + try { + LOGGER.log(Level.WARN, "Failure closing a closeable resource", th); + } catch (Throwable loggingFailure) { + // Do nothing + } + root = ExceptionUtils.suppress(root, th); + } + } + return root; + } + + /** + * Close the IIndexDataflowHelper and suppress any Throwable thrown by the close call. + * This method must NEVER throw any Throwable + * + * @param indexHelper + * the indexHelper to close + * @param root + * the first exception encountered during release of resources + * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null + */ + public static Throwable close(IIndexDataflowHelper indexHelper, Throwable root) { + if (indexHelper != null) { + try { + indexHelper.close(); + } catch (Throwable th) { // NOSONAR Will be re-thrown + try { + LOGGER.log(Level.WARN, "Failure closing a closeable resource", th); + } catch (Throwable loggingFailure) { + // Do nothing + } + root = ExceptionUtils.suppress(root, th); + } + } + return root; + } + + /** + * Close the IIndexDataflowHelpers and suppress any Throwable thrown by any close call. + * This method must NEVER throw any Throwable + * + * @param indexHelpers + * the indexHelpers to close + * @param root + * the first exception encountered during release of resources + * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null + */ + public static Throwable close(List indexHelpers, Throwable root) { + for (int i = 0; i < indexHelpers.size(); i++) { + root = close(indexHelpers.get(i), root); + } + return root; + } + + /** + * Close the IFrameWriter and suppress any Throwable thrown by the close call. + * This method must NEVER throw any Throwable + * + * @param writer + * the writer to close + * @param root + * the first exception encountered during release of resources + * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null + */ + public static Throwable close(IFrameWriter writer, Throwable root) { + if (writer != null) { + try { + writer.close(); + } catch (Throwable th) { // NOSONAR Will be re-thrown + try { + LOGGER.log(Level.WARN, "Failure closing a closeable resource", th); + } catch (Throwable loggingFailure) { + // Do nothing + } + root = ExceptionUtils.suppress(root, th); + } + } + return root; + } +} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/DummyEnforcedIndexCursor.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/DummyEnforcedIndexCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/DummyEnforcedIndexCursor.java new file mode 100644 index 0000000..eec9542 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/DummyEnforcedIndexCursor.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.hyracks.storage.am.common.test; + +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference; +import org.apache.hyracks.storage.common.EnforcedIndexCursor; +import org.apache.hyracks.storage.common.ICursorInitialState; +import org.apache.hyracks.storage.common.ISearchPredicate; + +public class DummyEnforcedIndexCursor extends EnforcedIndexCursor { + @Override + protected void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { + // Dummy + } + + @Override + protected boolean doHasNext() throws HyracksDataException { + // Dummy + return false; + } + + @Override + protected void doNext() throws HyracksDataException { + // Dummy + } + + @Override + protected void doDestroy() throws HyracksDataException { + // Dummy + } + + @Override + protected void doClose() throws HyracksDataException { + // Dummy + } + + @Override + protected ITupleReference doGetTuple() { + return null; + } + +} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/EnforcedIndexCursorTest.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/EnforcedIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/EnforcedIndexCursorTest.java new file mode 100644 index 0000000..9625c2f --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/EnforcedIndexCursorTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.hyracks.storage.am.common.test; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.storage.common.EnforcedIndexCursor; +import org.apache.hyracks.storage.common.IIndexAccessor; +import org.apache.hyracks.storage.common.IIndexCursor; +import org.apache.hyracks.storage.common.ISearchPredicate; +import org.mockito.Matchers; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +public class EnforcedIndexCursorTest extends IIndexCursorTest { + @Override + protected List createSearchPredicates() { + List predicates = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + predicates.add(Mockito.mock(ISearchPredicate.class)); + } + return predicates; + } + + @Override + protected IIndexAccessor createAccessor() throws HyracksDataException { + EnforcedIndexCursor cursor = new DummyEnforcedIndexCursor(); + IIndexAccessor accessor = Mockito.mock(IIndexAccessor.class); + Mockito.when(accessor.createSearchCursor(Mockito.anyBoolean())).thenReturn(cursor); + Mockito.doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + IIndexCursor cursor = (IIndexCursor) args[0]; + ISearchPredicate pred = (ISearchPredicate) args[1]; + cursor.open(null, pred); + return null; + } + }).when(accessor).search(Matchers.any(IIndexCursor.class), Matchers.any(ISearchPredicate.class)); + return accessor; + } +} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/IIndexCursorTest.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/IIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/IIndexCursorTest.java new file mode 100644 index 0000000..6a3fdf1 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/IIndexCursorTest.java @@ -0,0 +1,294 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hyracks.storage.am.common.test; + +import java.util.List; + +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.storage.common.IIndexAccessor; +import org.apache.hyracks.storage.common.IIndexCursor; +import org.apache.hyracks.storage.common.ISearchPredicate; +import org.junit.Assert; +import org.junit.Test; + +/** + * This is a test class that forms the basis for unit tests of different implementations of the IIndexCursor interface + */ +public abstract class IIndexCursorTest { + @Test + public void testNormalLifeCycle() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + for (ISearchPredicate predicate : predicates) { + open(accessor, cursor, predicate); + while (cursor.hasNext()) { + cursor.next(); + } + cursor.close(); + } + cursor.destroy(); + destroy(accessor); + } + + protected void destroy(IIndexAccessor accessor) throws HyracksDataException { + accessor.destroy(); + } + + @Test + public void testCreateDestroySucceed() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + cursor.destroy(); + destroy(accessor); + } + + @Test + public void testDoubleOpenFails() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + boolean expectedExceptionThrown = false; + try { + open(accessor, cursor, predicates.get(0)); + } catch (Exception e) { + expectedExceptionThrown = true; + } + cursor.close(); + cursor.destroy(); + destroy(accessor); + Assert.assertTrue(expectedExceptionThrown); + } + + @Test + public void testCloseWithoutOpenSucceeds() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + cursor.close(); + cursor.destroy(); + destroy(accessor); + } + + @Test + public void testDoubleCloseSucceeds() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + cursor.close(); + cursor.close(); + cursor.destroy(); + destroy(accessor); + } + + @Test + public void testDoubleDestroySucceeds() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + cursor.close(); + cursor.destroy(); + cursor.destroy(); + destroy(accessor); + } + + @Test + public void testHasNextBeforeOpenFails() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + boolean expectedExceptionThrown = false; + try { + cursor.hasNext(); + } catch (Exception e) { + expectedExceptionThrown = true; + } + cursor.destroy(); + destroy(accessor); + Assert.assertTrue(expectedExceptionThrown); + } + + @Test + public void testHasNextAfterCloseFails() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + cursor.close(); + boolean expectedExceptionThrown = false; + try { + cursor.hasNext(); + } catch (Exception e) { + expectedExceptionThrown = true; + } + cursor.destroy(); + destroy(accessor); + Assert.assertTrue(expectedExceptionThrown); + } + + @Test + public void testNextBeforeOpenFails() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + boolean expectedExceptionThrown = false; + try { + cursor.next(); + } catch (Exception e) { + expectedExceptionThrown = true; + } + cursor.destroy(); + destroy(accessor); + Assert.assertTrue(expectedExceptionThrown); + } + + @Test + public void testNextAfterCloseFails() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + cursor.close(); + boolean expectedExceptionThrown = false; + try { + cursor.next(); + } catch (Exception e) { + expectedExceptionThrown = true; + } + cursor.destroy(); + destroy(accessor); + Assert.assertTrue(expectedExceptionThrown); + } + + @Test + public void testDestroyWhileOpenFails() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + boolean expectedExceptionThrown = false; + try { + cursor.destroy(); + } catch (Exception e) { + expectedExceptionThrown = true; + } + cursor.close(); + cursor.destroy(); + destroy(accessor); + Assert.assertTrue(expectedExceptionThrown); + } + + @Test + public void testOpenAfterDestroyFails() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + cursor.close(); + cursor.destroy(); + boolean expectedExceptionThrown = false; + try { + open(accessor, cursor, predicates.get(0)); + } catch (Exception e) { + expectedExceptionThrown = true; + } + destroy(accessor); + Assert.assertTrue(expectedExceptionThrown); + } + + @Test + public void testCloseAfterDestroyFails() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + cursor.close(); + cursor.destroy(); + boolean expectedExceptionThrown = false; + try { + cursor.close(); + } catch (Exception e) { + expectedExceptionThrown = true; + } + destroy(accessor); + Assert.assertTrue(expectedExceptionThrown); + } + + @Test + public void testNextAfterDestroyFails() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + cursor.close(); + cursor.destroy(); + boolean expectedExceptionThrown = false; + try { + cursor.next(); + } catch (Exception e) { + expectedExceptionThrown = true; + } + destroy(accessor); + Assert.assertTrue(expectedExceptionThrown); + } + + @Test + public void testHasNextAfterDestroyFails() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + cursor.close(); + cursor.destroy(); + boolean expectedExceptionThrown = false; + try { + cursor.hasNext(); + } catch (Exception e) { + expectedExceptionThrown = true; + } + destroy(accessor); + Assert.assertTrue(expectedExceptionThrown); + } + + @Test + public void testGetTupleReturnsNullAfterDestroy() throws Exception { + IIndexAccessor accessor = createAccessor(); + IIndexCursor cursor = createCursor(accessor); + List predicates = createSearchPredicates(); + open(accessor, cursor, predicates.get(0)); + cursor.close(); + cursor.destroy(); + destroy(accessor); + Assert.assertNull(cursor.getTuple()); + } + + protected IIndexCursor createCursor(IIndexAccessor accessor) { + return accessor.createSearchCursor(false); + } + + protected void open(IIndexAccessor accessor, IIndexCursor cursor, ISearchPredicate predicate) + throws HyracksDataException { + accessor.search(cursor, predicate); + } + + protected abstract List createSearchPredicates() throws Exception; + + protected abstract IIndexAccessor createAccessor() throws Exception; +} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml index 0e9c501..934ab65 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml @@ -16,17 +16,15 @@ ! specific language governing permissions and limitations ! under the License. !--> - - + 4.0.0 hyracks-storage-am-lsm-btree - org.apache.hyracks hyracks 0.3.4-SNAPSHOT - Apache License, Version 2.0 @@ -35,11 +33,9 @@ A business-friendly OSS license - ${basedir}/../.. - org.apache.hyracks @@ -92,4 +88,4 @@ ${project.version} - + \ No newline at end of file http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/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 b0c31ad..0b49b03 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 @@ -19,6 +19,7 @@ package org.apache.hyracks.storage.am.lsm.btree.impls; import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame; import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness; @@ -73,4 +74,9 @@ public class ExternalBTreeOpContext extends AbstractLSMIndexOperationContext { public LSMBTreeCursorInitialState getSearchInitialState() { return searchInitialState; } + + @Override + public void destroy() throws HyracksDataException { + // No Op + } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/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 6993013..0be5556 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 @@ -331,9 +331,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd numElements += ((AbstractLSMWithBloomFilterDiskComponent) mergeOp.getMergingComponents().get(i)) .getBloomFilter().getNumElements(); } - componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false); - try { while (buddyBtreeCursor.hasNext()) { buddyBtreeCursor.next(); @@ -341,7 +339,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd componentBulkLoader.delete(tuple); } } finally { - buddyBtreeCursor.destroy(); + buddyBtreeCursor.close(); } } else { componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false); @@ -354,7 +352,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd componentBulkLoader.add(frameTuple); } } finally { - cursor.destroy(); + cursor.close(); } componentBulkLoader.end(); return mergedComponent; http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/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 85d4ab2..1c74275 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 @@ -19,6 +19,7 @@ package org.apache.hyracks.storage.am.lsm.btree.impls; import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory; import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback; import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness; @@ -75,4 +76,9 @@ public class ExternalBTreeWithBuddyOpContext extends AbstractLSMIndexOperationCo public LSMBTreeWithBuddyCursorInitialState getSearchInitialState() { return searchInitialState; } + + @Override + public void destroy() throws HyracksDataException { + // No Op + } }