You are viewing a plain text version of this content. The canonical link for it is here.
Posted to derby-commits@db.apache.org by rh...@apache.org on 2013/10/23 14:15:26 UTC

svn commit: r1535001 - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/store/ engine/org/apache/derby/iapi/store/access/ engine/org/apache/derby/iapi/types/ engine/org/apache/derby/impl/sql/execute/ engine/org/apache/derby/impl/store/access/...

Author: rhillegas
Date: Wed Oct 23 12:15:25 2013
New Revision: 1535001

URL: http://svn.apache.org/r1535001
Log:
DERBY-3155: Add (optional) row locations to backing hash tables: derby-3155-03-ag-backingStoreHashtableWithRowLocation.diff.

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/LocatedRow.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/build.xml
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DistinctScanResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/HashScanResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoPutResultSetImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScrollInsensitiveResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/BackingStoreHashTableFromScan.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeForwardScan.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/GenericScanController.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapScan.java
    db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_QualifierTest.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java Wed Oct 23 12:15:25 2013
@@ -24,9 +24,13 @@ package org.apache.derby.iapi.store.acce
 import org.apache.derby.iapi.error.StandardException; 
 
 import org.apache.derby.iapi.types.DataValueDescriptor;
+import org.apache.derby.iapi.types.LocatedRow;
+import org.apache.derby.iapi.types.RowLocation;
 
 import org.apache.derby.iapi.services.cache.ClassSize;
 
+import org.apache.derby.shared.common.sanity.SanityManager;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Enumeration;
@@ -37,14 +41,18 @@ import java.util.Properties; 
 import java.util.NoSuchElementException;
 
 /**
+<p>
 A BackingStoreHashtable is a utility class which will store a set of rows into
 an in memory hash table, or overflow the hash table to a tempory on disk 
 structure.
+</p>
+
 <p>
 All rows must contain the same number of columns, and the column at position
 N of all the rows must have the same format id.  If the BackingStoreHashtable needs to be
 overflowed to disk, then an arbitrary row will be chosen and used as a template
 for creating the underlying overflow container.
+</p>
 
 <p>
 The hash table will be built logically as follows (actual implementation
@@ -53,13 +61,15 @@ java hash value on the row[key_column_nu
 or row[key_column_numbers[0, 1, ...]] if key_column_numbers.length > 1, 
 and that duplicate detection is done by the standard java duplicate detection provided by 
 java.util.Hashtable.
-<p>
+</p>
+
 <pre>
 import java.util.Hashtable;
 
 hash_table = new Hashtable();
 
-Object[] row;
+Object row; // is a DataValueDescriptor[] or a LocatedRow
+
 boolean  needsToClone = rowSource.needsToClone();
 
 while((row = rowSource.getNextRowFromRowSource()) != null)
@@ -97,6 +107,27 @@ while((row = rowSource.getNextRowFromRow
 }
 </pre>
 
+<p>
+What actually goes into the hash table is a little complicated. That is because
+the row may either be an array of column values (i.e. DataValueDescriptor[])
+or a LocatedRow (i.e., a structure holding the columns plus a RowLocation).
+In addition, the hash value itself may either be one of these rows or
+(in the case of multiple rows which hash to the same value) a bucket (List)
+of rows. To sum this up, the values in a hash table which does not spill
+to disk may be the following:
+</p>
+
+<ul>
+<li>DataValueDescriptor[] and ArrayList<DataValueDescriptor></li>
+<li>or LocatedRow and ArrayList<LocatedRow></li>
+</ul>
+
+<p>
+If rows spill to disk, then they just become arrays of columns. In this case,
+a LocatedRow becomes a DataValueDescriptor[], where the last cell contains
+the RowLocation.
+</p>
+
 **/
 
 public class BackingStoreHashtable
@@ -141,7 +172,9 @@ public class BackingStoreHashtable
      * <p>
      * This routine drains the RowSource.  The performance characteristics
      * depends on the number of rows inserted and the parameters to the 
-     * constructor.  
+     * constructor. RowLocations are supported iff row_source is null.
+     * RowLocations in a non-null row_source can be added later
+     * if there is a use-case that stresses this behavior.
      * <p>
      * If the number of rows is <= "max_inmemory_rowcnt", then the rows are
      * inserted into a java.util.HashMap. In this case no
@@ -204,13 +237,28 @@ public class BackingStoreHashtable
 		this.row_source			   = row_source;
 		this.skipNullKeyColumns	   = skipNullKeyColumns;
         this.max_inmemory_rowcnt = max_inmemory_rowcnt;
-        if( max_inmemory_rowcnt > 0)
+        if ( max_inmemory_rowcnt > 0)
+        {
             max_inmemory_size = Long.MAX_VALUE;
+        }
         else
+        {
             max_inmemory_size = Runtime.getRuntime().totalMemory()/100;
+        }
         this.tc = tc;
         this.keepAfterCommit = keepAfterCommit;
 
+        if (SanityManager.DEBUG)
+        {
+            // RowLocations are not currently supported if the
+            // hash table is being filled from a non-null
+            // row_source arg.
+            if ( row_source != null )
+            {
+                SanityManager.ASSERT( !includeRowLocations() );
+            }
+        }
+
         // use passed in capacity and loadfactor if not -1, you must specify
         // capacity if you want to specify loadfactor.
         if (initialCapacity != -1)
@@ -280,7 +328,7 @@ public class BackingStoreHashtable
                         new HashMap<Object,Object>((int)(max_inmemory_size / rowUsage));
                 }
                
-                add_row_to_hash_table(row, needsToClone);
+                add_row_to_hash_table(row, null, needsToClone);
             }
         }
 
@@ -292,7 +340,18 @@ public class BackingStoreHashtable
         // BackingStoreHashtable (ex. "size()") will have a working hash_table
         // on which to operate.
         if (hash_table == null)
+        {
             hash_table = new HashMap<Object,Object>();
+        }
+    }
+
+    /**
+     * Return true if we should include RowLocations with the rows
+     * stored in this hash table.
+     */
+    public  boolean includeRowLocations()
+    {
+        return false;
     }
 
     /**************************************************************************
@@ -302,7 +361,9 @@ public class BackingStoreHashtable
 
 	/**
 	 * Call method to either get next row or next row with non-null
-	 * key columns.
+	 * key columns. Currently, RowLocation information is not included in
+     * rows siphoned out of a RowSource. That functionality is only supported
+     * if the rows come from the scan of a base table.
 	 *
      *
 	 * @exception  StandardException  Standard exception policy.
@@ -354,8 +415,10 @@ public class BackingStoreHashtable
         // Beetle 4896.
         for (int i = 0; i < old_row.length; i++)
         {
-            if( old_row[i] != null)
+            if ( old_row[i] != null)
+            {
                 new_row[i] = old_row[i].cloneValue(false);
+            }
         }
 
         return(new_row);
@@ -378,8 +441,10 @@ public class BackingStoreHashtable
         // DERBY-802
         for (int i = 0; i < old_row.length; i++)
         {
-            if( old_row[i] != null)
+            if ( old_row[i] != null)
+            {
                 new_row[i] = old_row[i].cloneHolder();
+            }
         }
 
         return(new_row);
@@ -389,27 +454,39 @@ public class BackingStoreHashtable
      * Do the work to add one row to the hash table.
      * <p>
      *
-     * @param row               Row to add to the hash table.
+     * @param columnValues               Row to add to the hash table.
+     * @param rowLocation   Location of row in conglomerate; could be null.
      * @param needsToClone      If the row needs to be cloned
      *
 	 * @exception  StandardException  Standard exception policy.
      **/
-    private void add_row_to_hash_table(DataValueDescriptor[] row, boolean needsToClone)
+    private void add_row_to_hash_table
+        (
+         DataValueDescriptor[] columnValues,
+         RowLocation rowLocation,
+         boolean needsToClone
+         )
 		throws StandardException
     {
-        if (spillToDisk(row))
+        if (spillToDisk( columnValues, rowLocation ))
+        {
             return;
+        }
         
         if (needsToClone)
         {
-            row = cloneRow(row);
+            columnValues = cloneRow( columnValues );
         }
 
-        Object key = KeyHasher.buildHashKey(row, key_column_numbers);
-        Object duplicate_value = hash_table.put(key, row);
+        Object key = KeyHasher.buildHashKey( columnValues, key_column_numbers );
+        Object hashValue = !includeRowLocations() ?
+            columnValues : new LocatedRow( columnValues, rowLocation );
+        Object duplicate_value = hash_table.put( key, hashValue );
 
         if (duplicate_value == null)
-            doSpaceAccounting( row, false);
+        {
+            doSpaceAccounting( hashValue, false );
+        }
         else
         {
             if (!remove_duplicates)
@@ -419,7 +496,7 @@ public class BackingStoreHashtable
                 // inserted a duplicate
                 if (duplicate_value instanceof RowList)
                 {
-                    doSpaceAccounting( row, false);
+                    doSpaceAccounting( hashValue, false );
                     row_vec = (RowList) duplicate_value;
                 }
                 else
@@ -428,12 +505,12 @@ public class BackingStoreHashtable
                     row_vec = new RowList(2);
 
                     // insert original row into vector
-                    row_vec.add((DataValueDescriptor[]) duplicate_value);
-                    doSpaceAccounting( row, true);
+                    row_vec.add( duplicate_value );
+                    doSpaceAccounting( hashValue, true );
                 }
 
                 // insert new row into list
-                row_vec.add(row);
+                row_vec.add( hashValue );
 
                 // store list of rows back into hash table,
                 // overwriting the duplicate key that was 
@@ -443,56 +520,81 @@ public class BackingStoreHashtable
         }
     }
 
-    private void doSpaceAccounting(DataValueDescriptor[] row,
+    private void doSpaceAccounting(Object hashValue,
                                     boolean firstDuplicate)
     {
         inmemory_rowcnt++;
-        if( max_inmemory_rowcnt <= 0)
+        if ( max_inmemory_rowcnt <= 0)
         {
-            max_inmemory_size -= getEstimatedMemUsage(row);
-            if( firstDuplicate)
+            max_inmemory_size -= getEstimatedMemUsage( hashValue );
+            if ( firstDuplicate)
+            {
                 max_inmemory_size -= ARRAY_LIST_SIZE;
+            }
         }
     } // end of doSpaceAccounting
 
     /**
      * Determine whether a new row should be spilled to disk and, if so, do it.
      *
-     * @param row
+     * @param columnValues  Actual columns from source row.
+     * @param rowLocation       Optional row location.
      *
      * @return true if the row was spilled to disk, false if not
      *
      * @exception  StandardException  Standard exception policy.
      */
-    private boolean spillToDisk(DataValueDescriptor[] row) throws StandardException {
+    private boolean spillToDisk
+        (
+         DataValueDescriptor[] columnValues,
+         RowLocation rowLocation
+         )
+        throws StandardException
+    {
         // Once we have started spilling all new rows will go to disk, even if we have freed up some
         // memory by moving duplicates to disk. This simplifies handling of duplicates and accounting.
-        if( diskHashtable == null)
+
+        DataValueDescriptor[]   diskRow = null;
+        
+        if ( diskHashtable == null)
         {
-            if( max_inmemory_rowcnt > 0)
+            if ( max_inmemory_rowcnt > 0)
             {
-                if( inmemory_rowcnt < max_inmemory_rowcnt)
+                if ( inmemory_rowcnt < max_inmemory_rowcnt)
+                {
                     return false; // Do not spill
+                }
             }
-            else if( max_inmemory_size > getEstimatedMemUsage(row))
-                
+            else if
+                (
+                 max_inmemory_size >
+                 getEstimatedMemUsage
+                 (
+                  !includeRowLocations() ?
+                  columnValues : new LocatedRow( columnValues, rowLocation )
+                 )
+                )
+            {
                 return false;
+            }
+            
             // Want to start spilling
+            diskRow = makeDiskRow( columnValues, rowLocation );
  
             diskHashtable = 
                 new DiskHashtable(
                        tc,
-                       row,
+                       diskRow,
                        (int[]) null, //TODO-COLLATION, set non default collation if necessary.
                        key_column_numbers,
                        remove_duplicates,
                        keepAfterCommit);
         }
-        Object key = KeyHasher.buildHashKey(row, key_column_numbers);
+        Object key = KeyHasher.buildHashKey( columnValues, key_column_numbers );
         Object duplicateValue = hash_table.get( key);
-        if( duplicateValue != null)
+        if ( duplicateValue != null)
         {
-            if( remove_duplicates)
+            if ( remove_duplicates)
                 return true; // a degenerate case of spilling
             // If we are keeping duplicates then move all the duplicates from memory to disk
             // This simplifies finding duplicates: they are either all in memory or all on disk.
@@ -501,35 +603,153 @@ public class BackingStoreHashtable
                 List duplicateVec = (List) duplicateValue;
                 for( int i = duplicateVec.size() - 1; i >= 0; i--)
                 {
-                    DataValueDescriptor[] dupRow = (DataValueDescriptor[]) duplicateVec.get(i);
-                    diskHashtable.put( key, dupRow);
+                    diskHashtable.put
+                        ( key, makeDiskRow( duplicateVec.get( i ) ));
                 }
             }
             else
-                diskHashtable.put( key, (DataValueDescriptor[]) duplicateValue);
+            {
+                diskHashtable.put( key, makeDiskRow( duplicateValue ) );
+            }
             hash_table.remove( key);
         }
-        diskHashtable.put( key, row);
+
+        if ( diskRow == null )
+        { diskRow = makeDiskRow( columnValues, rowLocation ); }
+        
+        diskHashtable.put( key, diskRow );
         return true;
     } // end of spillToDisk
 
     /**
-     * Take a row and return an estimate as to how much memory that
-     * row will consume.
+     * <p>
+     * Make a full set of columns from an object which is either already
+     * an array of column or otherwise a LocatedRow. The full set of columns
+     * is what's stored on disk when we spill to disk. This is the inverse of
+     * makeInMemoryRow().
+     * </p>
+     */
+    private DataValueDescriptor[]   makeDiskRow( Object raw )
+    {
+        DataValueDescriptor[]   allColumns = null;
+        if ( includeRowLocations() )
+        {
+            LocatedRow  locatedRow = (LocatedRow) raw;
+            allColumns = makeDiskRow
+                ( locatedRow.columnValues(), locatedRow.rowLocation() );
+        }
+        else { allColumns = (DataValueDescriptor[]) raw; }
+
+        return allColumns;
+    }
+
+    /**
+     * <p>
+     * Turn a list of disk rows into a list of in-memory rows. The on disk
+     * rows are always of type DataValueDescriptor[]. But the in-memory rows
+     * could be of type LocatedRow.
+     * </p>
+     */
+    private List    makeInMemoryRows( List diskRows )
+    {
+        if ( !includeRowLocations() )
+        {
+            return diskRows;
+        }
+        else
+        {
+            ArrayList<Object>   result = new ArrayList<Object>();
+            for ( Object diskRow : diskRows )
+            {
+                result.add
+                    ( makeInMemoryRow( (DataValueDescriptor[]) diskRow ) );
+            }
+
+            return result;
+        }
+    }
+
+    /**
+     * <p>
+     * Make an in-memory row from an on-disk row. This is the inverse
+     * of makeDiskRow().
+     * </p>
+     */
+    private Object  makeInMemoryRow( DataValueDescriptor[] diskRow )
+    {
+        if ( !includeRowLocations() )
+        {
+            return diskRow;
+        }
+        else
+        {
+            return new LocatedRow( diskRow );
+        }
+    }
+
+    /**
+     * <p>
+     * Construct a full set of columns, which may need to end
+     * with the row location.The full set of columns is what's
+     * stored on disk when we spill to disk.
+     * </p>
+     */
+    private DataValueDescriptor[]   makeDiskRow
+        ( DataValueDescriptor[] columnValues, RowLocation rowLocation )
+    {
+        if ( !includeRowLocations() )
+        {
+            return columnValues;
+        }
+        else
+        {
+            return LocatedRow.flatten( columnValues, rowLocation );
+        }
+    }
+
+    /**
+     * Take a value which will go into the hash table and return an estimate
+     * of how much memory that value will consume. The hash value could
+     * be either an array of columns or a LocatedRow.
      * 
-     * @param row The row for which we want to know the memory usage.
-     * @return A guess as to how much memory the current row will
+     * @param hashValue The object for which we want to know the memory usage.
+     * @return A guess as to how much memory the current hash value will
      *  use.
      */
-    private long getEstimatedMemUsage(DataValueDescriptor[] row)
+    private long getEstimatedMemUsage( Object hashValue )
     {
         long rowMem = 0;
+        DataValueDescriptor[] row = null;
+
+        if ( hashValue instanceof DataValueDescriptor[] )
+        {
+            row = (DataValueDescriptor[]) hashValue;
+        }
+        else
+        {
+            LocatedRow  locatedRow = (LocatedRow) hashValue;
+            row = locatedRow.columnValues();
+
+            // account for the RowLocation size and class overhead
+            RowLocation rowLocation = locatedRow.rowLocation();
+            if ( rowLocation != null )
+            {
+                rowMem += locatedRow.rowLocation().estimateMemoryUsage();
+                rowMem += ClassSize.refSize;
+            }
+
+            // account for class overhead of the LocatedRow itself
+            rowMem += ClassSize.refSize;
+        }
+        
         for( int i = 0; i < row.length; i++)
         {
+            // account for the column's size and class overhead
             rowMem += row[i].estimateMemoryUsage();
             rowMem += ClassSize.refSize;
         }
 
+        // account for the class overhead of the array itself
         rowMem += ClassSize.refSize;
         return rowMem;
     }
@@ -553,7 +773,7 @@ public class BackingStoreHashtable
 		throws StandardException
     {
         hash_table = null;
-        if( diskHashtable != null)
+        if ( diskHashtable != null)
         {
             diskHashtable.close();
             diskHashtable = null;
@@ -562,10 +782,44 @@ public class BackingStoreHashtable
     }
 
     /**
-     * Return an Enumeration that can be used to scan entire table.
+     * <p>
+     * Return an Enumeration that can be used to scan the entire table. The objects
+     * in the Enumeration can be either of the following:
+     * </p>
+     *
+     * <ul>
+     * <li>a row - This is a single row with a unique hash key.</li>
+     * <li>or a bucket of rows - This is a list of rows which all have the same hash key.</li>
+     * </ul>
+     *
+     * <p>
+     * The situation is a little more complicated because the row representation
+     * is different depending on whether the row includes a RowLocation.
+     * If includeRowLocations()== true, then the row is a LocatedRow. Otherwise,
+     * the row is an array of DataValueDescriptor. Putting all of this together,
+     * if the row contains a RowLocation, then the objects in the Enumeration returned
+     * by this method can be either of the following:
+     * </p>
+     *
+     * <ul>
+     * <li>a LocatedRow</li>
+     * <li>or a List&lt;LocatedRow&gt;</li>
+     * </ul>
+     *
+     * <p>
+     * But if the row does not contain a RowLocation, then the objects in the
+     * Enumeration returned by this method can be either of the following:
+     * </p>
+     *
+     * <ul>
+     * <li>a DataValueDescriptor[]</li>
+     * <li>or a List&lt;DataValueDescriptor[]&gt;</li>
+     * </ul>
+     *
      * <p>
      * RESOLVE - is it worth it to support this routine when we have a
      *           disk overflow hash table?
+     * </p>
      *
 	 * @return The Enumeration.
      *
@@ -574,15 +828,22 @@ public class BackingStoreHashtable
     public Enumeration<Object> elements()
         throws StandardException
     {
-        if( diskHashtable == null)
+        if ( diskHashtable == null)
+        {
             return Collections.enumeration(hash_table.values());
+        }
         return new BackingStoreHashtableEnumeration();
     }
 
     /**
-     * get data associated with given key.
+     * <p>
+     * Get data associated with given key.
+     * </p>
+     *
      * <p>
      * There are 2 different types of objects returned from this routine.
+     * </p>
+     *
      * <p>
 	 * In both cases, the key value is either the object stored in 
      * row[key_column_numbers[0]], if key_column_numbers.length is 1, 
@@ -590,13 +851,48 @@ public class BackingStoreHashtable
 	 * the objects stored in row[key_column_numbers[0, 1, ...]].
      * For every qualifying unique row value an entry is placed into the 
      * hash table.
+     * </p>
+     *
      * <p>
      * For row values with duplicates, the value of the data is a list of
      * rows.
+     * </p>
+     *
+     * <p>
+     * The situation is a little more complicated because the row representation
+     * is different depending on whether the row includes a RowLocation.
+     * If includeRowLocations() == true, then the row is a LocatedRow. Otherwise,
+     * the row is an array of DataValueDescriptor. Putting all of this together,
+     * if the row contains a RowLocation, then the objects returned by this method
+     * can be either of the following:
+     * </p>
+     *
+     * <ul>
+     * <li>a LocatedRow</li>
+     * <li>or a List&lt;LocatedRow&gt;</li>
+     * </ul>
+     *
+     * <p>
+     * But if the row does not contain a RowLocation, then the objects
+     * returned by this method can be either of the following:
+     * </p>
+     *
+     * <ul>
+     * <li>a DataValueDescriptor[]</li>
+     * <li>or a List&lt;DataValueDescriptor[]&gt;</li>
+     * </ul>
+     *
      * <p>
      * The caller will have to call "instanceof" on the data value
      * object if duplicates are expected, to determine if the data value
      * of the hash table entry is a row or is a list of rows.
+     * </p>
+     *
+     * <p>
+     * See the javadoc for elements() for more information on the objects
+     * returned by this method.
+     * </p>
+     *
      * <p>
      * The BackingStoreHashtable "owns" the objects returned from the get()
      * routine.  They remain valid until the next access to the 
@@ -605,6 +901,7 @@ public class BackingStoreHashtable
      * BackingStoreHashtable can place all rows into a disk based conglomerate,
      * declare a row buffer and then reuse that row buffer for every get()
      * call.
+     * </p>
      *
 	 * @return The value to which the key is mapped in this hashtable; 
      *         null if the key is not mapped to any value in this hashtable.
@@ -617,9 +914,24 @@ public class BackingStoreHashtable
 		throws StandardException
     {
         Object obj = hash_table.get(key);
-        if( diskHashtable == null || obj != null)
+        if ( diskHashtable == null || obj != null)
+        {
             return obj;
-        return diskHashtable.get( key);
+        }
+
+        Object  diskHashtableValue = diskHashtable.get( key );
+        if ( diskHashtableValue == null )
+        { return null; }
+
+        if ( diskHashtableValue instanceof List )
+        {
+            return makeInMemoryRows( (List) diskHashtableValue );
+        }
+        else
+        {
+            return makeInMemoryRow
+                ( (DataValueDescriptor[]) diskHashtableValue );
+        }
     }
 
     /**
@@ -634,7 +946,10 @@ public class BackingStoreHashtable
 		throws StandardException
     {
         if (auxillary_runtimestats != null)
-            org.apache.derby.iapi.util.PropertyUtil.copyProperties(auxillary_runtimestats, prop);
+        {
+            org.apache.derby.iapi.util.PropertyUtil.copyProperties
+                (auxillary_runtimestats, prop);
+        }
     }
 
     /**
@@ -651,8 +966,10 @@ public class BackingStoreHashtable
 		throws StandardException
     {
         Object obj = hash_table.remove(key);
-        if( obj != null || diskHashtable == null)
+        if ( obj != null || diskHashtable == null)
+        {
             return obj;
+        }
         return diskHashtable.remove(key);
     }
 
@@ -698,9 +1015,12 @@ public class BackingStoreHashtable
      *
 	 * @exception  StandardException  Standard exception policy.
      **/
-    public boolean putRow(
-    boolean     needsToClone,
-    DataValueDescriptor[]    row)
+    public boolean putRow
+        (
+         boolean     needsToClone,
+         DataValueDescriptor[]    row,
+         RowLocation    rowLocation
+         )
 		throws StandardException
     {
 		// Are any key columns null?
@@ -724,7 +1044,7 @@ public class BackingStoreHashtable
         }
         else
         {
-            add_row_to_hash_table(row, needsToClone);
+            add_row_to_hash_table( row, rowLocation, needsToClone );
             return(true);
         }
     }
@@ -740,8 +1060,10 @@ public class BackingStoreHashtable
     public int size()
 		throws StandardException
     {
-        if( diskHashtable == null)
+        if ( diskHashtable == null)
+        {
             return(hash_table.size());
+        }
         return hash_table.size() + diskHashtable.size();
     }
 
@@ -753,7 +1075,7 @@ public class BackingStoreHashtable
         BackingStoreHashtableEnumeration()
         {
             memoryIterator = hash_table.values().iterator();
-            if( diskHashtable != null)
+            if ( diskHashtable != null)
             {
                 try
                 {
@@ -774,8 +1096,10 @@ public class BackingStoreHashtable
                 }
                 memoryIterator = null;
             }
-            if( diskEnumeration == null)
+            if ( diskEnumeration == null)
+            {
                 return false;
+            }
             return diskEnumeration.hasMoreElements();
         }
 
@@ -787,7 +1111,8 @@ public class BackingStoreHashtable
                 }
                 memoryIterator = null;
             }
-            return diskEnumeration.nextElement();
+            return makeInMemoryRow
+                ( ((DataValueDescriptor[]) diskEnumeration.nextElement()) );
         }
     } // end of class BackingStoreHashtableEnumeration
 
@@ -796,14 +1121,14 @@ public class BackingStoreHashtable
      * This class is used when the hash table contains multiple rows for the
      * same hash key.
      */
-    private static class RowList extends ArrayList<DataValueDescriptor[]> {
+    private static class RowList extends ArrayList<Object> {
 
         private RowList(int initialCapacity) {
             super(initialCapacity);
         }
 
         // The class is mostly empty and provides no functionality in addition
-        // to what's provided by ArrayList<DataValueDescriptor[]>. The main
+        // to what's provided by ArrayList<Object>. The main
         // purpose of the class is to allow type-safe casts from Object. These
         // casts are needed because the hash table can store both DVD[] and
         // List<DVD[]>, so its declared type is HashMap<Object, Object>.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java Wed Oct 23 12:15:25 2013
@@ -934,8 +934,10 @@ public interface TransactionController
      *                              BackingStoreHashtable.getRuntimeStats().
 	 * @param skipNullKeyColumns	Whether or not to skip rows with 1 or more null key columns
      *
-     * @param keepAfterCommit       If true then the hash table is kept after a
-     *                              commit
+     * @param keepAfterCommit If true then keep hash table after commit
+     *
+     * @param includeRowLocations If true then rows should include RowLocations.
+     *
      * @see BackingStoreHashtable
      * @see TransactionController#openScan
      *
@@ -961,7 +963,8 @@ public interface TransactionController
     float                   loadFactor,
     boolean                 collect_runtimestats,
     boolean		            skipNullKeyColumns,
-    boolean                 keepAfterCommit)
+    boolean                 keepAfterCommit,
+    boolean                 includeRowLocations)
         throws StandardException;
 
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/build.xml
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/build.xml?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/build.xml (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/build.xml Wed Oct 23 12:15:25 2013
@@ -53,7 +53,7 @@
         <pathelement path="${java16compile.classpath}"/>
       </classpath>
       <include name="${derby.dir}/iapi/store/**"/>
-      <!-- <compilerarg value="-Xlint:unchecked"/> -->
+      <compilerarg value="-Xlint:unchecked"/>
     </javac>
   </target>
 

Added: db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/LocatedRow.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/LocatedRow.java?rev=1535001&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/LocatedRow.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/LocatedRow.java Wed Oct 23 12:15:25 2013
@@ -0,0 +1,122 @@
+/*
+
+   Derby - Class org.apache.derby.iapi.types.LocatedRow
+
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to you under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+ */
+
+package org.apache.derby.iapi.types;
+
+/**
+ * <p>
+ * Mutable holder for the column values and RowLocation of a conglomerate row.
+ * Use with caution because values and arrays are not copied when they
+ * are passed in and out.
+ * </p>
+ */
+public class LocatedRow
+{
+    ////////////////////////////////////////////////////////////////
+    //
+    // STATE
+    //
+    ////////////////////////////////////////////////////////////////
+
+    private DataValueDescriptor[]   _columnValues;
+    private RowLocation                 _rowLocation;
+
+    ////////////////////////////////////////////////////////////////
+    //
+    // CONSTRUCTORS
+    //
+    ////////////////////////////////////////////////////////////////
+
+    /**
+     * <p>
+     * Construct from column values and row location.
+     * </p>
+     */
+    public  LocatedRow
+        (
+         DataValueDescriptor[]  columnValues,
+         RowLocation                rowLocation
+         )
+    {
+        _columnValues = columnValues;
+        _rowLocation = rowLocation;
+    }
+
+    /**
+     * <p>
+     * Construct from an array of column values, where the last value
+     * is the RowLocation.
+     * </p>
+     */
+    public  LocatedRow( DataValueDescriptor[] columnsAndRowLocation )
+    {
+        int     columnCount = columnsAndRowLocation.length - 1;
+        int     idx = 0;
+
+        _columnValues = new DataValueDescriptor[ columnCount ];
+        for ( ; idx < columnCount; idx++ )
+        { _columnValues[ idx ] = columnsAndRowLocation[ idx ]; }
+        _rowLocation = (RowLocation) columnsAndRowLocation[ idx ];
+    }
+
+    ////////////////////////////////////////////////////////////////
+    //
+    // ACCESSORS
+    //
+    ////////////////////////////////////////////////////////////////
+
+    /** Get the array of column values */
+    public  DataValueDescriptor[]   columnValues() { return _columnValues; }
+
+    /**
+     * Flatten this LocatedRow into a DataValueDescriptor[] where the last cell
+     * contains the RowLocation.
+     */
+    public  DataValueDescriptor[]   flatten()
+    {
+        return flatten( _columnValues, _rowLocation );
+    }
+
+    /** Get the RowLocation */
+    public  RowLocation rowLocation() { return _rowLocation; }
+
+    ////////////////////////////////////////////////////////////////
+    //
+    // STATIC BEHAVIOR
+    //
+    ////////////////////////////////////////////////////////////////
+
+    /** Append a RowLocation to the end of a column array */
+    public  static  DataValueDescriptor[]   flatten
+        ( DataValueDescriptor[] columnValues, RowLocation rowLocation )
+    {
+        DataValueDescriptor[]   result =
+            new DataValueDescriptor[ columnValues.length + 1 ];
+        int                             idx = 0;
+
+        for ( ; idx < columnValues.length; idx++ )
+        { result[ idx ] = columnValues[ idx ]; }
+        result[ idx ] = rowLocation;
+
+        return result;
+    }
+
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/LocatedRow.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DistinctScanResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DistinctScanResultSet.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DistinctScanResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DistinctScanResultSet.java Wed Oct 23 12:15:25 2013
@@ -112,7 +112,7 @@ class DistinctScanResultSet extends Hash
 
 			if (element.hasMoreElements())
 			{
-                DataValueDescriptor[] columns = (DataValueDescriptor[]) element.nextElement();
+                DataValueDescriptor[] columns = unpackHashValue( element.nextElement() );
 
 				setCompatRow(compactRow, columns);
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/HashScanResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/HashScanResultSet.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/HashScanResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/HashScanResultSet.java Wed Oct 23 12:15:25 2013
@@ -89,6 +89,7 @@ public class HashScanResultSet extends S
 	private boolean sameStartStopPosition;
 	private boolean skipNullKeyColumns;
 	private boolean keepAfterCommit;
+    private boolean includeRowLocations = false;
 
 	protected BackingStoreHashtable hashtable;
 	protected boolean eliminateDuplicates;		// set to true in DistinctScanResultSet
@@ -273,8 +274,8 @@ public class HashScanResultSet extends S
                     loadFactor,         // in memory Hashtable load factor
                     runTimeStatisticsOn,
 					skipNullKeyColumns,
-					keepAfterCommit);
-
+					keepAfterCommit,
+					includeRowLocations);
 
 			if (runTimeStatisticsOn)
 			{
@@ -424,26 +425,23 @@ public class HashScanResultSet extends S
 					{
 						entryVector = (List) hashEntry;
 						entryVectorSize = entryVector.size();
-						columns = 
-                            (DataValueDescriptor[]) entryVector.get(0);
+						columns = unpackHashValue( entryVector.get( 0 ) );
 					}
 					else
 					{
 						entryVector = null;
 						entryVectorSize = 0;
-						columns = (DataValueDescriptor[]) hashEntry;
+						columns = unpackHashValue( hashEntry );
 					}
 				}
 				else if (numFetchedOnNext < entryVectorSize)
 				{
 					// We are walking a list and there are more rows left.
-					columns = (DataValueDescriptor[]) 
-                        entryVector.get(numFetchedOnNext);
+					columns = unpackHashValue( entryVector.get( numFetchedOnNext ) );
 				}
 
 				if (columns != null)
 				{
-
 					// See if the entry satisfies all of the other qualifiers
 
 					/* We've already "evaluated" the 1st keyColumns qualifiers 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoPutResultSetImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoPutResultSetImpl.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoPutResultSetImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoPutResultSetImpl.java Wed Oct 23 12:15:25 2013
@@ -44,6 +44,7 @@ import org.apache.derby.iapi.store.acces
 import org.apache.derby.iapi.store.access.RowLocationRetRowSource;
 import org.apache.derby.iapi.store.access.RowSource;
 import org.apache.derby.iapi.types.DataValueDescriptor;
+import org.apache.derby.iapi.types.LocatedRow;
 import org.apache.derby.iapi.types.Orderable;
 import org.apache.derby.iapi.types.RowLocation;
 
@@ -555,5 +556,14 @@ extends BasicNoPutResultSetImpl
 		// is a no-op
 	}
 
+    /**
+     * Get all of the columns out of a value stored in a BackingStoreHashtable.
+     */
+    protected DataValueDescriptor[]   unpackHashValue( Object hashValue )
+    {
+        if ( hashValue == null ) { return null; }
+        else if ( hashValue instanceof DataValueDescriptor[] ) { return (DataValueDescriptor[]) hashValue; }
+        else { return ((LocatedRow) hashValue).flatten(); }
+    }
 
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScrollInsensitiveResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScrollInsensitiveResultSet.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScrollInsensitiveResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScrollInsensitiveResultSet.java Wed Oct 23 12:15:25 2013
@@ -993,7 +993,7 @@ public class ScrollInsensitiveResultSet 
 		System.arraycopy(sourceRowArray, 0, hashRowArray, extraColumns, 
 				sourceRowArray.length);
 
-		ht.putRow(true, hashRowArray);
+		ht.putRow(true, hashRowArray, null);
 
 		numToHashTable++;
 	}
@@ -1014,9 +1014,7 @@ public class ScrollInsensitiveResultSet 
 
 		// Get the row from the hash table
 		positionInHashTable.setValue(position);
-		DataValueDescriptor[] hashRowArray = (DataValueDescriptor[]) 
-				ht.get(positionInHashTable);
-
+		DataValueDescriptor[] hashRowArray = getCurrentRowFromHashtable();
 
 		if (SanityManager.DEBUG)
 		{
@@ -1069,8 +1067,7 @@ public class ScrollInsensitiveResultSet 
 		throws StandardException
 	{
 		positionInHashTable.setValue(position);
-		final DataValueDescriptor[] hashRowArray = (DataValueDescriptor[]) 
-			ht.get(positionInHashTable);
+		final DataValueDescriptor[] hashRowArray = getCurrentRowFromHashtable();
 		
 		// Copy out the Object[] without the position.
 		final DataValueDescriptor[] resultRowArray = new 
@@ -1088,8 +1085,7 @@ public class ScrollInsensitiveResultSet 
 	private void positionInLastFetchedRow() throws StandardException {
 		if (positionInSource > 0) {
 			positionInHashTable.setValue(positionInSource);
-			DataValueDescriptor[] hashRowArray = (DataValueDescriptor[]) 
-					ht.get(positionInHashTable);
+			DataValueDescriptor[] hashRowArray = getCurrentRowFromHashtable();
 			RowLocation rowLoc = (RowLocation) hashRowArray[POS_ROWLOCATION];
 			((NoPutResultSet)target).positionScanAtRowLocation(rowLoc);
 			currentPosition = positionInSource;
@@ -1117,8 +1113,7 @@ public class ScrollInsensitiveResultSet 
 		}
 
 		positionInHashTable.setValue(currentPosition);
-		DataValueDescriptor[] hashRowArray = (DataValueDescriptor[])
-				ht.get(positionInHashTable);
+		DataValueDescriptor[] hashRowArray = getCurrentRowFromHashtable();
 		RowLocation rowLoc = (RowLocation) hashRowArray[POS_ROWLOCATION];
 
 		// Maps from each selected column to underlying base table column
@@ -1192,8 +1187,7 @@ public class ScrollInsensitiveResultSet 
 	 */
 	public void markRowAsDeleted() throws StandardException  {
 		positionInHashTable.setValue(currentPosition);
-		DataValueDescriptor[] hashRowArray = (DataValueDescriptor[]) 
-				ht.get(positionInHashTable);
+		DataValueDescriptor[] hashRowArray = getCurrentRowFromHashtable();
 		RowLocation rowLoc = (RowLocation) hashRowArray[POS_ROWLOCATION];
 		ht.remove(new SQLInteger(currentPosition));
 		((SQLBoolean)hashRowArray[POS_ROWDELETED]).setValue(true);
@@ -1202,7 +1196,7 @@ public class ScrollInsensitiveResultSet 
 			hashRowArray[i].setToNull();
 		}
 
-		ht.putRow(true, hashRowArray);
+		ht.putRow(true, hashRowArray, null);
 	}
 
 	/**
@@ -1216,8 +1210,7 @@ public class ScrollInsensitiveResultSet 
 	public boolean isDeleted() throws StandardException  {
 		if (currentPosition <= positionInSource && currentPosition > 0) {
 			positionInHashTable.setValue(currentPosition);
-			DataValueDescriptor[] hashRowArray = (DataValueDescriptor[]) 
-					ht.get(positionInHashTable);
+			DataValueDescriptor[] hashRowArray = getCurrentRowFromHashtable();
 			return hashRowArray[POS_ROWDELETED].getBoolean();
 		}
 		return false;
@@ -1234,8 +1227,7 @@ public class ScrollInsensitiveResultSet 
 	public boolean isUpdated() throws StandardException {
 		if (currentPosition <= positionInSource && currentPosition > 0) {
 			positionInHashTable.setValue(currentPosition);
-			DataValueDescriptor[] hashRowArray = (DataValueDescriptor[]) 
-					ht.get(positionInHashTable);
+			DataValueDescriptor[] hashRowArray = getCurrentRowFromHashtable();
 			return hashRowArray[POS_ROWUPDATED].getBoolean();
 		}
 		return false;
@@ -1245,4 +1237,11 @@ public class ScrollInsensitiveResultSet 
 		return source.isForUpdate();
 	}
 
+    /** Get the column array from the current position in the hash table */
+    private DataValueDescriptor[]   getCurrentRowFromHashtable()
+        throws StandardException
+    {
+        return unpackHashValue( ht.get(positionInHashTable) );
+    }
+
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateResultSet.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateResultSet.java Wed Oct 23 12:15:25 2013
@@ -695,9 +695,12 @@ class UpdateResultSet extends DMLWriteRe
              * Need to save a clone because when we get cached currentRow, "rl"
              * shares the same reference, so is changed at the same time.
              */
-            tableScan.past2FutureTbl.putRow(
-                false,
-                new DataValueDescriptor[] { rl.cloneValue(false) });
+            tableScan.past2FutureTbl.putRow
+                (
+                 false,
+                 new DataValueDescriptor[] { rl.cloneValue(false) },
+                 null
+                 );
 		}
 	}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/BackingStoreHashTableFromScan.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/BackingStoreHashTableFromScan.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/BackingStoreHashTableFromScan.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/BackingStoreHashTableFromScan.java Wed Oct 23 12:15:25 2013
@@ -58,6 +58,7 @@ class BackingStoreHashTableFromScan exte
      **************************************************************************
      */
     private ScanManager             open_scan;
+    private boolean                     includeRowLocations;
 
     /**************************************************************************
      * Constructors for This class:
@@ -84,7 +85,8 @@ class BackingStoreHashTableFromScan exte
         float                   loadFactor,
         boolean                 collect_runtimestats,
 		boolean					skipNullKeyColumns,
-        boolean                 keepAfterCommit)
+        boolean                 keepAfterCommit,
+        boolean                 includeRowLocations )
             throws StandardException
     {
 
@@ -99,6 +101,7 @@ class BackingStoreHashTableFromScan exte
             loadFactor,
 			skipNullKeyColumns,
             keepAfterCommit);
+        this.includeRowLocations = includeRowLocations;
 
         open_scan =  (ScanManager)
             tc.openScan(
@@ -137,6 +140,9 @@ class BackingStoreHashTableFromScan exte
      **************************************************************************
      */
 
+    @Override
+    public  boolean includeRowLocations() { return includeRowLocations; }
+
     /**
      * Close the BackingStoreHashtable.
      * <p>

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java Wed Oct 23 12:15:25 2013
@@ -1328,7 +1328,8 @@ public class RAMTransaction 
     float                   loadFactor,
     boolean                 collect_runtimestats,
     boolean		            skipNullKeyColumns,
-    boolean                 keepAfterCommit)
+    boolean                 keepAfterCommit,
+    boolean                 includeRowLocations )
         throws StandardException
     {
         return (
@@ -1353,7 +1354,8 @@ public class RAMTransaction 
                 loadFactor,
                 collect_runtimestats,
 				skipNullKeyColumns,
-                keepAfterCommit));
+                keepAfterCommit,
+                includeRowLocations));
     }
 
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeForwardScan.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeForwardScan.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeForwardScan.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeForwardScan.java Wed Oct 23 12:15:25 2013
@@ -86,6 +86,13 @@ public class BTreeForwardScan extends BT
     int[]                   key_column_numbers)
         throws StandardException
 	{
+        if (SanityManager.DEBUG)
+        {
+            // RowLocations in the BTree itself are unstable and should
+            // not be put in long-lived structures like persistent hash tables.
+            SanityManager.ASSERT
+                ( (hash_table == null) || !hash_table.includeRowLocations() );
+        }
 
         int                     ret_row_count     = 0;
         DataValueDescriptor[]   fetch_row         = null;
@@ -443,7 +450,7 @@ public class BTreeForwardScan extends BT
 
                     if (hash_table != null)
                     {
-                        if (hash_table.putRow(false, fetch_row))
+                        if (hash_table.putRow(false, fetch_row, null))
                             fetch_row = null;
                     }
                     else

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/GenericScanController.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/GenericScanController.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/GenericScanController.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/GenericScanController.java Wed Oct 23 12:15:25 2013
@@ -800,7 +800,13 @@ public abstract class GenericScanControl
                     }
                     else
                     {
-                        if (hash_table.putRow(false, fetch_row))
+                        RowLocation     rowLocation =
+                            hash_table.includeRowLocations() ?
+                            makeRowLocation( scan_position ) : null;
+                        if (
+                            hash_table.putRow
+                            ( false, fetch_row, rowLocation )
+                            )
                         {
                             // The row was inserted into the hash table so we
                             // need to create a new row next time through.
@@ -889,6 +895,10 @@ public abstract class GenericScanControl
         throws StandardException
         ;
 
+    /** Make a RowLocation from a RowPosition */
+    protected abstract RowLocation makeRowLocation( RowPosition pos )
+        throws StandardException;
+
     /**************************************************************************
      * abstract protected Methods of This class:
      **************************************************************************

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapScan.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapScan.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapScan.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapScan.java Wed Oct 23 12:15:25 2013
@@ -130,6 +130,12 @@ class HeapScan 
         }
     }
 
+    protected RowLocation makeRowLocation( RowPosition pos )
+        throws StandardException
+    {
+        return new HeapRowLocation( pos.current_rh );
+    }
+
     protected void setRowLocationArray(
     RowLocation[]   rowloc_array,
     int             index,

Modified: db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java (original)
+++ db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java Wed Oct 23 12:15:25 2013
@@ -155,7 +155,8 @@ class NoOpTransaction implements Transac
             boolean remove_duplicates, long estimated_rowcnt,
             long max_inmemory_rowcnt, int initialCapacity, float loadFactor,
             boolean collect_runtimestats, boolean skipNullKeyColumns,
-            boolean keepAfterCommit)
+            boolean keepAfterCommit,
+            boolean includeRowLocations)
             throws StandardException {
         // Auto-generated method stub
         return null;

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_QualifierTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_QualifierTest.java?rev=1535001&r1=1535000&r2=1535001&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_QualifierTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_QualifierTest.java Wed Oct 23 12:15:25 2013
@@ -1017,7 +1017,8 @@ public class T_QualifierTest 
                 -1,             // use default load factor
                 false,          // don't maintain runtime statistics
                 false,          // don't skip null key columns
-                false);         // don't keep after commit
+                false,          // don't keep after commit
+                false);         // don't include row locations
 
         // make sure the expected result set is the same as the actual result
         // set.
@@ -1106,7 +1107,8 @@ public class T_QualifierTest 
                 -1,             // use default load factor
                 false,         // don't maintain runtime statistics
 				false,			// don't skip null key columns
-                false);         // don't keep after commit
+                false,          // don't keep after commit
+                false);         // don't include row locations
 
         Object removed_obj;
         for (numrows = 0; numrows < expect_numrows; numrows++)