Added index accessor for BTree and RTree.
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_btree_updates_next@706 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 8cfd5d6..919ba18 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -30,16 +30,15 @@
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
public class BTreeSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
@@ -62,7 +61,7 @@
private MultiComparator highKeySearchCmp;
private ITreeIndexCursor cursor;
private ITreeIndexFrame cursorFrame;
- private BTreeOpContext opCtx;
+ private ITreeIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
@@ -134,9 +133,7 @@
dos = tb.getDataOutput();
appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
-
- opCtx = btree.createOpContext(IndexOp.SEARCH);
-
+ indexAccessor = btree.createAccessor();
} catch (Exception e) {
treeIndexOpHelper.deinit();
throw new HyracksDataException(e);
@@ -179,7 +176,7 @@
rangePred.setHighKey(highKey, highKeyInclusive);
cursor.reset();
- btree.search(cursor, rangePred, opCtx);
+ indexAccessor.search(cursor, rangePred);
writeSearchResults();
}
} catch (Exception e) {
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index 14c7208..8eea5f6 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -30,8 +30,9 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrame;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -118,10 +119,8 @@
ctx.freePages.clear();
}
- @Override
- public void diskOrderScan(ITreeIndexCursor icursor, IIndexOpContext ictx) throws HyracksDataException {
+ private void diskOrderScan(ITreeIndexCursor icursor, BTreeOpContext ctx) throws HyracksDataException {
TreeDiskOrderScanCursor cursor = (TreeDiskOrderScanCursor) icursor;
- BTreeOpContext ctx = (BTreeOpContext) ictx;
ctx.reset();
int currentPageId = rootPage;
@@ -143,9 +142,10 @@
}
}
- public void search(ITreeIndexCursor cursor, RangePredicate pred, BTreeOpContext ctx) throws Exception {
+ private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, BTreeOpContext ctx)
+ throws TreeIndexException, HyracksDataException {
ctx.reset();
- ctx.pred = pred;
+ ctx.pred = (RangePredicate) searchPred;
ctx.cursor = cursor;
// simple index scan
if (ctx.pred.getLowKeyComparator() == null) {
@@ -250,8 +250,7 @@
}
}
- private void insertUpdateOrDelete(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
- BTreeOpContext ctx = (BTreeOpContext) ictx;
+ private void insertUpdateOrDelete(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
ctx.reset();
ctx.pred.setLowKeyComparator(cmp);
ctx.pred.setHighKeyComparator(cmp);
@@ -287,25 +286,22 @@
}
}
- @Override
- public void insert(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
- insertUpdateOrDelete(tuple, ictx);
+ private void insert(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+ insertUpdateOrDelete(tuple, ctx);
}
- @Override
- public void update(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
+ private void update(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
// This call only allows updating of non-key fields.
// Updating a tuple's key necessitates deleting the old entry, and inserting the new entry.
// The user of the BTree is responsible for dealing with non-key updates (i.e., doing a delete + insert).
if (fieldCount == cmp.getKeyFieldCount()) {
throw new BTreeNotUpdateableException("Cannot perform updates when the entire tuple forms the key.");
}
- insertUpdateOrDelete(tuple, ictx);
+ insertUpdateOrDelete(tuple, ctx);
}
- @Override
- public void delete(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
- insertUpdateOrDelete(tuple, ictx);
+ private void delete(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+ insertUpdateOrDelete(tuple, ctx);
}
private void insertLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
@@ -1005,9 +1001,8 @@
}
}
- @Override
- public BTreeOpContext createOpContext(IndexOp op) {
- return new BTreeOpContext(op, (IBTreeLeafFrame) leafFrameFactory.createFrame(),
+ private BTreeOpContext createOpContext() {
+ return new BTreeOpContext((IBTreeLeafFrame) leafFrameFactory.createFrame(),
(IBTreeInteriorFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
.createFrame(), cmp);
}
@@ -1121,4 +1116,50 @@
e.printStackTrace();
}
}
+
+ @Override
+ public ITreeIndexAccessor createAccessor() {
+ return new BTreeAccessor(this);
+ }
+
+ private class BTreeAccessor implements ITreeIndexAccessor {
+ private BTree btree;
+ private BTreeOpContext ctx;
+
+ public BTreeAccessor(BTree btree) {
+ this.btree = btree;
+ this.ctx = btree.createOpContext();
+ }
+
+ @Override
+ public void insert(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+ ctx.reset(IndexOp.INSERT);
+ btree.insert(tuple, ctx);
+ }
+
+ @Override
+ public void update(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+ ctx.reset(IndexOp.UPDATE);
+ btree.update(tuple, ctx);
+ }
+
+ @Override
+ public void delete(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+ ctx.reset(IndexOp.DELETE);
+ btree.delete(tuple, ctx);
+ }
+
+ @Override
+ public void search(ITreeIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
+ TreeIndexException {
+ ctx.reset(IndexOp.SEARCH);
+ btree.search(cursor, searchPred, ctx);
+ }
+
+ @Override
+ public void diskOrderScan(ITreeIndexCursor cursor) throws HyracksDataException {
+ ctx.reset(IndexOp.DISKORDERSCAN);
+ btree.diskOrderScan(cursor, ctx);
+ }
+ }
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
index 82c747c..2cca3c9 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
@@ -40,7 +40,7 @@
public IntArrayList smPages;
public IntArrayList freePages;
- public BTreeOpContext(IndexOp op, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
+ public BTreeOpContext(IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
ITreeIndexMetaDataFrame metaFrame, MultiComparator cmp) {
if (leafFrame != null) {
leafFrame.setMultiComparator(cmp);
@@ -52,7 +52,6 @@
this.interiorFrame = interiorFrame;
this.metaFrame = metaFrame;
this.pageLsns = new LongArrayList(INIT_ARRAYLIST_SIZE, INIT_ARRAYLIST_SIZE);
- reset(op);
}
public void reset() {
@@ -67,6 +66,9 @@
@Override
public void reset(IndexOp newOp) {
+ if (op != null && newOp == op) {
+ return;
+ }
if (newOp == IndexOp.SEARCH || newOp == IndexOp.DISKORDERSCAN) {
if (cursorInitialState == null) {
cursorInitialState = new BTreeCursorInitialState(null);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
index 1072bfb..3520ec9 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
@@ -174,7 +174,7 @@
}
@Override
- public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws Exception {
+ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
// in case open is called multiple times without closing
if (page != null) {
page.releaseReadLatch();
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
index 83d0639..d223956 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
@@ -17,12 +17,12 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
/**
* Interface describing the operations of tree-based index structures. Indexes
* implementing this interface can easily reuse the tree index operators for
* dataflow. We assume that indexes store tuples with a fixed number of fields.
+ * Users must perform operations on an ITreeIndex via an ITreeIndexAccessor.
*/
public interface ITreeIndex {
@@ -51,69 +51,14 @@
public void close();
/**
- * Creates an operation context for a given index operation
- * (insert/delete/update/search/diskorderscan). An operation context
- * maintains a cache of objects used during the traversal of the tree index.
- * The context is intended to be reused for multiple subsequent operations
- * by the same user/thread. An index operation context is stateful, and
- * therefore, should not be shared among two threads.
+ * Creates an index accessor for performing operations on this index.
+ * (insert/delete/update/search/diskorderscan). An ITreeIndexAccessor is not
+ * thread safe, but different ITreeIndexAccessors can concurrently operate
+ * on the same ITreeIndex
*
- * @param indexOp
- * Intended index operation.
- *
- * @returns IITreeIndexOpContext Operation context for the desired index
- * operation.
+ * @returns ITreeIndexAccessor A tree index accessor for this tree.
*/
- public IIndexOpContext createOpContext(IndexOp op);
-
- /**
- * Inserts the given tuple into the index using an existing operation
- * context.
- *
- * @param tuple
- * Tuple to be inserted.
- * @param ictx
- * Existing operation context.
- * @throws HyracksDataException
- * If the BufferCache throws while un/pinning or un/latching.
- * @throws TreeIndexException
- * If an index-specific constraint is violated, e.g., the key
- * already exists.
- */
- public void insert(ITupleReference tuple, IIndexOpContext ictx)
- throws HyracksDataException, TreeIndexException;
-
- /**
- * Updates the tuple in the index matching the given tuple with the new
- * contents in the given tuple.
- *
- * @param tuple
- * Tuple whose match in the index is to be update with the given
- * tuples contents.
- * @param ictx
- * Existing operation context.
- * @throws HyracksDataException
- * If the BufferCache throws while un/pinning or un/latching.
- * @throws TreeIndexException
- * If there is no matching tuple in the index.
- */
- public void update(ITupleReference tuple, IIndexOpContext ictx)
- throws HyracksDataException, TreeIndexException;
-
- /**
- * Deletes the tuple in the index matching the given tuple.
- *
- * @param tuple
- * Tuple to be deleted.
- * @param ictx
- * Existing operation context.
- * @throws HyracksDataException
- * If the BufferCache throws while un/pinning or un/latching.
- * @throws TreeIndexException
- * If there is no matching tuple in the index.
- */
- public void delete(ITupleReference tuple, IIndexOpContext ictx)
- throws HyracksDataException, TreeIndexException;
+ public ITreeIndexAccessor createAccessor();
/**
* Prepares the index for bulk loading, returning a bulk load context. The
@@ -127,8 +72,8 @@
* If the tree is not empty.
* @returns A new context for bulk loading, required for appending tuples.
*/
- public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws TreeIndexException,
- HyracksDataException;
+ public IIndexBulkLoadContext beginBulkLoad(float fillFactor)
+ throws TreeIndexException, HyracksDataException;
/**
* Append a tuple to the index in the context of a bulk load.
@@ -155,19 +100,6 @@
throws HyracksDataException;
/**
- * Open the given cursor for a disk-order scan, positioning the cursor to
- * the first leaf tuple.
- *
- * @param icursor
- * Cursor to be opened for disk-order scanning.
- * @param ictx
- * Existing operation context.
- * @throws HyracksDataException
- * If the BufferCache throws while un/pinning or un/latching.
- */
- public void diskOrderScan(ITreeIndexCursor icursor, IIndexOpContext ictx) throws HyracksDataException;
-
- /**
* @return The index's leaf frame factory.
*/
public ITreeIndexFrameFactory getLeafFrameFactory();
@@ -176,7 +108,7 @@
* @return The index's interior frame factory.
*/
public ITreeIndexFrameFactory getInteriorFrameFactory();
-
+
/**
* @return The index's free page manager.
*/
@@ -186,7 +118,7 @@
* @return The number of fields tuples of this index have.
*/
public int getFieldCount();
-
+
/**
* @return The current root page id of this index.
*/
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexAccessor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexAccessor.java
new file mode 100644
index 0000000..34ce625
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexAccessor.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Client handle for performing operations
+ * (insert/delete/update/search/diskorderscan) on an ITreeIndex. An
+ * ITreeIndexAccessor is not thread safe, but different ITreeIndexAccessors can
+ * concurrently operate on the same ITreeIndex (i.e., the ITreeIndex must allow
+ * concurrent operations).
+ */
+public interface ITreeIndexAccessor {
+ /**
+ * Inserts the given tuple.
+ *
+ * @param tuple
+ * Tuple to be inserted.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ * @throws TreeIndexException
+ * If an index-specific constraint is violated, e.g., the key
+ * already exists.
+ */
+ public void insert(ITupleReference tuple) throws HyracksDataException,
+ TreeIndexException;
+
+ /**
+ * Updates the tuple in the index matching the given tuple with the new
+ * contents in the given tuple.
+ *
+ * @param tuple
+ * Tuple whose match in the index is to be update with the given
+ * tuples contents.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ * @throws TreeIndexException
+ * If there is no matching tuple in the index.
+ */
+ public void update(ITupleReference tuple) throws HyracksDataException,
+ TreeIndexException;
+
+ /**
+ * Deletes the tuple in the index matching the given tuple.
+ *
+ * @param tuple
+ * Tuple to be deleted.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ * @throws TreeIndexException
+ * If there is no matching tuple in the index.
+ */
+ public void delete(ITupleReference tuple) throws HyracksDataException,
+ TreeIndexException;
+
+ /**
+ * Open the given cursor for an index search using the given predicate as
+ * search condition.
+ *
+ * @param icursor
+ * Cursor over the index entries satisfying searchPred.
+ * @param searchPred
+ * Search condition.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ * @throws TreeIndexException
+ */
+ public void search(ITreeIndexCursor cursor, ISearchPredicate searchPred)
+ throws HyracksDataException, TreeIndexException;;
+
+ /**
+ * Open the given cursor for a disk-order scan, positioning the cursor to
+ * the first leaf tuple.
+ *
+ * @param icursor
+ * Cursor to be opened for disk-order scanning.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ */
+ public void diskOrderScan(ITreeIndexCursor cursor)
+ throws HyracksDataException;
+
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
index 56dfd47..fdfedb2 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.api;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
@@ -27,7 +28,7 @@
public void next() throws Exception;
public void open(ICursorInitialState initialState,
- ISearchPredicate searchPred) throws Exception;
+ ISearchPredicate searchPred) throws HyracksDataException;
public ICachedPage getPage();
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index ab7b335..184bb4e 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -24,10 +24,9 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
public class TreeIndexDiskOrderScanOperatorNodePushable extends
AbstractUnaryOutputSourceOperatorNodePushable {
@@ -43,19 +42,16 @@
@Override
public void initialize() throws HyracksDataException {
-
ITreeIndexFrame cursorFrame = treeIndexOpHelper.getOperatorDescriptor()
.getTreeIndexLeafFactory().createFrame();
ITreeIndexCursor cursor = treeIndexOpHelper
.createDiskOrderScanCursor(cursorFrame);
- IIndexOpContext diskOrderScanOpCtx = treeIndexOpHelper.getTreeIndex()
- .createOpContext(IndexOp.DISKORDERSCAN);
+ ITreeIndexAccessor indexAccessor = treeIndexOpHelper.getTreeIndex().createAccessor();
try {
-
treeIndexOpHelper.init();
writer.open();
try {
- treeIndexOpHelper.getTreeIndex().diskOrderScan(cursor, diskOrderScanOpCtx);
+ indexAccessor.diskOrderScan(cursor);
int fieldCount = treeIndexOpHelper.getTreeIndex()
.getFieldCount();
@@ -90,7 +86,6 @@
}
}
}
-
if (appender.getTupleCount() > 0) {
FrameUtils.flushFrame(frame, writer);
}
@@ -101,7 +96,6 @@
cursor.close();
writer.close();
}
-
} catch (Exception e) {
deinitialize();
throw new HyracksDataException(e);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
index 0ae513b..487d7ab 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -23,8 +23,8 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
public class TreeIndexInsertUpdateDeleteOperatorNodePushable extends
@@ -35,7 +35,7 @@
private final IndexOp op;
private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
private ByteBuffer writeBuffer;
- private IIndexOpContext opCtx;
+ private ITreeIndexAccessor indexAccessor;
public TreeIndexInsertUpdateDeleteOperatorNodePushable(
AbstractTreeIndexOperatorDescriptor opDesc,
@@ -63,7 +63,7 @@
treeIndexOpHelper.init();
treeIndexOpHelper.getTreeIndex().open(
treeIndexOpHelper.getIndexFileId());
- opCtx = treeIndexOpHelper.getTreeIndex().createOpContext(op);
+ indexAccessor = treeIndexOpHelper.getTreeIndex().createAccessor();
} catch (Exception e) {
// cleanup in case of failure
treeIndexOpHelper.deinit();
@@ -83,17 +83,17 @@
switch (op) {
case INSERT: {
- treeIndex.insert(tuple, opCtx);
+ indexAccessor.insert(tuple);
break;
}
case UPDATE: {
- treeIndex.update(tuple, opCtx);
+ indexAccessor.update(tuple);
break;
}
case DELETE: {
- treeIndex.delete(tuple, opCtx);
+ indexAccessor.delete(tuple);
break;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
index 75fdf2a..f1755ab 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
@@ -29,9 +29,9 @@
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
@@ -151,9 +151,9 @@
}
}
- public boolean openCursor(ITreeIndexCursor btreeCursor, RangePredicate btreePred, BTreeOpContext btreeOpCtx,
+ public boolean openCursor(ITreeIndexCursor btreeCursor, RangePredicate btreePred, ITreeIndexAccessor btreeAccessor,
IInvertedListCursor invListCursor) throws Exception {
- btree.search(btreeCursor, btreePred, btreeOpCtx);
+ btreeAccessor.search(btreeCursor, btreePred);
boolean ret = false;
if (btreeCursor.hasNext()) {
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index 0f88c0b..3cfa1d0 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -36,12 +36,11 @@
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexResultCursor;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
@@ -69,7 +68,7 @@
protected final ITreeIndexCursor btreeCursor;
protected final FrameTupleReference searchKey = new FrameTupleReference();
protected final RangePredicate btreePred = new RangePredicate(true, null, null, true, true, null, null);
- protected final BTreeOpContext btreeOpCtx;
+ protected final ITreeIndexAccessor btreeAccessor;
protected RecordDescriptor queryTokenRecDesc = new RecordDescriptor(
new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
@@ -107,8 +106,6 @@
invListFieldsWithCount[invListFields.length] = new TypeTrait(4);
invListKeyLength = tmp;
- btreeOpCtx = invIndex.getBTree().createOpContext(IndexOp.SEARCH);
-
resultFrameTupleApp = new FixedSizeFrameTupleAppender(ctx.getFrameSize(), invListFieldsWithCount);
resultFrameTupleAcc = new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), invListFieldsWithCount);
resultTuple = new FixedSizeTupleReference(invListFieldsWithCount);
@@ -129,7 +126,8 @@
queryTokenAppender = new FrameTupleAppender(ctx.getFrameSize());
queryTokenFrame = ctx.allocateFrame();
-
+
+ btreeAccessor = invIndex.getBTree().createAccessor();
currentNumResults = 0;
}
@@ -181,7 +179,7 @@
invListCursors.clear();
for (int i = 0; i < numQueryTokens; i++) {
searchKey.reset(queryTokenAccessor, i);
- invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursorCache.get(i));
+ invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursorCache.get(i));
invListCursors.add(invListCursorCache.get(i));
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 4142ebf..daf312f 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -29,18 +29,17 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeOpContext;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
@@ -60,7 +59,7 @@
private ITreeIndexCursor cursor;
private ITreeIndexFrame interiorFrame;
private ITreeIndexFrame leafFrame;
- private RTreeOpContext opCtx;
+ private ITreeIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
@@ -110,7 +109,7 @@
appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
- opCtx = rtree.createOpContext(IndexOp.SEARCH);
+ indexAccessor = rtree.createAccessor();
} catch (Exception e) {
writer.fail();
throw e;
@@ -154,7 +153,7 @@
searchPred.setSearchKey(searchKey);
cursor.reset();
- rtree.search(cursor, searchPred, opCtx);
+ indexAccessor.search(cursor, searchPred);
writeSearchResults();
}
} catch (Exception e) {
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
index 46baae2..37c1520 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
@@ -26,7 +26,9 @@
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -242,15 +244,13 @@
fileId = -1;
}
- @Override
- public RTreeOpContext createOpContext(IndexOp op) {
- return new RTreeOpContext(op, (IRTreeLeafFrame) leafFrameFactory.createFrame(),
+ private RTreeOpContext createOpContext() {
+ return new RTreeOpContext((IRTreeLeafFrame) leafFrameFactory.createFrame(),
(IRTreeInteriorFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
.createFrame(), 8);
}
- @Override
- public void insert(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
+ private void insert(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
RTreeOpContext ctx = (RTreeOpContext) ictx;
ctx.reset();
ctx.setTuple(tuple);
@@ -686,9 +686,7 @@
}
}
- @Override
- public void delete(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
- RTreeOpContext ctx = (RTreeOpContext) ictx;
+ public void delete(ITupleReference tuple, RTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
ctx.reset();
ctx.setTuple(tuple);
ctx.splitKey.reset();
@@ -934,14 +932,14 @@
}
}
- public void search(ITreeIndexCursor cursor, SearchPredicate pred, RTreeOpContext ctx) throws Exception {
- ctx.reset();
+ private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, RTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+ ctx.reset();
ctx.cursor = cursor;
cursor.setBufferCache(bufferCache);
cursor.setFileId(fileId);
ctx.cursorInitialState.setRootPage(rootPage);
- ctx.cursor.open(ctx.cursorInitialState, pred);
+ ctx.cursor.open(ctx.cursorInitialState, (SearchPredicate)searchPred);
}
public ITreeIndexFrameFactory getInteriorFrameFactory() {
@@ -960,18 +958,17 @@
return freePageManager;
}
- @Override
- public void update(ITupleReference tuple, IIndexOpContext ictx) {
+ private void update(ITupleReference tuple, RTreeOpContext ctx) {
throw new UnsupportedOperationException("RTree Update not implemented.");
}
public final class BulkLoadContext implements IIndexBulkLoadContext {
- public RTreeOpContext insertOpCtx;
+ public ITreeIndexAccessor indexAccessor;
public BulkLoadContext(float fillFactor, IRTreeFrame leafFrame, IRTreeFrame interiorFrame,
ITreeIndexMetaDataFrame metaFrame) throws HyracksDataException {
- insertOpCtx = createOpContext(IndexOp.INSERT);
+ indexAccessor = createAccessor();
}
}
@@ -990,7 +987,7 @@
@Override
public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException {
try {
- insert(tuple, ((BulkLoadContext) ictx).insertOpCtx);
+ ((BulkLoadContext) ictx).indexAccessor.insert(tuple);
} catch (Exception e) {
throw new HyracksDataException("BulkLoad Error");
}
@@ -1001,10 +998,8 @@
loaded = true;
}
- @Override
- public void diskOrderScan(ITreeIndexCursor icursor, IIndexOpContext ictx) throws HyracksDataException {
+ private void diskOrderScan(ITreeIndexCursor icursor, RTreeOpContext ctx) throws HyracksDataException {
TreeDiskOrderScanCursor cursor = (TreeDiskOrderScanCursor) icursor;
- RTreeOpContext ctx = (RTreeOpContext) ictx;
ctx.reset();
int currentPageId = rootPage + 1;
@@ -1040,4 +1035,50 @@
public IndexType getIndexType() {
return IndexType.RTREE;
}
+
+ @Override
+ public ITreeIndexAccessor createAccessor() {
+ return new RTreeAccessor(this);
+ }
+
+ private class RTreeAccessor implements ITreeIndexAccessor {
+ private RTree rtree;
+ private RTreeOpContext ctx;
+
+ public RTreeAccessor(RTree rtree) {
+ this.rtree = rtree;
+ this.ctx = rtree.createOpContext();
+ }
+
+ @Override
+ public void insert(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+ ctx.reset(IndexOp.INSERT);
+ rtree.insert(tuple, ctx);
+ }
+
+ @Override
+ public void update(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+ ctx.reset(IndexOp.UPDATE);
+ rtree.update(tuple, ctx);
+ }
+
+ @Override
+ public void delete(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+ ctx.reset(IndexOp.DELETE);
+ rtree.delete(tuple, ctx);
+ }
+
+ @Override
+ public void search(ITreeIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
+ TreeIndexException {
+ ctx.reset(IndexOp.SEARCH);
+ rtree.search(cursor, searchPred, ctx);
+ }
+
+ @Override
+ public void diskOrderScan(ITreeIndexCursor cursor) throws HyracksDataException {
+ ctx.reset(IndexOp.DISKORDERSCAN);
+ rtree.diskOrderScan(cursor, ctx);
+ }
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
index dc4a753..c258377 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
@@ -37,24 +37,13 @@
public PathList traverseList; // used for traversing the tree
private static final int initTraverseListSize = 100;
- public RTreeOpContext(IndexOp op, IRTreeLeafFrame leafFrame,
+ public RTreeOpContext(IRTreeLeafFrame leafFrame,
IRTreeInteriorFrame interiorFrame,
ITreeIndexMetaDataFrame metaFrame, int treeHeightHint) {
this.interiorFrame = interiorFrame;
this.leafFrame = leafFrame;
this.metaFrame = metaFrame;
pathList = new PathList(treeHeightHint, treeHeightHint);
- if (op != IndexOp.SEARCH && op != IndexOp.DISKORDERSCAN) {
- splitKey = new RTreeSplitKey(interiorFrame.getTupleWriter()
- .createTupleReference(), interiorFrame.getTupleWriter()
- .createTupleReference());
- traverseList = new PathList(initTraverseListSize,
- initTraverseListSize);
- } else {
- splitKey = null;
- traverseList = null;
- cursorInitialState = new RTreeCursorInitialState(pathList, 1);
- }
}
public ITupleReference getTuple() {
@@ -76,6 +65,9 @@
@Override
public void reset(IndexOp newOp) {
+ if (op != null && newOp == op) {
+ return;
+ }
if (op != IndexOp.SEARCH && op != IndexOp.DISKORDERSCAN) {
if (splitKey == null) {
splitKey = new RTreeSplitKey(interiorFrame.getTupleWriter()
@@ -86,11 +78,9 @@
traverseList = new PathList(initTraverseListSize,
initTraverseListSize);
}
-
- } else {
- if (cursorInitialState == null) {
- cursorInitialState = new RTreeCursorInitialState(pathList, 1);
- }
+ }
+ if (cursorInitialState == null) {
+ cursorInitialState = new RTreeCursorInitialState(pathList, 1);
}
this.op = newOp;
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
index 16b96b2..82f07ad 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
@@ -169,7 +169,7 @@
}
@Override
- public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws Exception {
+ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
// in case open is called multiple times without closing
if (this.page != null) {
this.page.releaseReadLatch();
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
index 7c181c8..84ff196 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
@@ -26,17 +26,16 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.util.AbstractBTreeTest;
import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexBufferCacheWarmup;
@@ -116,8 +115,7 @@
accessor.reset(frame);
FrameTupleReference tuple = new FrameTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
// 10000
for (int i = 0; i < 100000; i++) {
@@ -141,7 +139,7 @@
}
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
index a357f2a..eb00f95 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
@@ -52,13 +52,13 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.btree.util.AbstractBTreeTest;
import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -68,7 +68,6 @@
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -149,8 +148,8 @@
accessor.reset(frame);
FrameTupleReference tuple = new FrameTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+
// 10000
for (int i = 0; i < 10000; i++) {
@@ -177,7 +176,7 @@
}
try {
- btree.insert(t, insertOpCtx);
+ indexAccessor.insert(t);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -197,8 +196,7 @@
LOGGER.info("ORDERED SCAN:");
ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(scanCursor, nullPred, searchOpCtx);
+ indexAccessor.search(scanCursor, nullPred);
try {
while (scanCursor.hasNext()) {
scanCursor.next();
@@ -215,8 +213,7 @@
// disk-order scan
LOGGER.info("DISK-ORDER SCAN:");
TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
- BTreeOpContext diskOrderScanOpCtx = btree.createOpContext(IndexOp.DISKORDERSCAN);
- btree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+ indexAccessor.diskOrderScan(diskOrderCursor);
try {
while (diskOrderCursor.hasNext()) {
diskOrderCursor.next();
@@ -270,7 +267,7 @@
MultiComparator searchCmp = new MultiComparator(searchCmps);
RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
- btree.search(rangeCursor, rangePred, searchOpCtx);
+ indexAccessor.search(rangeCursor, rangePred);
try {
while (rangeCursor.hasNext()) {
@@ -357,7 +354,7 @@
accessor.reset(frame);
FrameTupleReference tuple = new FrameTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
// Magic test number: 3029. 6398. 4875.
for (int i = 0; i < 10000; i++) {
@@ -383,7 +380,7 @@
}
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (Exception e) {
}
@@ -400,9 +397,7 @@
LOGGER.info("ORDERED SCAN:");
ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(scanCursor, nullPred, searchOpCtx);
-
+ indexAccessor.search(scanCursor, nullPred);
try {
while (scanCursor.hasNext()) {
scanCursor.next();
@@ -461,7 +456,7 @@
// searching
RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
- btree.search(rangeCursor, rangePred, searchOpCtx);
+ indexAccessor.search(rangeCursor, rangePred);
try {
while (rangeCursor.hasNext()) {
@@ -476,7 +471,6 @@
rangeCursor.close();
}
-
btree.close();
bufferCache.closeFile(fileId);
bufferCache.close();
@@ -543,7 +537,7 @@
accessor.reset(frame);
FrameTupleReference tuple = new FrameTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
int maxLength = 10; // max string length to be generated
for (int i = 0; i < 10000; i++) {
@@ -566,7 +560,7 @@
}
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (Exception e) {
}
}
@@ -578,9 +572,7 @@
LOGGER.info("ORDERED SCAN:");
ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(scanCursor, nullPred, searchOpCtx);
-
+ indexAccessor.search(scanCursor, nullPred);
try {
while (scanCursor.hasNext()) {
scanCursor.next();
@@ -634,7 +626,7 @@
MultiComparator searchCmp = new MultiComparator(searchCmps);
RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
- btree.search(rangeCursor, rangePred, searchOpCtx);
+ indexAccessor.search(rangeCursor, rangePred);
try {
while (rangeCursor.hasNext()) {
@@ -716,9 +708,8 @@
accessor.reset(frame);
FrameTupleReference tuple = new FrameTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
- BTreeOpContext deleteOpCtx = btree.createOpContext(IndexOp.DELETE);
-
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+
int runs = 3;
for (int run = 0; run < runs; run++) {
@@ -755,7 +746,7 @@
}
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
insDone++;
} catch (TreeIndexException e) {
//e.printStackTrace();
@@ -786,7 +777,7 @@
}
try {
- btree.delete(tuple, deleteOpCtx);
+ indexAccessor.delete(tuple);
delDone++;
} catch (TreeIndexException e) {
//e.printStackTrace();
@@ -819,8 +810,8 @@
LOGGER.info("ORDERED SCAN:");
ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(scanCursor, nullPred, searchOpCtx);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ indexAccessor.search(scanCursor, nullPred);
StringBuilder scanResults = new StringBuilder();
try {
while (scanCursor.hasNext()) {
@@ -918,8 +909,7 @@
accessor.reset(frame);
FrameTupleReference tuple = new FrameTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
- BTreeOpContext updateOpCtx = btree.createOpContext(IndexOp.UPDATE);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
Map<String, String> expectedValues = new HashMap<String, String>();
@@ -952,7 +942,7 @@
LOGGER.info("INSERTING " + i);
}
try {
- btree.insert(t, insertOpCtx);
+ indexAccessor.insert(t);
expectedValues.put(f0, f1);
} catch (TreeIndexException e) {
// e.printStackTrace();
@@ -962,8 +952,7 @@
}
ITreeIndexCursor insertCheckCursor = new BTreeRangeSearchCursor(leafFrame);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(insertCheckCursor, nullPred, searchOpCtx);
+ indexAccessor.search(insertCheckCursor, nullPred);
try {
compareActualAndExpected(insertCheckCursor, expectedValues, fieldSerdes);
} finally {
@@ -999,7 +988,7 @@
t.reset(tb.getFieldEndOffsets(), tb.getByteArray());
try {
- btree.update(t, updateOpCtx);
+ indexAccessor.update(t);
expectedValues.put(f0s[i], f1);
} catch (TreeIndexException e) {
e.printStackTrace();
@@ -1009,7 +998,7 @@
}
ITreeIndexCursor updateCheckCursor = new BTreeRangeSearchCursor(leafFrame);
- btree.search(updateCheckCursor, nullPred, searchOpCtx);
+ indexAccessor.search(updateCheckCursor, nullPred);
try {
compareActualAndExpected(updateCheckCursor, expectedValues, fieldSerdes);
} finally {
@@ -1156,9 +1145,8 @@
// TODO: check when searching backwards
RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(rangeCursor, rangePred, searchOpCtx);
-
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ indexAccessor.search(rangeCursor, rangePred);
try {
while (rangeCursor.hasNext()) {
rangeCursor.next();
@@ -1273,9 +1261,7 @@
intervals[9][0] = 20;
intervals[9][1] = 35;
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-
- // int exceptionCount = 0;
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
for (int i = 0; i < intervalCount; i++) {
int f0 = intervals[i][0];
int f1 = intervals[i][1];
@@ -1297,7 +1283,7 @@
LOGGER.info("INSERTING " + i);
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (Exception e) {
}
}
@@ -1313,9 +1299,7 @@
LOGGER.info("ORDERED SCAN:");
ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(scanCursor, nullPred, searchOpCtx);
-
+ indexAccessor.search(scanCursor, nullPred);
try {
while (scanCursor.hasNext()) {
scanCursor.next();
@@ -1375,7 +1359,7 @@
MultiComparator searchCmp = new MultiComparator(searchCmps);
RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
- btree.search(rangeCursor, rangePred, searchOpCtx);
+ indexAccessor.search(rangeCursor, rangePred);
try {
while (rangeCursor.hasNext()) {
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
index 14246f7..20e6383 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
@@ -43,19 +43,18 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.btree.util.AbstractBTreeTest;
import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
@@ -106,7 +105,7 @@
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
// generate keys
int numKeys = 50;
@@ -128,7 +127,7 @@
tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (BTreeException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -192,7 +191,7 @@
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
// generate keys
int numKeys = 50;
@@ -211,7 +210,7 @@
tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (BTreeException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -275,7 +274,7 @@
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
// generate keys
int numKeys = 50;
@@ -294,7 +293,7 @@
tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (BTreeException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -409,8 +408,8 @@
RangePredicate rangePred = createRangePredicate(lowKey,
highKey, isForward, lowKeyInclusive, highKeyInclusive,
btree.getMultiComparator());
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(rangeCursor, rangePred, searchOpCtx);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ indexAccessor.search(rangeCursor, rangePred);
try {
while (rangeCursor.hasNext()) {
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
index c56f1b6..f1b03c1 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
@@ -23,39 +23,40 @@
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@SuppressWarnings("rawtypes")
-public final class BTreeTestContext {
+public final class BTreeTestContext {
public final ISerializerDeserializer[] fieldSerdes;
public final IBufferCache bufferCache;
public final BTree btree;
public final IBTreeLeafFrame leafFrame;
public final IBTreeInteriorFrame interiorFrame;
- public final ITreeIndexMetaDataFrame metaFrame;
+ public final ITreeIndexMetaDataFrame metaFrame;
public final ArrayTupleBuilder tupleBuilder;
public final ArrayTupleReference tuple = new ArrayTupleReference();
public final TreeSet<CheckTuple> checkTuples = new TreeSet<CheckTuple>();
- public final BTreeOpContext opCtx;
-
- public BTreeTestContext(IBufferCache bufferCache, ISerializerDeserializer[] fieldSerdes, BTree btree, IBTreeLeafFrame leafFrame,
- IBTreeInteriorFrame interiorFrame, ITreeIndexMetaDataFrame metaFrame, BTreeOpContext opCtx) {
+ public final ITreeIndexAccessor indexAccessor;
+
+ public BTreeTestContext(IBufferCache bufferCache, ISerializerDeserializer[] fieldSerdes, BTree btree,
+ IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame, ITreeIndexMetaDataFrame metaFrame,
+ ITreeIndexAccessor indexAccessor) {
this.bufferCache = bufferCache;
this.fieldSerdes = fieldSerdes;
this.btree = btree;
this.leafFrame = leafFrame;
this.interiorFrame = interiorFrame;
this.metaFrame = metaFrame;
- this.opCtx = opCtx;
+ this.indexAccessor = indexAccessor;
this.tupleBuilder = new ArrayTupleBuilder(fieldSerdes.length);
}
-
+
public int getFieldCount() {
return fieldSerdes.length;
}
-
+
public int getKeyFieldCount() {
return btree.getMultiComparator().getKeyFieldCount();
}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
index a86f50c..b041403 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
@@ -28,15 +28,14 @@
import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -51,13 +50,12 @@
BTree btree = BTreeUtils.createBTree(bufferCache, btreeFileId, typeTraits, cmps, leafType);
btree.create(btreeFileId);
btree.open(btreeFileId);
- // Set an arbitrary index op in the context. Will be reset anyway.
- BTreeOpContext opCtx = btree.createOpContext(IndexOp.SEARCH);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) btree.getInteriorFrameFactory().createFrame();
ITreeIndexMetaDataFrame metaFrame = btree.getFreePageManager().getMetaDataFrameFactory().createFrame();
- BTreeTestContext testCtx = new BTreeTestContext(bufferCache, fieldSerdes, btree, leafFrame, interiorFrame, metaFrame, opCtx);
+ BTreeTestContext testCtx = new BTreeTestContext(bufferCache, fieldSerdes, btree, leafFrame, interiorFrame, metaFrame, indexAccessor);
return testCtx;
}
@@ -105,8 +103,7 @@
LOGGER.info("Testing Ordered Scan.");
ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- testCtx.opCtx.reset(IndexOp.SEARCH);
- testCtx.btree.search(scanCursor, nullPred, testCtx.opCtx);
+ testCtx.indexAccessor.search(scanCursor, nullPred);
Iterator<CheckTuple> checkIter = testCtx.checkTuples.iterator();
int actualCount = 0;
try {
@@ -131,8 +128,7 @@
public static void checkDiskOrderScan(BTreeTestContext testCtx) throws Exception {
LOGGER.info("Testing Disk-Order Scan.");
ITreeIndexCursor diskOrderCursor = new TreeDiskOrderScanCursor(testCtx.leafFrame);
- testCtx.opCtx.reset(IndexOp.DISKORDERSCAN);
- testCtx.btree.diskOrderScan(diskOrderCursor, testCtx.opCtx);
+ testCtx.indexAccessor.diskOrderScan(diskOrderCursor);
int actualCount = 0;
try {
while (diskOrderCursor.hasNext()) {
@@ -161,8 +157,7 @@
MultiComparator highKeyCmp = BTreeUtils.getSearchMultiComparator(testCtx.btree.getMultiComparator(), highKey);
ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeyCmp, highKeyCmp);
- testCtx.opCtx.reset(IndexOp.SEARCH);
- testCtx.btree.search(searchCursor, rangePred, testCtx.opCtx);
+ testCtx.indexAccessor.search(searchCursor, rangePred);
// Get the subset of elements from the expected set within given key range.
CheckTuple lowKeyCheck = createCheckTupleFromTuple(lowKey, testCtx.fieldSerdes, lowKeyCmp.getKeyFieldCount());
CheckTuple highKeyCheck = createCheckTupleFromTuple(highKey, testCtx.fieldSerdes, highKeyCmp.getKeyFieldCount());
@@ -204,7 +199,6 @@
ArrayTupleReference lowKey = new ArrayTupleReference();
ArrayTupleBuilder highKeyBuilder = new ArrayTupleBuilder(testCtx.btree.getMultiComparator().getKeyFieldCount());
ArrayTupleReference highKey = new ArrayTupleReference();
- testCtx.opCtx.reset(IndexOp.SEARCH);
RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, null, null);
// Iterate through expected tuples, and perform a point search in the BTree to verify the tuple can be reached.
@@ -219,7 +213,7 @@
rangePred.setLowKeyComparator(lowKeyCmp);
rangePred.setHighKeyComparator(highKeyCmp);
- testCtx.btree.search(searchCursor, rangePred, testCtx.opCtx);
+ testCtx.indexAccessor.search(searchCursor, rangePred);
try {
// We expect exactly one answer.
@@ -269,9 +263,6 @@
public static void insertIntTuples(BTreeTestContext testCtx, int numTuples, Random rnd) throws Exception {
int fieldCount = testCtx.getFieldCount();
int numKeyFields = testCtx.getKeyFieldCount();
-
- testCtx.opCtx.reset(IndexOp.INSERT);
-
int[] tupleValues = new int[testCtx.getFieldCount()];
// Scale range of values according to number of keys.
// For example, for 2 keys we want the square root of numTuples, for 3 keys the cube root of numTuples, etc.
@@ -290,7 +281,7 @@
LOGGER.info("Inserting Tuple " + (i + 1) + "/" + numTuples);
}
try {
- testCtx.btree.insert(testCtx.tuple, testCtx.opCtx);
+ testCtx.indexAccessor.insert(testCtx.tuple);
// Set expected values. Do this only after insertion succeeds because we ignore duplicate keys.
CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(fieldCount, numKeyFields);
for(int v : tupleValues) {
@@ -306,9 +297,6 @@
public static void insertStringTuples(BTreeTestContext testCtx, int numTuples, Random rnd) throws Exception {
int fieldCount = testCtx.getFieldCount();
int numKeyFields = testCtx.getKeyFieldCount();
-
- testCtx.opCtx.reset(IndexOp.INSERT);
-
Object[] tupleValues = new Object[fieldCount];
for (int i = 0; i < numTuples; i++) {
if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
@@ -325,7 +313,7 @@
}
TupleUtils.createTuple(testCtx.tupleBuilder, testCtx.tuple, testCtx.fieldSerdes, tupleValues);
try {
- testCtx.btree.insert(testCtx.tuple, testCtx.opCtx);
+ testCtx.indexAccessor.insert(testCtx.tuple);
// Set expected values. Do this only after insertion succeeds because we ignore duplicate keys.
CheckTuple<String> checkTuple = new CheckTuple<String>(fieldCount, numKeyFields);
for(Object v : tupleValues) {
@@ -411,7 +399,6 @@
ArrayTupleBuilder deleteTupleBuilder = new ArrayTupleBuilder(testCtx.btree.getMultiComparator().getKeyFieldCount());
ArrayTupleReference deleteTuple = new ArrayTupleReference();
int numCheckTuples = testCtx.checkTuples.size();
- testCtx.opCtx.reset(IndexOp.DELETE);
// Copy CheckTuple references into array, so we can randomly pick from there.
CheckTuple[] checkTuples = new CheckTuple[numCheckTuples];
int idx = 0;
@@ -425,7 +412,7 @@
int checkTupleIdx = Math.abs(rnd.nextInt() % numCheckTuples);
CheckTuple checkTuple = checkTuples[checkTupleIdx];
createTupleFromCheckTuple(checkTuple, deleteTupleBuilder, deleteTuple, testCtx.fieldSerdes);
- testCtx.btree.delete(deleteTuple, testCtx.opCtx);
+ testCtx.indexAccessor.delete(deleteTuple);
// Remove check tuple from expected results.
testCtx.checkTuples.remove(checkTuple);
@@ -449,7 +436,6 @@
ArrayTupleBuilder updateTupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference updateTuple = new ArrayTupleReference();
int numCheckTuples = testCtx.checkTuples.size();
- testCtx.opCtx.reset(IndexOp.UPDATE);
// Copy CheckTuple references into array, so we can randomly pick from there.
CheckTuple[] checkTuples = new CheckTuple[numCheckTuples];
int idx = 0;
@@ -469,7 +455,7 @@
}
createTupleFromCheckTuple(checkTuple, updateTupleBuilder, updateTuple, testCtx.fieldSerdes);
- testCtx.btree.update(updateTuple, testCtx.opCtx);
+ testCtx.indexAccessor.update(updateTuple);
// Swap with last "valid" CheckTuple.
CheckTuple tmp = checkTuples[numCheckTuples - 1];
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
index 86b3107..0d55dec 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
@@ -48,10 +48,10 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -59,7 +59,6 @@
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
@@ -241,7 +240,7 @@
IFrameTupleAccessor tokenAccessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), tokenRecDesc);
tokenAccessor.reset(frame);
- BTreeOpContext btreeOpCtx = invIndex.getBTree().createOpContext(IndexOp.SEARCH);
+ ITreeIndexAccessor btreeAccessor = invIndex.getBTree().createAccessor();
// verify created inverted lists one-by-one
for (int i = 0; i < tokens.size(); i++) {
@@ -256,7 +255,7 @@
searchKey.reset(tokenAccessor, 0);
- invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursor);
+ invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursor);
invListCursor.pinPagesSync();
int checkIndex = 0;
@@ -291,7 +290,7 @@
searchKey.reset(tokenAccessor, 0);
- invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursor);
+ invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursor);
invListCursor.pinPagesSync();
Assert.assertEquals(invListCursor.hasNext(), false);
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
index ce3bd3c..8c0e376 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
@@ -42,6 +42,7 @@
import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
@@ -49,7 +50,6 @@
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStats;
import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStatsGatherer;
@@ -57,7 +57,6 @@
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeOpContext;
import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -155,7 +154,7 @@
accessor.reset(hyracksFrame);
FrameTupleReference tuple = new FrameTupleReference();
- RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
Random rnd = new Random();
rnd.setSeed(50);
@@ -206,7 +205,7 @@
}
try {
- rtree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -223,8 +222,7 @@
print("DISK-ORDER SCAN:\n");
TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
leafFrame);
- RTreeOpContext diskOrderScanOpCtx = rtree.createOpContext(IndexOp.DISKORDERSCAN);
- rtree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+ indexAccessor.diskOrderScan(diskOrderCursor);
try {
while (diskOrderCursor.hasNext()) {
diskOrderCursor.next();
@@ -334,7 +332,7 @@
accessor.reset(hyracksFrame);
FrameTupleReference tuple = new FrameTupleReference();
- RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
Random rnd = new Random();
rnd.setSeed(50);
@@ -383,7 +381,7 @@
}
try {
- rtree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -396,7 +394,6 @@
String rtreeStats = rtree.printStats();
print(rtreeStats);
- RTreeOpContext deleteOpCtx = rtree.createOpContext(IndexOp.DELETE);
rnd.setSeed(50);
for (int i = 0; i < 5000; i++) {
@@ -442,8 +439,7 @@
}
try {
- rtree.delete(tuple, deleteOpCtx);
-
+ indexAccessor.delete(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -552,7 +548,7 @@
accessor.reset(hyracksFrame);
FrameTupleReference tuple = new FrameTupleReference();
- RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
Random rnd = new Random();
rnd.setSeed(50);
@@ -610,7 +606,7 @@
}
try {
- rtree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -627,8 +623,7 @@
print("DISK-ORDER SCAN:\n");
TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
leafFrame);
- RTreeOpContext diskOrderScanOpCtx = rtree.createOpContext(IndexOp.DISKORDERSCAN);
- rtree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+ indexAccessor.diskOrderScan(diskOrderCursor);
try {
while (diskOrderCursor.hasNext()) {
diskOrderCursor.next();
@@ -738,7 +733,7 @@
accessor.reset(hyracksFrame);
FrameTupleReference tuple = new FrameTupleReference();
- RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
Random rnd = new Random();
rnd.setSeed(50);
@@ -789,7 +784,7 @@
}
try {
- rtree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -806,8 +801,7 @@
print("DISK-ORDER SCAN:\n");
TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
leafFrame);
- RTreeOpContext diskOrderScanOpCtx = rtree.createOpContext(IndexOp.DISKORDERSCAN);
- rtree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+ indexAccessor.diskOrderScan(diskOrderCursor);
try {
while (diskOrderCursor.hasNext()) {
diskOrderCursor.next();
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
index c87f0da..be91b88 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
@@ -44,6 +44,7 @@
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
@@ -51,14 +52,12 @@
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeOpContext;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
@@ -155,7 +154,7 @@
accessor.reset(hyracksFrame);
FrameTupleReference tuple = new FrameTupleReference();
- RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
Random rnd = new Random();
rnd.setSeed(50);
@@ -197,7 +196,7 @@
}
try {
- rtree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -242,8 +241,7 @@
interiorFrame, leafFrame);
SearchPredicate searchPredicate = new SearchPredicate(tuple, cmp);
- RTreeOpContext searchOpCtx = rtree.createOpContext(IndexOp.SEARCH);
- rtree.search(searchCursor, searchPredicate, searchOpCtx);
+ indexAccessor.search(searchCursor, searchPredicate);
ArrayList<Integer> results = new ArrayList<Integer>();
try {