Added BTreeUpdateSearch operator for Yingyi.
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_btree_updates_next@824 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/IntegerSerializerDeserializer.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/IntegerSerializerDeserializer.java
index 032fddf..171dbb7 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/IntegerSerializerDeserializer.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/IntegerSerializerDeserializer.java
@@ -51,4 +51,11 @@
return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
+ ((bytes[offset + 3] & 0xff) << 0);
}
+
+ public static void putInt(int val, byte[] bytes, int offset) {
+ bytes[offset] = (byte)((val >>> 24) & 0xFF);
+ bytes[offset + 1] = (byte)((val >>> 16) & 0xFF);
+ bytes[offset + 2] = (byte)((val >>> 8) & 0xFF);
+ bytes[offset + 3] = (byte)((val >>> 0) & 0xFF);
+ }
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 649cb3f..fa64877 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -34,12 +34,12 @@
private static final long serialVersionUID = 1L;
- private boolean isForward;
- private int[] lowKeyFields; // fields in input tuple to be used as low keys
- private int[] highKeyFields; // fields in input tuple to be used as high
+ protected boolean isForward;
+ protected int[] lowKeyFields; // fields in input tuple to be used as low keys
+ protected int[] highKeyFields; // fields in input tuple to be used as high
// keys
- private boolean lowKeyInclusive;
- private boolean highKeyInclusive;
+ protected boolean lowKeyInclusive;
+ protected boolean highKeyInclusive;
public BTreeSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
IStorageManagerInterface storageManager, IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
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 48ee8d4..8a24759 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
@@ -42,28 +42,28 @@
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
public class BTreeSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
- private TreeIndexOpHelper treeIndexOpHelper;
- private FrameTupleAccessor accessor;
+ protected TreeIndexOpHelper treeIndexOpHelper;
+ protected FrameTupleAccessor accessor;
- private ByteBuffer writeBuffer;
- private FrameTupleAppender appender;
- private ArrayTupleBuilder tb;
- private DataOutput dos;
+ protected ByteBuffer writeBuffer;
+ protected FrameTupleAppender appender;
+ protected ArrayTupleBuilder tb;
+ protected DataOutput dos;
- private BTree btree;
- private boolean isForward;
- private PermutingFrameTupleReference lowKey;
- private PermutingFrameTupleReference highKey;
- private boolean lowKeyInclusive;
- private boolean highKeyInclusive;
- private RangePredicate rangePred;
- private MultiComparator lowKeySearchCmp;
- private MultiComparator highKeySearchCmp;
- private ITreeIndexCursor cursor;
- private ITreeIndexFrame cursorFrame;
- private ITreeIndexAccessor indexAccessor;
+ protected BTree btree;
+ protected boolean isForward;
+ protected PermutingFrameTupleReference lowKey;
+ protected PermutingFrameTupleReference highKey;
+ protected boolean lowKeyInclusive;
+ protected boolean highKeyInclusive;
+ protected RangePredicate rangePred;
+ protected MultiComparator lowKeySearchCmp;
+ protected MultiComparator highKeySearchCmp;
+ protected ITreeIndexCursor cursor;
+ protected ITreeIndexFrame cursorFrame;
+ protected ITreeIndexAccessor indexAccessor;
- private RecordDescriptor recDesc;
+ protected RecordDescriptor recDesc;
public BTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, int[] lowKeyFields,
@@ -91,7 +91,7 @@
accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
cursorFrame = opDesc.getTreeIndexLeafFactory().createFrame();
- cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame);
+ setCursor();
writer.open();
try {
@@ -116,14 +116,18 @@
}
}
- private void writeSearchResults() throws Exception {
+ protected void setCursor() {
+ cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, false);
+ }
+
+ protected void writeSearchResults() throws Exception {
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));
+ ITupleReference tuple = cursor.getTuple();
+ for (int i = 0; i < tuple.getFieldCount(); i++) {
+ dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
tb.addFieldEndOffset();
}
@@ -140,17 +144,17 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
accessor.reset(buffer);
-
int tupleCount = accessor.getTupleCount();
try {
for (int i = 0; i < tupleCount; i++) {
- if (lowKey != null)
+ if (lowKey != null) {
lowKey.reset(accessor, i);
- if (highKey != null)
+ }
+ if (highKey != null) {
highKey.reset(accessor, i);
+ }
rangePred.setLowKey(lowKey, lowKeyInclusive);
rangePred.setHighKey(highKey, highKeyInclusive);
-
cursor.reset();
indexAccessor.search(cursor, rangePred);
writeSearchResults();
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
new file mode 100644
index 0000000..ed4db73
--- /dev/null
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -0,0 +1,43 @@
+package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleUpdaterFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public class BTreeUpdateSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ private final ITupleUpdaterFactory tupleUpdaterFactory;
+
+ public BTreeUpdateSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
+ IStorageManagerInterface storageManager, IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
+ IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
+ ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+ IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
+ boolean lowKeyInclusive, boolean highKeyInclusive, ITreeIndexOpHelperFactory opHelperFactory,
+ ITupleUpdaterFactory tupleUpdaterFactory) {
+ super(spec, recDesc, storageManager, treeIndexRegistryProvider, fileSplitProvider, interiorFrameFactory,
+ leafFrameFactory, typeTraits, comparatorFactories, isForward, lowKeyFields, highKeyFields, lowKeyInclusive,
+ highKeyInclusive, opHelperFactory);
+ this.tupleUpdaterFactory = tupleUpdaterFactory;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
+ int partition, int nPartitions) {
+ return new BTreeUpdateSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, isForward, lowKeyFields,
+ highKeyFields, lowKeyInclusive, highKeyInclusive, tupleUpdaterFactory.createTupleUpdater());
+ }
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
new file mode 100644
index 0000000..06a42bd
--- /dev/null
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -0,0 +1,51 @@
+package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+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.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleUpdater;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
+
+public class BTreeUpdateSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable {
+ private final ITupleUpdater tupleUpdater;
+
+ public BTreeUpdateSearchOperatorNodePushable(
+ AbstractTreeIndexOperatorDescriptor opDesc,
+ IHyracksTaskContext ctx, int partition,
+ IRecordDescriptorProvider recordDescProvider, boolean isForward,
+ int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive,
+ boolean highKeyInclusive, ITupleUpdater tupleUpdater) {
+ super(opDesc, ctx, partition, recordDescProvider, isForward, lowKeyFields,
+ highKeyFields, lowKeyInclusive, highKeyInclusive);
+ this.tupleUpdater = tupleUpdater;
+ }
+
+ @Override
+ protected void setCursor() {
+ cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true);
+ }
+
+ @Override
+ protected void writeSearchResults() throws Exception {
+ while (cursor.hasNext()) {
+ tb.reset();
+ cursor.next();
+ ITupleReference tuple = cursor.getTuple();
+ tupleUpdater.updateTuple(tuple);
+ for (int i = 0; i < tuple.getFieldCount(); i++) {
+ dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(writeBuffer, writer);
+ appender.reset(writeBuffer, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+ }
+}
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 50e3c0b..57f1b79 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
@@ -628,19 +628,19 @@
}
}
- private final void acquireLatch(ICachedPage node, IndexOp op, boolean isLeaf) {
- if (isLeaf && (op == IndexOp.INSERT || op == IndexOp.DELETE || op == IndexOp.UPDATE)) {
- node.acquireWriteLatch();
- } else {
+ private final void acquireLatch(ICachedPage node, BTreeOpContext ctx, boolean isLeaf) {
+ if (!isLeaf || (ctx.op == IndexOp.SEARCH && !ctx.cursor.exclusiveLatchNodes())) {
node.acquireReadLatch();
+ } else {
+ node.acquireWriteLatch();
}
}
- private final void releaseLatch(ICachedPage node, IndexOp op, boolean isLeaf) {
- if (isLeaf && (op == IndexOp.INSERT || op == IndexOp.DELETE || op == IndexOp.UPDATE)) {
- node.releaseWriteLatch();
- } else {
+ private final void releaseLatch(ICachedPage node, BTreeOpContext ctx, boolean isLeaf) {
+ if (!isLeaf || (ctx.op == IndexOp.SEARCH && !ctx.cursor.exclusiveLatchNodes())) {
node.releaseReadLatch();
+ } else {
+ node.releaseWriteLatch();
}
}
@@ -665,7 +665,7 @@
// this check performs an unprotected read in the page
// the following could happen: TODO fill out
boolean unsafeIsLeaf = ctx.interiorFrame.isLeaf();
- acquireLatch(node, ctx.op, unsafeIsLeaf);
+ acquireLatch(node, ctx, unsafeIsLeaf);
boolean smFlag = ctx.interiorFrame.getSmFlag();
// re-check leafness after latching
boolean isLeaf = ctx.interiorFrame.isLeaf();
@@ -744,7 +744,7 @@
ctx.opRestarts++;
System.out.println("ONGOING SM ON PAGE " + pageId + " AT LEVEL " + ctx.interiorFrame.getLevel()
+ ", RESTARTS: " + ctx.opRestarts);
- releaseLatch(node, ctx.op, unsafeIsLeaf);
+ releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
// TODO: this should be an instant duration lock, how to do
@@ -785,20 +785,20 @@
}
} catch (TreeIndexException e) {
if (!ctx.exceptionHandled) {
- releaseLatch(node, ctx.op, unsafeIsLeaf);
+ releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
ctx.exceptionHandled = true;
}
throw e;
} catch (PageAllocationException e) {
if (!ctx.exceptionHandled) {
- releaseLatch(node, ctx.op, unsafeIsLeaf);
+ releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
ctx.exceptionHandled = true;
}
throw e;
} catch (Exception e) {
- releaseLatch(node, ctx.op, unsafeIsLeaf);
+ releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
BTreeException wrappedException = new BTreeException(e);
throw wrappedException;
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 3520ec9..8122314 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
@@ -33,7 +33,6 @@
private int fileId = -1;
private ICachedPage page = null;
- private IBTreeLeafFrame frame = null;
private IBufferCache bufferCache = null;
private int tupleIndex = 0;
@@ -46,7 +45,9 @@
private FindTupleNoExactMatchPolicy lowKeyFtp;
private FindTupleNoExactMatchPolicy highKeyFtp;
- private ITreeIndexTupleReference frameTuple;
+ private final IBTreeLeafFrame frame;
+ private final ITreeIndexTupleReference frameTuple;
+ private final boolean exclusiveLatchNodes;
private RangePredicate pred;
private MultiComparator lowKeyCmp;
@@ -54,20 +55,25 @@
private ITupleReference lowKey;
private ITupleReference highKey;
- public BTreeRangeSearchCursor(IBTreeLeafFrame frame) {
+ public BTreeRangeSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
this.frame = frame;
- this.frameTuple = frame.createTupleReference();
+ this.frameTuple = frame.createTupleReference();
+ this.exclusiveLatchNodes = exclusiveLatchNodes;
}
@Override
public void close() throws Exception {
if (page != null) {
- page.releaseReadLatch();
+ if (exclusiveLatchNodes) {
+ page.releaseWriteLatch();
+ } else {
+ page.releaseReadLatch();
+ }
bufferCache.unpin(page);
}
tupleIndex = 0;
- page = null;
- pred = null;
+ page = null;
+ pred = null;
}
public ITupleReference getTuple() {
@@ -81,18 +87,20 @@
private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException {
ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
- nextLeaf.acquireReadLatch();
-
- page.releaseReadLatch();
+ if (exclusiveLatchNodes) {
+ nextLeaf.acquireWriteLatch();
+ page.releaseWriteLatch();
+ } else {
+ nextLeaf.acquireReadLatch();
+ page.releaseReadLatch();
+ }
bufferCache.unpin(page);
-
page = nextLeaf;
- frame.setPage(page);
+ frame.setPage(page);
}
@Override
public boolean hasNext() throws Exception {
-
if (pred.isForward()) {
if (tupleIndex >= frame.getTupleCount()) {
int nextLeafPage = frame.getNextLeaf();
@@ -142,32 +150,31 @@
}
private int getLowKeyIndex() throws HyracksDataException {
- int index;
- if (lowKey == null)
- index = 0;
- else {
- index = frame.findTupleIndex(lowKey, frameTuple, lowKeyCmp, lowKeyFtm, lowKeyFtp);
- if (pred.lowKeyInclusive) {
- index++;
- } else {
- if (index < 0)
- index = frame.getTupleCount();
+ if (lowKey == null) {
+ return 0;
+ }
+ int index = frame.findTupleIndex(lowKey, frameTuple, lowKeyCmp, lowKeyFtm, lowKeyFtp);
+ if (pred.lowKeyInclusive) {
+ index++;
+ } else {
+ if (index < 0) {
+ index = frame.getTupleCount();
}
}
return index;
}
private int getHighKeyIndex() throws HyracksDataException {
- int index;
- if (highKey == null)
- index = frame.getTupleCount() - 1;
- else {
- index = frame.findTupleIndex(highKey, frameTuple, highKeyCmp, highKeyFtm, highKeyFtp);
- if (pred.highKeyInclusive) {
- if (index < 0)
- index = frame.getTupleCount() - 1;
- else
- index--;
+ if (highKey == null) {
+ return frame.getTupleCount() - 1;
+ }
+ int index = frame.findTupleIndex(highKey, frameTuple, highKeyCmp, highKeyFtm, highKeyFtp);
+ if (pred.highKeyInclusive) {
+ if (index < 0) {
+ index = frame.getTupleCount() - 1;
+ }
+ else {
+ index--;
}
}
return index;
@@ -177,7 +184,11 @@
public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
// in case open is called multiple times without closing
if (page != null) {
- page.releaseReadLatch();
+ if (exclusiveLatchNodes) {
+ page.releaseWriteLatch();
+ } else {
+ page.releaseReadLatch();
+ }
bufferCache.unpin(page);
}
@@ -220,10 +231,10 @@
@Override
public void reset() {
try {
- close();
- } catch (Exception e) {
- e.printStackTrace();
- }
+ close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
}
@Override
@@ -235,4 +246,9 @@
public void setFileId(int fileId) {
this.fileId = fileId;
}
+
+ @Override
+ public boolean exclusiveLatchNodes() {
+ return exclusiveLatchNodes;
+ }
}
\ No newline at end of file
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 fdfedb2..d3ce3867 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
@@ -39,4 +39,7 @@
public void setFileId(int fileId);
public ITupleReference getTuple();
+
+ // For allowing updates.
+ public boolean exclusiveLatchNodes();
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdater.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdater.java
new file mode 100644
index 0000000..e201cc3
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdater.java
@@ -0,0 +1,29 @@
+/*
+ * 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.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Interface for updating a tuple. Warning: By convention, this interface
+ * assumes that the modifications do not change the size of the tuple, and that
+ * it does not change keys (e.g., BTree keys). This interface is used to
+ * implement update scans.
+ *
+ */
+public interface ITupleUpdater {
+ public void updateTuple(ITupleReference tuple);
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdaterFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdaterFactory.java
new file mode 100644
index 0000000..ee20b6c
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdaterFactory.java
@@ -0,0 +1,22 @@
+/*
+ * 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 java.io.Serializable;
+
+public interface ITupleUpdaterFactory extends Serializable {
+ public ITupleUpdater createTupleUpdater();
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java
index 18d60ec..1a81c3f 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java
@@ -31,12 +31,12 @@
private int fileId = -1;
private int currentPageId = -1;
private int maxPageId = -1;
- private ICachedPage page = null;
- private ITreeIndexFrame frame = null;
+ private ICachedPage page = null;
private IBufferCache bufferCache = null;
- private ITreeIndexTupleReference frameTuple;
-
+ private final ITreeIndexFrame frame;
+ private final ITreeIndexTupleReference frameTuple;
+
public TreeDiskOrderScanCursor(ITreeIndexFrame frame) {
this.frame = frame;
this.frameTuple = frame.createTupleReference();
@@ -145,4 +145,9 @@
public void setMaxPageId(int maxPageId) {
this.maxPageId = maxPageId;
}
+
+ @Override
+ public boolean exclusiveLatchNodes() {
+ return false;
+ }
}
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 63efd5f..dd09ef7 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
@@ -94,7 +94,7 @@
leafFrame = invIndex.getBTree().getLeafFrameFactory().createFrame();
interiorFrame = invIndex.getBTree().getInteriorFrameFactory().createFrame();
- btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame);
+ btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
ITypeTrait[] invListFields = invIndex.getTypeTraits();
invListFieldsWithCount = new TypeTrait[invListFields.length + 1];
int tmp = 0;
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 82f07ad..e65f1d4 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
@@ -219,4 +219,9 @@
public void setFileId(int fileId) {
this.fileId = fileId;
}
+
+ @Override
+ public boolean exclusiveLatchNodes() {
+ return false;
+ }
}
\ No newline at end of file
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 e5ed5c2..291263e 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
@@ -206,7 +206,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("ORDERED SCAN:");
}
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, false);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
indexAccessor.search(scanCursor, nullPred);
try {
@@ -250,7 +250,7 @@
LOGGER.info("RANGE SEARCH:");
}
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
// build low and high keys
ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
@@ -427,7 +427,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("ORDERED SCAN:");
}
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, false);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
indexAccessor.search(scanCursor, nullPred);
try {
@@ -449,7 +449,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("RANGE SEARCH:");
}
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
// build low and high keys
ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
@@ -616,7 +616,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("ORDERED SCAN:");
}
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, false);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
indexAccessor.search(scanCursor, nullPred);
try {
@@ -639,7 +639,7 @@
LOGGER.info("RANGE SEARCH:");
}
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
// build low and high keys
ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
@@ -875,7 +875,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("ORDERED SCAN:");
}
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, false);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
ITreeIndexAccessor indexAccessor = btree.createAccessor();
indexAccessor.search(scanCursor, nullPred);
@@ -1025,7 +1025,7 @@
e.printStackTrace();
}
}
- ITreeIndexCursor insertCheckCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor insertCheckCursor = new BTreeRangeSearchCursor(leafFrame, false);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
indexAccessor.search(insertCheckCursor, nullPred);
try {
@@ -1074,7 +1074,7 @@
}
}
- ITreeIndexCursor updateCheckCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor updateCheckCursor = new BTreeRangeSearchCursor(leafFrame, false);
indexAccessor.search(updateCheckCursor, nullPred);
try {
compareActualAndExpected(updateCheckCursor, expectedValues, fieldSerdes);
@@ -1192,7 +1192,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("RANGE SEARCH:");
}
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame);
+ ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
// build low and high keys
ArrayTupleBuilder ktb = new ArrayTupleBuilder(1);
@@ -1391,7 +1391,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("ORDERED SCAN:");
}
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, false);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
indexAccessor.search(scanCursor, nullPred);
try {
@@ -1412,7 +1412,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("RANGE SEARCH:");
}
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
// build low and high keys
ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
@@ -1477,6 +1477,231 @@
bufferCache.close();
}
+ // Update scan test on fixed-length keys.
+ // create a B-tree with one fixed-length "key" field and one fixed-length
+ // "value" field
+ // fill B-tree with random values using insertions (not bulk load)
+ // perform ordered scan and range search
+ @Test
+ public void test08() throws Exception {
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("FIXED-LENGTH KEY TEST");
+ }
+
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+ IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+ FileReference file = new FileReference(new File(fileName));
+ bufferCache.createFile(file);
+ int fileId = fmp.lookupFileId(file);
+ bufferCache.openFile(fileId);
+
+ // declare fields
+ int fieldCount = 2;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ typeTraits[0] = new TypeTrait(4);
+ typeTraits[1] = new TypeTrait(4);
+
+ // declare keys
+ int keyFieldCount = 1;
+ IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+ cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
+
+ TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+ IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
+ ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
+
+ BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ btree.create(fileId);
+ btree.open(fileId);
+
+ Random rnd = new Random();
+ rnd.setSeed(50);
+
+ long start = System.currentTimeMillis();
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("INSERTING INTO TREE");
+ }
+
+ ByteBuffer frame = ctx.allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ DataOutput dos = tb.getDataOutput();
+
+ ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE };
+ RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+ accessor.reset(frame);
+ FrameTupleReference tuple = new FrameTupleReference();
+
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+
+ // 10000
+ for (int i = 0; i < 10000; i++) {
+
+ int f0 = rnd.nextInt() % 10000;
+ int f1 = 5;
+
+ tb.reset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(f0, dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(f1, dos);
+ tb.addFieldEndOffset();
+
+ appender.reset(frame, true);
+ appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
+
+ tuple.reset(accessor, 0);
+
+ ArrayTupleReference t = new ArrayTupleReference();
+ t.reset(tb.getFieldEndOffsets(), tb.getByteArray());
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ long end = System.currentTimeMillis();
+ LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+ }
+ }
+
+ try {
+ indexAccessor.insert(t);
+ } catch (TreeIndexException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ // btree.printTree(leafFrame, interiorFrame);
+
+ int maxPage = btree.getFreePageManager().getMaxPage(metaFrame);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("MAXPAGE: " + maxPage);
+ }
+
+ long end = System.currentTimeMillis();
+ long duration = end - start;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DURATION: " + duration);
+ }
+
+ // ordered scan
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("ORDERED SCAN:");
+ }
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, false);
+ RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
+ indexAccessor.search(scanCursor, nullPred);
+ try {
+ while (scanCursor.hasNext()) {
+ scanCursor.next();
+ ITupleReference frameTuple = scanCursor.getTuple();
+ String rec = TupleUtils.printTuple(frameTuple, recDescSers);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ scanCursor.close();
+ }
+
+ // disk-order scan
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DISK-ORDER SCAN:");
+ }
+ TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
+ indexAccessor.diskOrderScan(diskOrderCursor);
+ try {
+ while (diskOrderCursor.hasNext()) {
+ diskOrderCursor.next();
+ ITupleReference frameTuple = diskOrderCursor.getTuple();
+ String rec = TupleUtils.printTuple(frameTuple, recDescSers);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ diskOrderCursor.close();
+ }
+
+ // range search in [-1000, 1000]
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("RANGE SEARCH:");
+ }
+
+ ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
+
+ // build low and high keys
+ ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
+ DataOutput kdos = ktb.getDataOutput();
+
+ ISerializerDeserializer[] keyDescSers = { IntegerSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
+ IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
+ keyAccessor.reset(frame);
+
+ appender.reset(frame, true);
+
+ // build and append low key
+ ktb.reset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(-1000, kdos);
+ ktb.addFieldEndOffset();
+ appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
+
+ // build and append high key
+ ktb.reset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(1000, kdos);
+ ktb.addFieldEndOffset();
+ appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
+
+ // create tuplereferences for search keys
+ FrameTupleReference lowKey = new FrameTupleReference();
+ lowKey.reset(keyAccessor, 0);
+
+ FrameTupleReference highKey = new FrameTupleReference();
+ highKey.reset(keyAccessor, 1);
+
+ IBinaryComparator[] searchCmps = new IBinaryComparator[1];
+ searchCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ MultiComparator searchCmp = new MultiComparator(searchCmps);
+
+ RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
+ indexAccessor.search(rangeCursor, rangePred);
+
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ ITupleReference frameTuple = rangeCursor.getTuple();
+ String rec = TupleUtils.printTuple(frameTuple, recDescSers);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ rangeCursor.close();
+ }
+
+ btree.close();
+ bufferCache.closeFile(fileId);
+ bufferCache.close();
+ }
+
public static String randomString(int length, Random random) {
String s = Long.toHexString(Double.doubleToLongBits(random.nextDouble()));
StringBuilder strBuilder = new StringBuilder();
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 79134ae..922c715 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
@@ -411,7 +411,7 @@
int lowKey = i;
int highKey = j;
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
RangePredicate rangePred = createRangePredicate(lowKey,
highKey, isForward, lowKeyInclusive, highKeyInclusive,
btree.getMultiComparator());
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java
new file mode 100644
index 0000000..7a61adc
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java
@@ -0,0 +1,152 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import java.util.Random;
+import java.util.logging.Level;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+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.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.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.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.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
+
+public class UpdateSearchTest extends AbstractBTreeTest {
+
+ // Update scan test on fixed-length tuples.
+ @Test
+ public void test01() throws Exception {
+ // declare fields
+ int fieldCount = 2;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ typeTraits[0] = new TypeTrait(4);
+ typeTraits[1] = new TypeTrait(4);
+
+ // declare keys
+ int keyFieldCount = 1;
+ IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+ cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
+
+ ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE };
+
+ TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
+ BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
+
+ Random rnd = new Random();
+ rnd.setSeed(50);
+
+ long start = System.currentTimeMillis();
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("INSERTING INTO TREE");
+ }
+
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ ArrayTupleReference insertTuple = new ArrayTupleReference();
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+
+ int numInserts = 10000;
+ for (int i = 0; i < 10000; i++) {
+ int f0 = rnd.nextInt() % 10000;
+ int f1 = 5;
+ TupleUtils.createIntegerTuple(tb, insertTuple, f0, f1);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 10000 == 0) {
+ long end = System.currentTimeMillis();
+ LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+ }
+ }
+
+ try {
+ indexAccessor.insert(insertTuple);
+ } catch (TreeIndexException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ long end = System.currentTimeMillis();
+ long duration = end - start;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DURATION: " + duration);
+ }
+
+ // Update scan.
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("UPDATE SCAN:");
+ }
+ // Set the cursor to X latch nodes.
+ ITreeIndexCursor updateScanCursor = new BTreeRangeSearchCursor(leafFrame, true);
+ RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
+ indexAccessor.search(updateScanCursor, nullPred);
+ try {
+ while (updateScanCursor.hasNext()) {
+ updateScanCursor.next();
+ ITupleReference tuple = updateScanCursor.getTuple();
+ // Change the value field.
+ IntegerSerializerDeserializer.putInt(10, tuple.getFieldData(1), tuple.getFieldStart(1));
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ updateScanCursor.close();
+ }
+
+ // Ordered scan to verify the values.
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("ORDERED SCAN:");
+ }
+ // Set the cursor to X latch nodes.
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, true);
+ indexAccessor.search(scanCursor, nullPred);
+ try {
+ while (scanCursor.hasNext()) {
+ scanCursor.next();
+ ITupleReference tuple = scanCursor.getTuple();
+ String rec = TupleUtils.printTuple(tuple, recDescSers);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ scanCursor.close();
+ }
+ btree.close();
+ }
+}
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 8545d14..ca543a2 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
@@ -105,7 +105,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Testing Ordered Scan.");
}
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(testCtx.leafFrame, false);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
testCtx.indexAccessor.search(scanCursor, nullPred);
Iterator<CheckTuple> checkIter = testCtx.checkTuples.iterator();
@@ -163,7 +163,7 @@
}
MultiComparator lowKeyCmp = BTreeUtils.getSearchMultiComparator(testCtx.btree.getMultiComparator(), lowKey);
MultiComparator highKeyCmp = BTreeUtils.getSearchMultiComparator(testCtx.btree.getMultiComparator(), highKey);
- ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
+ ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame, false);
RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeyCmp, highKeyCmp);
testCtx.indexAccessor.search(searchCursor, rangePred);
// Get the subset of elements from the expected set within given key range.
@@ -203,7 +203,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Testing Point Searches On All Expected Keys.");
}
- ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
+ ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame, false);
ArrayTupleBuilder lowKeyBuilder = new ArrayTupleBuilder(testCtx.btree.getMultiComparator().getKeyFieldCount());
ArrayTupleReference lowKey = new ArrayTupleReference();
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 0d55dec..bd22d0d 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
@@ -225,7 +225,7 @@
// ------- START VERIFICATION -----------
- ITreeIndexCursor btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame);
+ ITreeIndexCursor btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
FrameTupleReference searchKey = new FrameTupleReference();
RangePredicate btreePred = new RangePredicate(true, searchKey, searchKey, true, true, btreeCmp, btreeCmp);