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 mi...@apache.org on 2005/04/15 16:28:04 UTC

svn commit: r161468 [1/2] - in incubator/derby/code/trunk/java: engine/org/apache/derby/catalog/ engine/org/apache/derby/iapi/db/ engine/org/apache/derby/iapi/store/access/ engine/org/apache/derby/iapi/store/access/conglomerate/ engine/org/apache/derby/iapi/store/raw/ engine/org/apache/derby/impl/sql/catalog/ engine/org/apache/derby/impl/store/access/ engine/org/apache/derby/impl/store/access/btree/ engine/org/apache/derby/impl/store/access/btree/index/ engine/org/apache/derby/impl/store/access/conglomerate/ engine/org/apache/derby/impl/store/access/heap/ engine/org/apache/derby/impl/store/access/sort/ engine/org/apache/derby/impl/store/raw/data/ testing/org/apache/derbyTesting/functionTests/master/ testing/org/apache/derbyTesting/functionTests/suites/ testing/org/apache/derbyTesting/functionTests/tests/store/ testing/org/apache/derbyTesting/functionTests/tests/storetests/

Author: mikem
Date: Fri Apr 15 07:27:56 2005
New Revision: 161468

URL: http://svn.apache.org/viewcvs?view=rev&rev=161468
Log:
The following is the 1st phase of a checkin to address DERBY-132 - improvement
request to add an inplace compress.

This checkin addresses the issue in 3 parts:

1) Code has been added to purge committed rows from heap tables.
This code uses the same basic algorithm to identify and purge
committed deleted heap rows that currently exists.  It scans the
entire heap table in the current thread processing all rows.

2) Code has been added to defragment remaining rows in the page, freeing
pages at the end of the table.
This code scans the table and moves rows from the end of the heap table
toward the front of the table.  For every row moved all index entries
must be updated after the move.  The allocation system is updated to
put new rows toward the front of the table.  After it is finished there
will be a chunk of empty pages at the end of the file.

3) Code has been added to return the free space at the end of the table
back to the OS.
Finds the chunk of empty pages at the end of the file.  Updates the
Allocation data structures to remove the chunk of pages at the end.
Calls the JVM to return the space at the end of the file back to the
OS.

In order to test all of these paths a new system procedure,
SYSCS_UTIL.SYSCS_INPLACE_COMPRESS_TABLE has been added.  Eventually
as a zero admin database the system should call this routine internally.
It allows each of the 3 above described phases to be called either
individually or in sequence.


Added:
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/db/OnlineCompress.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapCompressScan.java
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/onlineCompressTable.out
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/BaseTest.java
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineCompressTest.java
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineCompressTest_app.properties
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineCompressTest_derby.properties
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/storetests/BaseTest.java
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/storetests/onlineCompressTable.sql
Modified:
    incubator/derby/code/trunk/java/engine/org/apache/derby/catalog/SystemProcedures.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/GroupFetchScanController.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/Conglomerate.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/ContainerHandle.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Page.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeController.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeScan.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/GenericScanController.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/OpenConglomerate.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/RowPosition.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapController.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapScan.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/sort/Scan.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/AllocExtent.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/AllocPage.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainer.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainerHandle.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BasePage.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/FileContainer.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InputStreamContainer.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/StoredPage.java
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storetests.runall
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/copyfiles.ant
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/storetests/copyfiles.ant

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/catalog/SystemProcedures.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/catalog/SystemProcedures.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/catalog/SystemProcedures.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/catalog/SystemProcedures.java Fri Apr 15 07:27:56 2005
@@ -728,6 +728,24 @@
         return(ret_val ? 1 : 0);
     }
 
+    public static void SYSCS_INPLACE_COMPRESS_TABLE(
+    String  schema,
+    String  tablename,
+    int     purgeRows,
+    int     defragementRows,
+    int     truncateEnd)
+		throws SQLException
+    {
+        org.apache.derby.iapi.db.OnlineCompress.compressTable(
+            schema, 
+            tablename, 
+            (purgeRows == 1),
+            (defragementRows == 1),
+            (truncateEnd == 1));
+
+        return;
+    }
+
     public static String SYSCS_GET_RUNTIMESTATISTICS()
 		throws SQLException
     {
@@ -1044,14 +1062,3 @@
 	}
 	
 }
-
-
-
-
-
-
-
-
-
-
-

Added: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/db/OnlineCompress.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/db/OnlineCompress.java?view=auto&rev=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/db/OnlineCompress.java (added)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/db/OnlineCompress.java Fri Apr 15 07:27:56 2005
@@ -0,0 +1,532 @@
+/*
+
+   Derby - Class org.apache.derby.iapi.db.OnlineCompress
+
+   Copyright 2005 The Apache Software Foundation or its licensors, as applicable.
+
+   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 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.db;
+
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.error.PublicAPI;
+
+import org.apache.derby.iapi.sql.dictionary.DataDictionaryContext;
+import org.apache.derby.iapi.sql.dictionary.DataDictionary;
+import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
+import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
+import org.apache.derby.iapi.sql.dictionary.ColumnDescriptor;
+import org.apache.derby.iapi.sql.dictionary.ColumnDescriptorList;
+import org.apache.derby.iapi.sql.dictionary.ConstraintDescriptor;
+import org.apache.derby.iapi.sql.dictionary.ConstraintDescriptorList;
+import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
+
+import org.apache.derby.iapi.sql.depend.DependencyManager;
+
+import org.apache.derby.iapi.sql.execute.ExecRow;
+import org.apache.derby.iapi.sql.execute.ExecutionContext;
+
+import org.apache.derby.iapi.types.DataValueDescriptor;
+import org.apache.derby.iapi.types.DataValueFactory;
+
+
+import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
+import org.apache.derby.iapi.sql.conn.ConnectionUtil;
+
+import org.apache.derby.iapi.store.access.TransactionController;
+import org.apache.derby.iapi.types.RowLocation;
+import org.apache.derby.iapi.store.access.ScanController;
+import org.apache.derby.iapi.store.access.ConglomerateController;
+import org.apache.derby.iapi.store.access.GroupFetchScanController;
+import org.apache.derby.iapi.store.access.RowUtil;
+import org.apache.derby.iapi.store.access.Qualifier;
+
+import org.apache.derby.iapi.services.sanity.SanityManager;
+
+import org.apache.derby.iapi.reference.SQLState;
+
+import org.apache.derby.iapi.services.io.FormatableBitSet;
+
+import java.sql.SQLException;
+
+public class OnlineCompress
+{
+
+	/** no requirement for a constructor */
+	private OnlineCompress() {
+	}
+	public static void compressTable(
+    String  schemaName, 
+    String  tableName,
+    boolean purgeRows,
+    boolean defragmentRows,
+    boolean truncateEnd)
+        throws SQLException
+	{
+		LanguageConnectionContext lcc       = ConnectionUtil.getCurrentLCC();
+		TransactionController     tc        = lcc.getTransactionExecute();
+
+		try 
+        {
+            DataDictionary data_dictionary = lcc.getDataDictionary();
+
+            // Each of the following may give up locks allowing ddl on the
+            // table, so each phase needs to do the data dictionary lookup.
+
+            if (purgeRows)
+                purgeRows(schemaName, tableName, data_dictionary, tc);
+
+            if (defragmentRows)
+                defragmentRows(schemaName, tableName, data_dictionary, tc);
+
+            if (truncateEnd)
+                truncateEnd(schemaName, tableName, data_dictionary, tc);
+        }
+		catch (StandardException se)
+		{
+			throw PublicAPI.wrapStandardException(se);
+		}
+
+	}
+
+	private static void defragmentRows(
+    String                  schemaName, 
+    String                  tableName,
+    DataDictionary          data_dictionary,
+    TransactionController   tc)
+        throws SQLException
+	{
+        GroupFetchScanController base_group_fetch_cc = null;
+        int                      num_indexes         = 0;
+
+        int[][]                  index_col_map       =  null;
+        ScanController[]         index_scan          =  null;
+        ConglomerateController[] index_cc            =  null;
+        DataValueDescriptor[][]  index_row           =  null;
+
+		LanguageConnectionContext lcc       = ConnectionUtil.getCurrentLCC();
+		TransactionController     nested_tc = null;
+
+		try {
+
+            SchemaDescriptor sd = 
+                data_dictionary.getSchemaDescriptor(
+                    schemaName, nested_tc, true);
+            TableDescriptor td = 
+                data_dictionary.getTableDescriptor(tableName, sd);
+            nested_tc = 
+                tc.startNestedUserTransaction(false);
+
+            if (td == null)
+            {
+                throw StandardException.newException(
+                    SQLState.LANG_TABLE_NOT_FOUND, 
+                    schemaName + "." + tableName);
+            }
+
+            /* Skip views */
+            if (td.getTableType() == TableDescriptor.VIEW_TYPE)
+            {
+                return;
+            }
+
+
+			ConglomerateDescriptor heapCD = 
+                td.getConglomerateDescriptor(td.getHeapConglomerateId());
+
+			/* Get a row template for the base table */
+			ExecRow baseRow = 
+                lcc.getExecutionContext().getExecutionFactory().getValueRow(
+                    td.getNumberOfColumns());
+
+
+			/* Fill the row with nulls of the correct type */
+			ColumnDescriptorList cdl = td.getColumnDescriptorList();
+			int					 cdlSize = cdl.size();
+
+			for (int index = 0; index < cdlSize; index++)
+			{
+				ColumnDescriptor cd = (ColumnDescriptor) cdl.elementAt(index);
+				baseRow.setColumn(cd.getPosition(), cd.getType().getNull());
+			}
+
+            DataValueDescriptor[][] row_array = new DataValueDescriptor[100][];
+            row_array[0] = baseRow.getRowArray();
+            RowLocation[] old_row_location_array = new RowLocation[100];
+            RowLocation[] new_row_location_array = new RowLocation[100];
+
+            // Create the following 3 arrays which will be used to update
+            // each index as the scan moves rows about the heap as part of
+            // the compress:
+            //     index_col_map - map location of index cols in the base row, 
+            //                     ie. index_col_map[0] is column offset of 1st
+            //                     key collumn in base row.  All offsets are 0 
+            //                     based.
+            //     index_scan - open ScanController used to delete old index row
+            //     index_cc   - open ConglomerateController used to insert new 
+            //                  row
+
+            ConglomerateDescriptor[] conglom_descriptors = 
+                td.getConglomerateDescriptors();
+
+            // conglom_descriptors has an entry for the conglomerate and each 
+            // one of it's indexes.
+            num_indexes = conglom_descriptors.length - 1;
+
+            // if indexes exist, set up data structures to update them
+            if (num_indexes > 0)
+            {
+                // allocate arrays
+                index_col_map   = new int[num_indexes][];
+                index_scan      = new ScanController[num_indexes];
+                index_cc        = new ConglomerateController[num_indexes];
+                index_row       = new DataValueDescriptor[num_indexes][];
+
+                setup_indexes(
+                    nested_tc,
+                    td,
+                    index_col_map,
+                    index_scan,
+                    index_cc,
+                    index_row);
+
+                SanityManager.DEBUG_PRINT("OnlineCompress", "index_col_map = " + index_col_map);
+            }
+
+			/* Open the heap for reading */
+			base_group_fetch_cc = 
+                nested_tc.defragmentConglomerate(
+                    td.getHeapConglomerateId(), 
+                    false,
+                    true, 
+                    TransactionController.OPENMODE_FORUPDATE, 
+				    TransactionController.MODE_TABLE,
+					TransactionController.ISOLATION_SERIALIZABLE);
+
+            int num_rows_fetched = 0;
+            while ((num_rows_fetched = 
+                        base_group_fetch_cc.fetchNextGroup(
+                            row_array, 
+                            old_row_location_array, 
+                            new_row_location_array)) != 0)
+            {
+                if (num_indexes > 0)
+                {
+                    for (int row = 0; row < num_rows_fetched; row++)
+                    {
+                        for (int index = 0; index < num_indexes; index++)
+                        {
+                            SanityManager.DEBUG_PRINT("OnlineCompress", "calling fixIndex, row = " + row + "; index = " + index);
+                SanityManager.DEBUG_PRINT("OnlineCompress", "before fixIndex call index_col_map = " + index_col_map);
+                SanityManager.DEBUG_PRINT("OnlineCompress", "before fixIndex call index_col_map[0] = " + index_col_map[0]);
+                            fixIndex(
+                                row_array[row],
+                                index_row[index],
+                                old_row_location_array[row],
+                                new_row_location_array[row],
+                                index_cc[index],
+                                index_scan[index],
+                                index_col_map[index]);
+                        }
+                    }
+                }
+            }
+			
+		}
+		catch (StandardException se)
+		{
+			throw PublicAPI.wrapStandardException(se);
+		}
+		finally
+		{
+            try
+            {
+                /* Clean up before we leave */
+                if (base_group_fetch_cc != null)
+                {
+                    base_group_fetch_cc.close();
+                    base_group_fetch_cc = null;
+                }
+
+                if (num_indexes > 0)
+                {
+                    for (int i = 0; i < num_indexes; i++)
+                    {
+                        if (index_scan != null && index_scan[i] != null)
+                        {
+                            index_scan[i].close();
+                            index_scan[i] = null;
+                        }
+                        if (index_cc != null && index_cc[i] != null)
+                        {
+                            index_cc[i].close();
+                            index_cc[i] = null;
+                        }
+                    }
+                }
+
+                if (nested_tc != null)
+                {
+                    nested_tc.destroy();
+                }
+
+            }
+            catch (StandardException se)
+            {
+                throw PublicAPI.wrapStandardException(se);
+            }
+		}
+
+		return;
+	}
+
+	private static void purgeRows(
+    String                  schemaName, 
+    String                  tableName,
+    DataDictionary          data_dictionary,
+    TransactionController   tc)
+        throws StandardException
+	{
+        SchemaDescriptor sd = 
+            data_dictionary.getSchemaDescriptor(schemaName, tc, true);
+        TableDescriptor  td = 
+            data_dictionary.getTableDescriptor(tableName, sd);
+
+        if (td == null)
+        {
+            throw StandardException.newException(
+                SQLState.LANG_TABLE_NOT_FOUND, 
+                schemaName + "." + tableName);
+        }
+
+        /* Skip views */
+        if (td.getTableType() != TableDescriptor.VIEW_TYPE)
+        {
+
+            ConglomerateDescriptor[] conglom_descriptors = 
+                td.getConglomerateDescriptors();
+
+            for (int cd_idx = 0; cd_idx < conglom_descriptors.length; cd_idx++)
+            {
+                ConglomerateDescriptor cd = conglom_descriptors[cd_idx];
+
+                tc.purgeConglomerate(cd.getConglomerateNumber());
+            }
+        }
+
+        return;
+    }
+
+	private static void truncateEnd(
+    String                  schemaName, 
+    String                  tableName,
+    DataDictionary          data_dictionary,
+    TransactionController   tc)
+        throws StandardException
+	{
+        SchemaDescriptor sd = 
+            data_dictionary.getSchemaDescriptor(schemaName, tc, true);
+        TableDescriptor  td = 
+            data_dictionary.getTableDescriptor(tableName, sd);
+
+        if (td == null)
+        {
+            throw StandardException.newException(
+                SQLState.LANG_TABLE_NOT_FOUND, 
+                schemaName + "." + tableName);
+        }
+
+        /* Skip views */
+        if (td.getTableType() != TableDescriptor.VIEW_TYPE)
+        {
+            ConglomerateDescriptor[] conglom_descriptors = 
+                td.getConglomerateDescriptors();
+
+            for (int cd_idx = 0; cd_idx < conglom_descriptors.length; cd_idx++)
+            {
+                ConglomerateDescriptor cd = conglom_descriptors[cd_idx];
+
+                tc.compressConglomerate(cd.getConglomerateNumber());
+            }
+        }
+
+        return;
+    }
+
+    private static void setup_indexes(
+    TransactionController       tc,
+    TableDescriptor             td,
+    int[][]                     index_col_map,
+    ScanController[]            index_scan,
+    ConglomerateController[]    index_cc,
+    DataValueDescriptor[][]     index_row)
+		throws StandardException
+    {
+
+        // Initialize the following 3 arrays which will be used to update
+        // each index as the scan moves rows about the heap as part of
+        // the compress:
+        //     index_col_map - map location of index cols in the base row, ie.
+        //                     index_col_map[0] is column offset of 1st key
+        //                     collumn in base row.  All offsets are 0 based.
+        //     index_scan - open ScanController used to delete old index row
+        //     index_cc   - open ConglomerateController used to insert new row
+
+        ConglomerateDescriptor[] conglom_descriptors =
+                td.getConglomerateDescriptors();
+
+
+        int index_idx = 0;
+        for (int cd_idx = 0; cd_idx < conglom_descriptors.length; cd_idx++)
+        {
+            SanityManager.DEBUG_PRINT("OnlineCompress", "setup loop: " + cd_idx);
+            ConglomerateDescriptor index_cd = conglom_descriptors[cd_idx];
+
+            if (!index_cd.isIndex())
+            {
+                // skip the heap descriptor entry
+                continue;
+            }
+            SanityManager.DEBUG_PRINT("OnlineCompress", "setup loop 1: " + cd_idx);
+
+            // ScanControllers are used to delete old index row
+            index_scan[index_idx] = 
+                tc.openScan(
+                    index_cd.getConglomerateNumber(),
+                    true,	// hold
+                    TransactionController.OPENMODE_FORUPDATE,
+                    TransactionController.MODE_TABLE,
+                    TransactionController.ISOLATION_SERIALIZABLE,
+                    null,   // full row is retrieved, 
+                            // so that full row can be used for start/stop keys
+                    null,	// startKeyValue - will be reset with reopenScan()
+                    0,		// 
+                    null,	// qualifier
+                    null,	// stopKeyValue  - will be reset with reopenScan()
+                    0);		// 
+
+            // ConglomerateControllers are used to insert new index row
+            index_cc[index_idx] = 
+                tc.openConglomerate(
+                    index_cd.getConglomerateNumber(),
+                    true,  // hold
+                    TransactionController.OPENMODE_FORUPDATE,
+                    TransactionController.MODE_TABLE,
+                    TransactionController.ISOLATION_SERIALIZABLE);
+
+            // build column map to allow index row to be built from base row
+            int[] baseColumnPositions   = 
+                index_cd.getIndexDescriptor().baseColumnPositions();
+            int[] zero_based_map        = 
+                new int[baseColumnPositions.length];
+
+            for (int i = 0; i < baseColumnPositions.length; i++)
+            {
+                zero_based_map[i] = baseColumnPositions[i] - 1; 
+            }
+
+            index_col_map[index_idx] = zero_based_map;
+
+            // build row array to delete from index and insert into index
+            //     length is length of column map + 1 for RowLocation.
+            index_row[index_idx] = 
+                new DataValueDescriptor[baseColumnPositions.length + 1];
+
+            index_idx++;
+        }
+
+        return;
+    }
+
+
+    /**
+     * Delete old index row and insert new index row in input index.
+     * <p>
+     *
+	 * @return The identifier to be used to open the conglomerate later.
+     *
+     * @param param1 param1 does this.
+     * @param param2 param2 does this.
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+    private static void fixIndex(
+    DataValueDescriptor[]   base_row,
+    DataValueDescriptor[]   index_row,
+    RowLocation             old_row_loc,
+    RowLocation             new_row_loc,
+    ConglomerateController  index_cc,
+    ScanController          index_scan,
+	int[]					index_col_map)
+        throws StandardException
+    {
+        if (SanityManager.DEBUG)
+        {
+            // baseColumnPositions should describe all columns in index row
+            // except for the final column, which is the RowLocation.
+            SanityManager.ASSERT(index_col_map != null);
+            SanityManager.ASSERT(index_row != null);
+            SanityManager.ASSERT(
+                (index_col_map.length == (index_row.length - 1)));
+        }
+
+        // create the index row to delete from from the base row, using map
+        for (int index = 0; index < index_col_map.length; index++)
+        {
+            index_row[index] = base_row[index_col_map[index]];
+        }
+        // last column in index in the RowLocation
+        index_row[index_row.length - 1] = old_row_loc;
+
+        SanityManager.DEBUG_PRINT("OnlineCompress", "row before delete = " +
+                RowUtil.toString(index_row));
+
+        // position the scan for the delete, the scan should already be open.
+        // This is done by setting start scan to full key, GE and stop scan
+        // to full key, GT.
+        index_scan.reopenScan(
+            index_row,
+            ScanController.GE,
+            (Qualifier[][]) null,
+            index_row,
+            ScanController.GT);
+
+        // position the scan, serious problem if scan does not find the row.
+        if (index_scan.next())
+        {
+            index_scan.delete();
+        }
+        else
+        {
+            // Didn't find the row we wanted to delete.
+            if (SanityManager.DEBUG)
+            {
+                SanityManager.THROWASSERT(
+                    "Did not find row to delete." +
+                    "base_row = " + RowUtil.toString(base_row) +
+                    "index_row = " + RowUtil.toString(index_row));
+            }
+        }
+
+        // insert the new index row into the conglomerate
+        index_row[index_row.length - 1] = new_row_loc;
+
+        SanityManager.DEBUG_PRINT("OnlineCompress", "row before insert = " +
+                RowUtil.toString(index_row));
+        index_cc.insert(index_row);
+
+        return;
+    }
+}

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/GroupFetchScanController.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/GroupFetchScanController.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/GroupFetchScanController.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/GroupFetchScanController.java Fri Apr 15 07:27:56 2005
@@ -137,6 +137,12 @@
     RowLocation[]           rowloc_array)
         throws StandardException;
 
+    public int fetchNextGroup(
+    DataValueDescriptor[][] row_array,
+    RowLocation[]           oldrowloc_array,
+    RowLocation[]           newrowloc_array)
+        throws StandardException;
+
     /**
     Move to the next position in the scan.  If this is the first
     call to next(), the position is set to the first row.

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java Fri Apr 15 07:27:56 2005
@@ -1191,6 +1191,80 @@
 		int                             stopSearchOperator)
 			throws StandardException;
 
+    /**
+     * Compress table in place
+     * <p>
+     * Returns a GroupFetchScanController which can be used to move rows
+     * around in a table, creating a block of free pages at the end of the
+     * table.  The process will move rows from the end of the table toward
+     * the beginning.  The GroupFetchScanController will return the 
+     * old row location, the new row location, and the actual data of any
+     * row moved.  Note that this scan only returns moved rows, not an
+     * entire set of rows, the scan is designed specifically to be
+     * used by either explicit user call of the SYSCS_ONLINE_COMPRESS_TABLE()
+     * procedure, or internal background calls to compress the table.
+     *
+     * The old and new row locations are returned so that the caller can
+     * update any indexes necessary.
+     *
+     * This scan always returns all collumns of the row.
+     * 
+     * All inputs work exactly as in openScan().  The return is 
+     * a GroupFetchScanController, which only allows fetches of groups
+     * of rows from the conglomerate.
+     * <p>
+     *
+	 * @return The GroupFetchScanController to be used to fetch the rows.
+     *
+	 * @param conglomId             see openScan()
+     * @param hold                  see openScan()
+     * @param open_mode             see openScan()
+     * @param lock_level            see openScan()
+     * @param isolation_level       see openScan()
+     *
+	 * @exception  StandardException  Standard exception policy.
+     *
+     * @see ScanController
+     * @see GroupFetchScanController
+     **/
+	GroupFetchScanController defragmentConglomerate(
+		long                            conglomId,
+        boolean                         online,
+		boolean                         hold,
+		int                             open_mode,
+        int                             lock_level,
+        int                             isolation_level)
+			throws StandardException;
+
+    /**
+     * Purge all committed deleted rows from the conglomerate.
+     * <p>
+     * This call will purge committed deleted rows from the conglomerate,
+     * that space will be available for future inserts into the conglomerate.
+     * <p>
+     *
+     * @param conglomId Id of the conglomerate to purge.
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+	void purgeConglomerate(long conglomId)
+			throws StandardException;
+
+    /**
+     * Return free space from the conglomerate back to the OS.
+     * <p>
+     * Returns free space from the conglomerate back to the OS.  Currently
+     * only the sequential free pages at the "end" of the conglomerate can
+     * be returned to the OS.
+     * <p>
+     *
+     * @param conglomId Id of the conglomerate to purge.
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+	void compressConglomerate(long conglomId)
+			throws StandardException;
+
 
     /**
      * Retrieve the maximum value row in an ordered conglomerate.

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/Conglomerate.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/Conglomerate.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/Conglomerate.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/Conglomerate.java Fri Apr 15 07:27:56 2005
@@ -312,7 +312,7 @@
     int                             lock_level,
     LockingPolicy                   locking_policy,
     int                             isolation_level,
-	FormatableBitSet				            scanColumnList,
+	FormatableBitSet				scanColumnList,
     DataValueDescriptor[]	        startKeyValue,
     int                             startSearchOperator,
     Qualifier                       qualifier[][],
@@ -320,6 +320,61 @@
     int                             stopSearchOperator,
     StaticCompiledOpenConglomInfo   static_info,
     DynamicCompiledOpenConglomInfo  dynamic_info)
+        throws StandardException;
+
+    /**
+     * Online compress table.
+     *
+     * Returns a ScanManager which can be used to move rows
+     * around in a table, creating a block of free pages at the end of the
+     * table.  The process of executing the scan will move rows from the end 
+     * of the table toward the beginning.  The GroupFetchScanController will
+     * return the old row location, the new row location, and the actual data 
+     * of any row moved.  Note that this scan only returns moved rows, not an
+     * entire set of rows, the scan is designed specifically to be
+     * used by either explicit user call of the SYSCS_ONLINE_COMPRESS_TABLE()
+     * procedure, or internal background calls to compress the table.
+     *
+     * The old and new row locations are returned so that the caller can
+     * update any indexes necessary.
+     *
+     * This scan always returns all collumns of the row.
+     * 
+     * All inputs work exactly as in openScan().  The return is 
+     * a GroupFetchScanController, which only allows fetches of groups
+     * of rows from the conglomerate.
+     * <p>
+     * Note that all Conglomerates may not implement openCompressScan(), 
+     * currently only the Heap conglomerate implements this scan.
+     *
+	 * @return The GroupFetchScanController to be used to fetch the rows.
+     *
+	 * @param conglomId             see openScan()
+     * @param hold                  see openScan()
+     * @param open_mode             see openScan()
+     * @param lock_level            see openScan()
+     * @param isolation_level       see openScan()
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+	ScanManager defragmentConglomerate(
+    TransactionManager              xact_manager,
+    Transaction                     rawtran,
+    boolean                         hold,
+    int                             open_mode,
+    int                             lock_level,
+    LockingPolicy                   locking_policy,
+    int                             isolation_level)
+        throws StandardException;
+
+	void purgeConglomerate(
+    TransactionManager              xact_manager,
+    Transaction                     rawtran)
+        throws StandardException;
+
+	void compressConglomerate(
+    TransactionManager              xact_manager,
+    Transaction                     rawtran)
         throws StandardException;
 
     /**

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/ContainerHandle.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/ContainerHandle.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/ContainerHandle.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/ContainerHandle.java Fri Apr 15 07:27:56 2005
@@ -194,6 +194,18 @@
 	public Page addPage() throws StandardException;
 
 
+	/**
+		Release free space to the OS.
+		<P>
+        As is possible release any free space to the operating system.  This
+        will usually mean releasing any free pages located at the end of the
+        file using the java truncate() interface.
+
+		@exception StandardException	Standard Cloudscape error policy
+	*/
+	public void compressContainer() throws StandardException;
+
+
 	/**	
 		Add an empty page to the container and obtain exclusive access to it.
 		<P>
@@ -416,6 +428,11 @@
 	public Page getPageForInsert(int flag) 
 		 throws StandardException;
 
+	public Page getPageForCompress(
+    int     flag,
+    long    pageno) 
+		 throws StandardException;
+
 	// Try to get a page that is unfilled, 'unfill-ness' is defined by the
 	// page.  Since unfill-ness is defined by the page, the only thing RawStore
 	// guarentees about the page is that it has space for a a minimum sized
@@ -427,7 +444,6 @@
 	// If for whatever reasons RawStore is unable to come up with such a page,
 	// null will be returned.
 	public static final int GET_PAGE_UNFILLED = 0x1;
-
 
 
     /**

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Page.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Page.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Page.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Page.java Fri Apr 15 07:27:56 2005
@@ -227,10 +227,10 @@
      * @see LockingPolicy
      **/
 	RecordHandle fetch(
-    RecordHandle            handle, 
-    Object[]   row, 
-    FormatableBitSet                 validColumns, 
-    boolean                 forUpdate)
+    RecordHandle        handle, 
+    Object[]            row, 
+    FormatableBitSet    validColumns, 
+    boolean             forUpdate)
 		throws StandardException;
 
     /**
@@ -283,9 +283,9 @@
 	 * @exception  StandardException  Standard exception policy.
      **/
 	boolean spaceForInsert(
-    Object[]   row, 
-    FormatableBitSet                 validColumns, 
-    int                     overflowThreshold) 
+    Object[]            row, 
+    FormatableBitSet    validColumns, 
+    int                 overflowThreshold) 
         throws StandardException;
 
     /**
@@ -311,10 +311,10 @@
      * @exception StandardException Row cannot fit on the page or row is null.
      **/
 	RecordHandle insert(
-    Object[]   row, 
-    FormatableBitSet                 validColumns,
-    byte                    insertFlag, 
-    int                     overflowThreshold)
+    Object[]            row, 
+    FormatableBitSet    validColumns,
+    byte                insertFlag, 
+    int                 overflowThreshold)
 		throws StandardException;
 
 	/**
@@ -350,8 +350,8 @@
 	 * @exception  StandardException  Standard exception policy.
      **/
 	boolean update(
-    RecordHandle            handle, 
-    Object[]   row, 
+    RecordHandle        handle, 
+    Object[]            row, 
     FormatableBitSet                 validColumns)
 		throws StandardException;
 
@@ -390,6 +390,55 @@
 	public boolean delete(
     RecordHandle    handle, 
     LogicalUndo     undo)
+		throws StandardException;
+
+    /**
+     * Move record to a page toward the beginning of the file.
+     * <p>
+     * As part of compressing the table records need to be moved from the
+     * end of the file toward the beginning of the file.  Only the 
+     * contiguous set of free pages at the very end of the file can
+     * be given back to the OS.  This call is used to purge the row from
+     * the current page, insert it into a previous page, and return the
+     * new row location 
+     * Mark the record identified by position as deleted. The record may be 
+     * undeleted sometime later using undelete() by any transaction that sees 
+     * the record.
+     * <p>
+     * The interface is optimized to work on a number of rows at a time, 
+     * optimally processing all rows on the page at once.  The call will 
+     * process either all rows on the page, or the number of slots in the
+     * input arrays - whichever is smaller.
+     * <B>Locking Policy</B>
+     * <P>
+     * MUST be called with table locked, not locks are requested.  Because
+     * it is called with table locks the call will go ahead and purge any
+     * row which is marked deleted.  It will also use purge rather than
+     * delete to remove the old row after it moves it to a new page.  This
+     * is ok since the table lock insures that no other transaction will
+     * use space on the table before this transaction commits.
+     *
+     * <BR>
+     * A page latch on the new page will be requested and released.
+     *
+     * @param old_handle     An array to be filled in by the call with the 
+     *                       old handles of all rows moved.
+     * @param new_handle     An array to be filled in by the call with the 
+     *                       new handles of all rows moved.
+     * @param new_pageno     An array to be filled in by the call with the 
+     *                       new page number of all rows moved.
+     *
+     * @return the number of rows processed.
+     *
+     * @exception StandardException	Standard Cloudscape error policy
+     *
+     * @see LockingPolicy
+     **/
+	public int moveRecordForCompressAtSlot(
+    int             slot,
+    Object[]        row,
+    RecordHandle[]  old_handle,
+    RecordHandle[]  new_handle)
 		throws StandardException;
 
     /**

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java Fri Apr 15 07:27:56 2005
@@ -8430,7 +8430,7 @@
                 tc);
         }
 
-        // SMALLINT SYSCS_UTIL.SYSCS_CHECK_TABLE(varchar(128))
+        // SMALLINT SYSCS_UTIL.SYSCS_CHECK_TABLE(varchar(128), varchar(128))
         {
             // procedure argument names
             String[] arg_names = {"SCHEMANAME", "TABLENAME"};
@@ -8476,6 +8476,48 @@
                 DataTypeDescriptor.getBuiltInDataTypeDescriptor(
                     Types.CLOB, DB2Limit.DB2_LOB_MAXWIDTH),
                 */
+                tc);
+        }
+
+        // void SYSCS_UTIL.SYSCS_INPLACE_COMPRESS_TABLE(
+        //     IN SCHEMANAME        VARCHAR(128), 
+        //     IN TABLENAME         VARCHAR(128),
+        //     IN PURGE_ROWS        SMALLINT,
+        //     IN DEFRAGMENT_ROWS   SMALLINT,
+        //     IN TRUNCATE_END      SMALLINT
+        //     )
+        {
+            // procedure argument names
+            String[] arg_names = {
+                "SCHEMANAME", 
+                "TABLENAME", 
+                "PURGE_ROWS", 
+                "DEFRAGMENT_ROWS", 
+                "TRUNCATE_END"};
+
+            // procedure argument types
+            TypeDescriptor[] arg_types = {
+                DataTypeDescriptor.getBuiltInDataTypeDescriptor(
+                    Types.VARCHAR, 128),
+                DataTypeDescriptor.getBuiltInDataTypeDescriptor(
+                    Types.VARCHAR, 128),
+                DataTypeDescriptor.getBuiltInDataTypeDescriptor(
+                    Types.SMALLINT),
+                DataTypeDescriptor.getBuiltInDataTypeDescriptor(
+                    Types.SMALLINT),
+                DataTypeDescriptor.getBuiltInDataTypeDescriptor(
+                    Types.SMALLINT)
+            };
+
+            createSystemProcedureOrFunction(
+                "SYSCS_INPLACE_COMPRESS_TABLE",
+                sysUtilUUID,
+                arg_names,
+                arg_types,
+				0,
+				0,
+                RoutineAliasInfo.MODIFIES_SQL_DATA,
+                (TypeDescriptor) null,
                 tc);
         }
 

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java Fri Apr 15 07:27:56 2005
@@ -1437,6 +1437,134 @@
 	}
 
 
+
+    /**
+     * Purge all committed deleted rows from the conglomerate.
+     * <p>
+     * This call will purge committed deleted rows from the conglomerate,
+     * that space will be available for future inserts into the conglomerate.
+     * <p>
+     *
+     * @param conglomId Id of the conglomerate to purge.
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+    public void purgeConglomerate(
+    long    conglomId)
+        throws StandardException
+    {
+        findExistingConglomerate(conglomId).purgeConglomerate(
+            this, 
+            rawtran);
+
+		return;
+    }
+
+    /**
+     * Return free space from the conglomerate back to the OS.
+     * <p>
+     * Returns free space from the conglomerate back to the OS.  Currently
+     * only the sequential free pages at the "end" of the conglomerate can
+     * be returned to the OS.
+     * <p>
+     *
+     * @param conglomId Id of the conglomerate to purge.
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+    public void compressConglomerate(
+    long    conglomId)
+        throws StandardException
+    {
+        findExistingConglomerate(conglomId).compressConglomerate(
+            this, 
+            rawtran); 
+
+		return;
+    }
+
+    /**
+     * Compress table in place
+     * <p>
+     * Returns a GroupFetchScanController which can be used to move rows
+     * around in a table, creating a block of free pages at the end of the
+     * table.  The process will move rows from the end of the table toward
+     * the beginning.  The GroupFetchScanController will return the 
+     * old row location, the new row location, and the actual data of any
+     * row moved.  Note that this scan only returns moved rows, not an
+     * entire set of rows, the scan is designed specifically to be
+     * used by either explicit user call of the SYSCS_ONLINE_COMPRESS_TABLE()
+     * procedure, or internal background calls to compress the table.
+     *
+     * The old and new row locations are returned so that the caller can
+     * update any indexes necessary.
+     *
+     * This scan always returns all collumns of the row.
+     * 
+     * All inputs work exactly as in openScan().  The return is 
+     * a GroupFetchScanController, which only allows fetches of groups
+     * of rows from the conglomerate.
+     * <p>
+     *
+	 * @return The GroupFetchScanController to be used to fetch the rows.
+     *
+	 * @param conglomId             see openScan()
+     * @param hold                  see openScan()
+     * @param open_mode             see openScan()
+     * @param lock_level            see openScan()
+     * @param isolation_level       see openScan()
+     *
+	 * @exception  StandardException  Standard exception policy.
+     *
+     * @see ScanController
+     * @see GroupFetchScanController
+     **/
+	public GroupFetchScanController defragmentConglomerate(
+    long                            conglomId,
+    boolean                         online,
+    boolean                         hold,
+    int                             open_mode,
+    int                             lock_level,
+    int                             isolation_level)
+        throws StandardException
+	{
+        if (SanityManager.DEBUG)
+        {
+			if ((open_mode & 
+                ~(TransactionController.OPENMODE_FORUPDATE | 
+                  TransactionController.OPENMODE_FOR_LOCK_ONLY |
+                  TransactionController.OPENMODE_SECONDARY_LOCKED)) != 0)
+				SanityManager.THROWASSERT(
+					"Bad open mode to openScan:" + 
+                    Integer.toHexString(open_mode));
+
+			if (!(lock_level == MODE_RECORD |
+                 lock_level == MODE_TABLE))
+				SanityManager.THROWASSERT(
+                "Bad lock level to openScan:" + lock_level);
+        }
+
+		// Find the conglomerate.
+		Conglomerate conglom = findExistingConglomerate(conglomId);
+
+		// Get a scan controller.
+		ScanManager sm = 
+            conglom.defragmentConglomerate(
+                this, 
+                rawtran, 
+                hold, 
+                open_mode, 
+                determine_lock_level(lock_level),
+                determine_locking_policy(lock_level, isolation_level),
+                isolation_level);
+
+		// Keep track of it so we can release on close.
+		scanControllers.addElement(sm);
+
+		return(sm);
+	}
+
+
 	public ScanController openScan(
     long                            conglomId,
     boolean                         hold,
@@ -1467,9 +1595,6 @@
                 (StaticCompiledOpenConglomInfo) null,
                 (DynamicCompiledOpenConglomInfo) null));
     }
-
-
-
 
 	public ScanController openCompiledScan(
     boolean                         hold,

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeController.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeController.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeController.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeController.java Fri Apr 15 07:27:56 2005
@@ -1018,11 +1018,18 @@
          throws StandardException
     {
 
-		if (this.container == null)       
-		{
-            throw StandardException.newException(
-                        SQLState.BTREE_IS_CLOSED,
-                        new Long(err_containerid));
+		if (isClosed())
+        {
+            if (getHold())
+            {
+                reopen();
+            }
+            else
+            {
+                throw StandardException.newException(
+                            SQLState.BTREE_IS_CLOSED,
+                            new Long(err_containerid));
+            } 
         }
 
         if (SanityManager.DEBUG)

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeScan.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeScan.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeScan.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTreeScan.java Fri Apr 15 07:27:56 2005
@@ -1838,6 +1838,19 @@
                 (int[]) null));
     }
 
+    public int fetchNextGroup(
+    DataValueDescriptor[][] row_array,
+    RowLocation[]           old_rowloc_array,
+    RowLocation[]           new_rowloc_array)
+        throws StandardException
+	{
+        // This interface is currently only used to move rows around in
+        // a heap table, unused in btree's -- so not implemented.
+
+        throw StandardException.newException(
+                SQLState.BTREE_UNIMPLEMENTED_FEATURE);
+    }
+
     /**
      * Insert all rows that qualify for the current scan into the input
      * Hash table.  

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I.java Fri Apr 15 07:27:56 2005
@@ -755,6 +755,48 @@
 	}
 
     /**
+     * Open a b-tree compress scan.
+     * <p>
+     * B2I does not support a compress scan.
+     * <p>
+	 * @see Conglomerate#openCompressScan
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+	public ScanManager defragmentConglomerate(
+    TransactionManager              xact_manager,
+    Transaction                     rawtran,
+    boolean                         hold,
+    int                             open_mode,
+    int                             lock_level,
+    LockingPolicy                   locking_policy,
+    int                             isolation_level)
+			throws StandardException
+	{
+        throw StandardException.newException(
+            SQLState.BTREE_UNIMPLEMENTED_FEATURE);
+	}
+
+	public void purgeConglomerate(
+    TransactionManager              xact_manager,
+    Transaction                     rawtran)
+        throws StandardException
+    {
+        // currently on work to do in btree's for purge rows, purging
+        // happens best when split is about to happen.
+        return;
+    }
+
+	public void compressConglomerate(
+    TransactionManager              xact_manager,
+    Transaction                     rawtran)
+        throws StandardException
+    {
+        // TODO - need to implement for btree
+        return;
+    }
+
+    /**
      * Return an open StoreCostController for the conglomerate.
      * <p>
      * Return an open StoreCostController which can be used to ask about 

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/GenericScanController.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/GenericScanController.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/GenericScanController.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/GenericScanController.java Fri Apr 15 07:27:56 2005
@@ -833,7 +833,6 @@
 		return(ret_row_count);
     }
 
-
     /**
     Reposition the current scan.  This call is semantically the same as if
     the current scan had been closed and a openScan() had been called instead.
@@ -888,7 +887,6 @@
                 SQLState.HEAP_UNIMPLEMENTED_FEATURE));
     }
 
-
     /**************************************************************************
      * abstract protected Methods of This class:
      **************************************************************************
@@ -1017,6 +1015,11 @@
     boolean closeHeldScan)
         throws StandardException
 	{
+        SanityManager.DEBUG_PRINT("GenericScanController.closeForEndTransaction", 
+                "closeHeldScan = " + closeHeldScan +
+                "open_conglom.getHold() = " + open_conglom.getHold());
+                
+
         if ((!open_conglom.getHold()) || closeHeldScan) 
         {
             // close the scan as part of the commit/abort

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/OpenConglomerate.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/OpenConglomerate.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/OpenConglomerate.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/OpenConglomerate.java Fri Apr 15 07:27:56 2005
@@ -141,7 +141,6 @@
      *     unlockPositionAfterRead(RowPosition)
      **************************************************************************
      */
-
     /**
      * Latch the page containing the current RowPosition, and reposition scan.
      * <p>
@@ -166,7 +165,7 @@
      *
 	 * @exception  StandardException  Standard exception policy.
      **/
-    protected boolean latchPageAndRepositionScan(RowPosition pos)
+    public boolean latchPageAndRepositionScan(RowPosition pos)
 		throws StandardException
     {
         boolean scan_repositioned = false;
@@ -176,8 +175,11 @@
 
         try
         {
-            pos.current_page = 
-                container.getPage(pos.current_rh.getPageNumber());
+            if (pos.current_rh != null)
+            {
+                pos.current_page = 
+                    container.getPage(pos.current_rh.getPageNumber());
+            }
 
         }
         catch (Throwable t)
@@ -243,10 +245,28 @@
         if (pos.current_page == null)
         {
             // position on the next page.
-            pos.current_page = 
-                container.getNextPage(pos.current_rh.getPageNumber());
+            long current_pageno;
+
+            if (pos.current_rh != null)
+            {
+                current_pageno = pos.current_rh.getPageNumber();
+            }
+            else if (pos.current_pageno != ContainerHandle.INVALID_PAGE_NUMBER)
+            {
+                current_pageno = pos.current_pageno;
+            }
+            else
+            {
+                // no valid position, return a null page
+                return(false);
+            }
+
+            pos.current_page = container.getNextPage(current_pageno);
+
+            pos.current_slot   = Page.FIRST_SLOT_NUMBER - 1;
 
-            pos.current_slot = Page.FIRST_SLOT_NUMBER - 1;
+            // now position is tracked by active page
+            pos.current_pageno = ContainerHandle.INVALID_PAGE_NUMBER;
 
             scan_repositioned = true;
         }

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/RowPosition.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/RowPosition.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/RowPosition.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/RowPosition.java Fri Apr 15 07:27:56 2005
@@ -22,6 +22,7 @@
 
 import org.apache.derby.iapi.services.sanity.SanityManager;
 
+import org.apache.derby.iapi.store.raw.ContainerHandle;
 import org.apache.derby.iapi.store.raw.Page;
 import org.apache.derby.iapi.store.raw.RecordHandle;
 
@@ -42,6 +43,7 @@
     public RecordHandle    current_rh;
     public int             current_slot;
     public boolean         current_rh_qualified;
+    public long            current_pageno;
 
     /**************************************************************************
      * Constructors for This class:
@@ -67,6 +69,7 @@
         current_rh              = null;
         current_slot            = Page.INVALID_SLOT_NUMBER;
         current_rh_qualified    = false;
+        current_pageno          = ContainerHandle.INVALID_PAGE_NUMBER;
     }
 
     public final void positionAtNextSlot()
@@ -75,6 +78,12 @@
         current_rh   = null;
     }
 
+    public final void positionAtPrevSlot()
+    {
+        current_slot--;
+        current_rh   = null;
+    }
+
     public void unlatch()
     {
         if (current_page != null)
@@ -94,7 +103,12 @@
             ret_string = 
                 ";current_slot=" + current_slot +
                 ";current_rh=" + current_rh +
-                ";current_page=" + current_page;
+                ";current_pageno=" + current_pageno +
+                ";current_page=" + 
+                    (current_page == null ? 
+                         "null" : String.valueOf(current_page.getPageNumber()));
+
+                // ";current_page=" + current_page;
         }
 
         return(ret_string);

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java?view=diff&r1=161467&r2=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java Fri Apr 15 07:27:56 2005
@@ -62,6 +62,7 @@
 import org.apache.derby.iapi.store.raw.Transaction;
 import org.apache.derby.iapi.store.raw.Page;
 import org.apache.derby.iapi.store.raw.RawStoreFactory;
+import org.apache.derby.iapi.store.raw.RecordHandle;
 
 import org.apache.derby.iapi.types.DataValueDescriptor;
 
@@ -650,7 +651,7 @@
     int                             lock_level,
     LockingPolicy                   locking_policy,
     int                             isolation_level,
-	FormatableBitSet				            scanColumnList,
+	FormatableBitSet				scanColumnList,
     DataValueDescriptor[]	        startKeyValue,
     int                             startSearchOperator,
     Qualifier                       qualifier[][],
@@ -700,6 +701,220 @@
             stopSearchOperator);
 
 		return(heapscan);
+	}
+
+	public void purgeConglomerate(
+    TransactionManager              xact_manager,
+    Transaction                     rawtran)
+        throws StandardException
+    {
+        OpenConglomerate        open_for_ddl_lock   = null;
+        HeapController          heapcontroller      = null;
+        TransactionManager      nested_xact         = null;
+
+        try
+        {
+            open_for_ddl_lock = new OpenHeap();
+
+            // Open table in intended exclusive mode in the top level 
+            // transaction, this will stop any ddl from happening until 
+            // purge of whole table is finished.
+
+            if (open_for_ddl_lock.init(
+                    (ContainerHandle) null,
+                    this,
+                    this.format_ids,
+                    xact_manager,
+                    rawtran,
+                    false,
+                    TransactionController.OPENMODE_FORUPDATE,
+                    TransactionController.MODE_RECORD,
+                    null,
+                    null) == null)
+            {
+                throw StandardException.newException(
+                        SQLState.HEAP_CONTAINER_NOT_FOUND, 
+                        new Long(id.getContainerId()));
+            }
+
+            // perform all the "real" work in a non-readonly nested user 
+            // transaction, so that as work is completed on each page resources
+            // can be released.  Must be careful as all locks obtained in nested
+            // transaction will conflict with parent transaction - so this call
+            // must be made only if parent transaction can have no conflicting
+            // locks on the table, otherwise the purge will fail with a self
+            // deadlock.
+            nested_xact = (TransactionManager) 
+                xact_manager.startNestedUserTransaction(false);
+
+            // now open the table in a nested user transaction so that each
+            // page worth of work can be committed after it is done.
+
+            OpenConglomerate open_conglom = new OpenHeap();
+
+            if (open_conglom.init(
+                (ContainerHandle) null,
+                this,
+                this.format_ids,
+                nested_xact,
+                rawtran,
+                true,
+                TransactionController.OPENMODE_FORUPDATE,
+                TransactionController.MODE_RECORD,
+                null,
+                null) == null)
+            {
+                throw StandardException.newException(
+                        SQLState.HEAP_CONTAINER_NOT_FOUND, 
+                        new Long(id.getContainerId()).toString());
+            }
+
+            heapcontroller = new HeapController();
+
+            heapcontroller.init(open_conglom);
+
+            Page page   = open_conglom.getContainer().getFirstPage();
+
+            boolean purgingDone = false;
+
+            while (page != null)
+            {
+                long pageno = page.getPageNumber();
+                purgingDone = heapcontroller.purgeCommittedDeletes(page);
+
+                if (purgingDone)
+                {
+                    page = null;
+
+                    // commit xact to free resouurces ASAP, commit will
+                    // unlatch the page if it has not already been unlatched
+                    // by a remove.
+                    open_conglom.getXactMgr().commitNoSync(
+                                TransactionController.RELEASE_LOCKS);
+                    open_conglom.reopen();
+                }
+                else
+                {
+                    page.unlatch();
+                    page = null;
+                }
+
+                page = open_conglom.getContainer().getNextPage(pageno);
+            }
+        }
+        finally
+        {
+            if (open_for_ddl_lock != null)
+                open_for_ddl_lock.close();
+            if (heapcontroller != null)
+                heapcontroller.close();
+            if (nested_xact != null)
+            {
+                nested_xact.commitNoSync(TransactionController.RELEASE_LOCKS);
+                nested_xact.destroy();
+            }
+        }
+
+        return;
+    }
+
+	public void compressConglomerate(
+    TransactionManager              xact_manager,
+    Transaction                     rawtran)
+        throws StandardException
+    {
+        OpenConglomerate        open_conglom    = null;
+        HeapController          heapcontroller  = null;
+
+        try
+        {
+            open_conglom = new OpenHeap();
+
+            // Open table in intended exclusive mode in the top level 
+            // transaction, this will stop any ddl from happening until 
+            // purge of whole table is finished.
+
+            if (open_conglom.init(
+                    (ContainerHandle) null,
+                    this,
+                    this.format_ids,
+                    xact_manager,
+                    rawtran,
+                    false,
+                    TransactionController.OPENMODE_FORUPDATE,
+                    TransactionController.MODE_RECORD,
+                    null,
+                    null) == null)
+            {
+                throw StandardException.newException(
+                        SQLState.HEAP_CONTAINER_NOT_FOUND, 
+                        new Long(id.getContainerId()));
+            }
+
+            heapcontroller = new HeapController();
+
+            heapcontroller.init(open_conglom);
+
+            open_conglom.getContainer().compressContainer();
+        }
+        finally
+        {
+            if (open_conglom != null)
+                open_conglom.close();
+        }
+
+        return;
+    }
+
+    /**
+     * Open a heap compress scan.
+     * <p>
+     *
+     * @see Conglomerate#openCompressScan
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+	public ScanManager defragmentConglomerate(
+    TransactionManager              xact_manager,
+    Transaction                     rawtran,
+    boolean                         hold,
+    int                             open_mode,
+    int                             lock_level,
+    LockingPolicy                   locking_policy,
+    int                             isolation_level)
+		throws StandardException
+	{
+        OpenConglomerate open_conglom = new OpenHeap();
+
+        if (open_conglom.init(
+                (ContainerHandle) null,
+                this,
+                this.format_ids,
+                xact_manager,
+                rawtran,
+                hold,
+                open_mode,
+                lock_level,
+                null,
+                null) == null)
+        {
+            throw StandardException.newException(
+                    SQLState.HEAP_CONTAINER_NOT_FOUND, 
+                    new Long(id.getContainerId()));
+        }
+
+		HeapCompressScan heap_compress_scan = new HeapCompressScan();
+
+        heap_compress_scan.init(
+            open_conglom,
+            null,
+            null,
+            0,
+            null,
+            null,
+            0);
+
+		return(heap_compress_scan);
 	}
 
 

Added: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapCompressScan.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapCompressScan.java?view=auto&rev=161468
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapCompressScan.java (added)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapCompressScan.java Fri Apr 15 07:27:56 2005
@@ -0,0 +1,439 @@
+/*
+
+   Derby - Class org.apache.derby.impl.store.access.heap.HeapScan
+
+   Copyright 2005 The Apache Software Foundation or its licensors, as applicable.
+
+   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 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.impl.store.access.heap;
+
+
+/**
+
+  A heap scan object represents an instance of an scan on a heap conglomerate.
+
+**/
+
+import org.apache.derby.iapi.reference.SQLState;
+
+import org.apache.derby.iapi.services.sanity.SanityManager;
+
+import org.apache.derby.iapi.services.io.Storable;
+
+import org.apache.derby.iapi.error.StandardException;
+
+import org.apache.derby.iapi.store.access.conglomerate.Conglomerate;
+import org.apache.derby.iapi.store.access.conglomerate.LogicalUndo;
+import org.apache.derby.iapi.store.access.conglomerate.ScanManager;
+import org.apache.derby.iapi.store.access.conglomerate.TransactionManager;
+
+import org.apache.derby.iapi.store.access.ConglomerateController;
+import org.apache.derby.iapi.store.access.DynamicCompiledOpenConglomInfo;
+import org.apache.derby.iapi.store.access.Qualifier;
+import org.apache.derby.iapi.store.access.RowUtil;
+import org.apache.derby.iapi.store.access.ScanInfo;
+import org.apache.derby.iapi.store.access.ScanController;
+import org.apache.derby.iapi.store.access.SpaceInfo;
+import org.apache.derby.iapi.store.access.TransactionController;
+
+import org.apache.derby.iapi.types.RowLocation;
+
+import org.apache.derby.iapi.store.raw.ContainerHandle;
+import org.apache.derby.iapi.store.raw.LockingPolicy;
+import org.apache.derby.iapi.store.raw.Transaction;
+import org.apache.derby.iapi.store.raw.Page;
+import org.apache.derby.iapi.store.raw.RecordHandle;
+
+import org.apache.derby.iapi.types.DataValueDescriptor;
+
+import org.apache.derby.impl.store.access.conglomerate.ConglomerateUtil;
+import org.apache.derby.impl.store.access.conglomerate.GenericScanController;
+import org.apache.derby.impl.store.access.conglomerate.RowPosition;
+
+import org.apache.derby.iapi.store.access.BackingStoreHashtable;
+import org.apache.derby.iapi.services.io.FormatableBitSet;
+
+import java.util.Hashtable;
+import java.util.Vector;
+
+class HeapCompressScan 
+    extends HeapScan
+{
+
+    /**************************************************************************
+     * Constants of HeapScan
+     **************************************************************************
+     */
+
+    /**************************************************************************
+     * Fields of HeapScan
+     **************************************************************************
+     */
+    private long pagenum_to_start_moving_rows = -1;
+
+
+
+    /**************************************************************************
+     * Constructors for This class:
+     **************************************************************************
+     */
+
+	/**
+	 ** The only constructor for a HeapCompressScan returns a scan in the
+	 ** closed state, the caller must call open.
+	 **/
+	
+	public HeapCompressScan()
+	{
+	}
+
+    /**************************************************************************
+     * Protected override implementation of routines in
+     *     GenericController class:
+     **************************************************************************
+     */
+
+    public int fetchNextGroup(
+    DataValueDescriptor[][] row_array,
+    RowLocation[]           old_rowloc_array,
+    RowLocation[]           new_rowloc_array)
+        throws StandardException
+	{
+        return(fetchRowsForCompress(
+                    row_array, old_rowloc_array, new_rowloc_array));
+    }
+
+    /**
+     * Fetch the next N rows from the table.
+     * <p>
+     * Utility routine used by both fetchSet() and fetchNextGroup().
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+    private int fetchRowsForCompress(
+    DataValueDescriptor[][] row_array,
+    RowLocation[]           oldrowloc_array,
+    RowLocation[]           newrowloc_array)
+        throws StandardException
+	{
+        int                     ret_row_count           = 0;
+        DataValueDescriptor[]   fetch_row               = null;
+
+        if (SanityManager.DEBUG)
+        {
+            SanityManager.ASSERT(row_array != null);
+            SanityManager.ASSERT(row_array[0] != null,
+                    "first array slot in fetchNextGroup() must be non-null.");
+        }
+
+        if (getScanState() == SCAN_INPROGRESS)
+        {
+            positionAtResumeScan(scan_position);
+        }
+        else if (getScanState() == SCAN_INIT)
+        {
+            // For first implementation of defragment use a conservative
+            // approach, only move rows from the last "number of free pages"
+            // of the container.  Should always at least be able to empty
+            // that number of pages.
+            SpaceInfo info = 
+                open_conglom.getContainer().getSpaceInfo();
+
+            pagenum_to_start_moving_rows = info.getNumAllocatedPages();
+
+            positionAtStartForForwardScan(scan_position);
+        }
+        else if (getScanState() == SCAN_HOLD_INPROGRESS)
+        {
+            open_conglom.reopen();
+
+            if (SanityManager.DEBUG)
+            {
+                SanityManager.ASSERT(
+                    scan_position.current_rh != null, this.toString()); 
+            }
+
+            // reposition the scan at the row just before the next one to 
+            // return.
+            // This routine handles the mess of repositioning if the row or 
+            // the page has disappeared. This can happen if a lock was not 
+            // held on the row while not holding the latch.
+            open_conglom.latchPageAndRepositionScan(scan_position);
+
+            setScanState(SCAN_INPROGRESS);
+        }
+        else if (getScanState() == SCAN_HOLD_INIT)
+        {
+            open_conglom.reopen();
+
+            positionAtStartForForwardScan(scan_position);
+
+        }
+        else
+        {
+            if (SanityManager.DEBUG)
+                SanityManager.ASSERT(getScanState() == SCAN_DONE);
+
+            return(0);
+        }
+
+        // At this point:
+        // scan_position.current_page is latched.  
+        // scan_position.current_slot is the slot on scan_position.current_page
+        // just before the "next" record this routine should process.
+
+        // loop through successive pages and successive slots on those
+        // pages.  Stop when either the last page is reached 
+        // (scan_position.current_page will be null).  
+        // Along the way apply qualifiers to skip rows which don't qualify.
+
+		while (scan_position.current_page != null)
+		{
+			while ((scan_position.current_slot + 1) < 
+                    scan_position.current_page.recordCount())
+			{
+                // Allocate a new row to read the row into.
+                if (fetch_row == null)
+                {
+                     // point at allocated row in array if one exists.
+                    if (row_array[ret_row_count] == null)
+                    {
+                        row_array[ret_row_count] = 
+                          open_conglom.getRuntimeMem().get_row_for_export();
+                    }
+
+                    fetch_row = row_array[ret_row_count];
+                }
+
+                // move scan current position forward.
+                scan_position.positionAtNextSlot();
+
+                this.stat_numrows_visited++;
+
+                if (scan_position.current_page.isDeletedAtSlot(
+                        scan_position.current_slot))
+                {
+                    // At this point assume table level lock, and that this
+                    // transcation did not delete the row, so any
+                    // deleted row must be a committed deleted row which can
+                    // be purged.
+                    scan_position.current_page.purgeAtSlot(
+                        scan_position.current_slot, 1, false);
+
+                    // raw store shuffles following rows down, so 
+                    // postion the scan at previous slot, so next trip
+                    // through loop will pick up correct row.
+                    scan_position.positionAtPrevSlot();
+                    continue;
+                }
+
+                if (scan_position.current_page.getPageNumber() > 
+                        pagenum_to_start_moving_rows)
+                {
+                    // Give raw store a chance to move the row for compression
+                    RecordHandle[] old_handle = new RecordHandle[1];
+                    RecordHandle[] new_handle = new RecordHandle[1];
+                    long[]         new_pageno = new long[1];
+
+                    if (scan_position.current_page.moveRecordForCompressAtSlot(
+                            scan_position.current_slot,
+                            fetch_row,
+                            old_handle,
+                            new_handle) == 1)
+                    {
+                        // raw store moved the row, so bump the row count but 
+                        // postion the scan at previous slot, so next trip
+                        // through loop will pick up correct row.
+                        // The subsequent rows will have been moved forward
+                        // to take place of moved row.
+                        scan_position.positionAtPrevSlot();
+
+                        ret_row_count++;
+                        stat_numrows_qualified++;
+
+
+                        setRowLocationArray(
+                            oldrowloc_array, ret_row_count - 1, old_handle[0]);
+                        setRowLocationArray(
+                            newrowloc_array, ret_row_count - 1, new_handle[0]);
+
+                        fetch_row = null;
+
+                    }
+                }
+			}
+
+            this.stat_numpages_visited++;
+
+            if (scan_position.current_page.recordCount() == 0)
+            {
+                // need to set the scan position before removing page
+                scan_position.current_pageno = 
+                    scan_position.current_page.getPageNumber();
+
+                open_conglom.getContainer().removePage(
+                    scan_position.current_page);
+
+                // removePage unlatches the page, and page not available
+                // again until after commit.
+                scan_position.current_page = null;
+            }
+            else
+            {
+                positionAfterThisPage(scan_position);
+                scan_position.unlatch();
+            }
+
+
+            if (ret_row_count > 0)
+            {
+                // rows were moved on this page, give caller a chance to
+                // process those and free up access to the table.
+                return(ret_row_count);
+            }
+            else
+            {
+                // no rows were moved so go ahead and commit the transaction
+                // to allow other threads a chance at table.  Compress does
+                // need to sync as long as transaction either completely 
+                // commits or backs out, either is fine.
+                /*
+                open_conglom.getXactMgr().commitNoSync(
+                    TransactionController.RELEASE_LOCKS);
+                open_conglom.reopen();
+                */
+                positionAtResumeScan(scan_position);
+
+            }
+		}
+
+        // Reached last page of scan.
+        positionAtDoneScan(scan_position);
+
+        // we need to decrement when we stop scan at the end of the table.
+        this.stat_numpages_visited--;
+
+		return(ret_row_count);
+    }
+
+    /**
+     * Reposition the scan upon entering the fetchRows loop.
+     * <p>
+     * Called upon entering fetchRows() while in the SCAN_INPROGRESS state.
+     * Do work necessary to look at rows in the current page of the scan.
+     * <p>
+     * The default implementation uses a record handle to maintain a scan
+     * position.  It will get the latch again on the current
+     * scan position and set the slot to the current record handle.
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+    protected void positionAtResumeScan(
+    RowPosition pos)
+		throws StandardException
+    {
+        // reposition the scan at the row just before the next one to return.
+        // This routine handles the mess of repositioning if the row or the
+        // page has disappeared. This can happen if a lock was not held on the
+        // row while not holding the latch.
+        open_conglom.latchPageAndRepositionScan(scan_position);
+    }
+
+    /**
+     * Move the scan from SCAN_INIT to SCAN_INPROGRESS.
+     * <p>
+     * This routine is called to move the scan from SCAN_INIT to 
+     * SCAN_INPROGRESS.  Upon return from this routine it is expected
+     * that scan_position is set such that calling the generic 
+     * scan loop will reach the first row of the scan.  Note that this
+     * usually means setting the scan_postion to one before the 1st 
+     * row to be returned.
+     * <p>
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+    protected void positionAtStartForForwardScan(
+    RowPosition pos)
+        throws StandardException
+    {
+        if (pos.current_rh == null)
+        {
+            // 1st positioning of scan (delayed from openScan).  Do not
+            // compress the first page, there is no previous page to move
+            // rows to, and moving the special Heap metadata row from the
+            // first page would cause problems.  Setting to next page is
+            // why this scan overrides generic implementation.
+            pos.current_page = 
+                open_conglom.getContainer().getNextPage(
+                    ContainerHandle.FIRST_PAGE_NUMBER);
+
+            // set up for scan to continue at beginning of page following
+            // the first page of the container.
+            pos.current_slot = Page.FIRST_SLOT_NUMBER - 1;
+        }
+        else
+        {
+            // 1st positioning of scan following a reopenScanByRowLocation
+
+            // reposition the scan at the row just before the next one to 
+            // return.  This routine handles the mess of repositioning if the 
+            // row or the page has disappeared. This can happen if a lock was 
+            // not held on the row while not holding the latch.
+            open_conglom.latchPageAndRepositionScan(pos);
+
+            // set up for scan to at the specified record handle (position one
+            // before it so that the loop increment and find it).
+            pos.current_slot -= 1;
+        }
+
+        pos.current_rh              = null;
+        this.stat_numpages_visited  = 1;
+        this.setScanState(SCAN_INPROGRESS);
+    }
+
+
+    /**************************************************************************
+     * Private/Protected methods of This class:
+     **************************************************************************
+     */
+
+    /**
+     * Set scan position to just after current page.
+     * <p>
+     * Used to set the position of the scan if a record handle is not
+     * avaliable.  In this case current_rh will be set to null, and 
+     * current_pageno will be set to the current page number.
+     * On resume of the scan, the scan will be set to just before the first
+     * row returned form a getNextPage(current_pageno) call.
+     * <p>
+     * A positionAtResumeScan(scan_position) is necessary to continue the
+     * scan after this call.
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+    private void positionAfterThisPage(
+    RowPosition pos)
+        throws StandardException
+    {
+        pos.current_rh = null;
+        pos.current_pageno = pos.current_page.getPageNumber();
+    }
+
+	/*
+	** Methods of ScanManager
+	*/
+
+}