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);