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 2014/11/24 17:37:39 UTC

svn commit: r1641418 - in /db/derby/code/trunk/java: build/org/apache/derbyBuild/lastgoodjarcontents/ engine/org/apache/derby/iapi/store/access/conglomerate/ engine/org/apache/derby/iapi/store/raw/ engine/org/apache/derby/iapi/store/raw/data/ engine/or...

Author: mikem
Date: Mon Nov 24 16:37:38 2014
New Revision: 1641418

URL: http://svn.apache.org/r1641418
Log:
DERBY-4057 Space is not reclaimed if transaction is rolled back

Added infrastructure called at insert abort time to queue post abort work.
After the abort work is queued in the case of aborted inserts to reclaim
space and if possible mark pages free, which then in turn allows them to
be used by subsequent work on the table.  This work queues this work
on heap tables when the aborted insert is the last row on a page, or
if the aborted insert is a row that contains a long column (a row that
is bigger than a page - usually a blob or clob), or a long row (a row
that spans multiple pages).


Added:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/UndoHandler.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessUndoHandler.java
Modified:
    db/derby/code/trunk/java/build/org/apache/derbyBuild/lastgoodjarcontents/insane.derby.jar.lastcontents
    db/derby/code/trunk/java/build/org/apache/derbyBuild/lastgoodjarcontents/sane.derby.jar.lastcontents
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/ConglomerateFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/TransactionManager.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Page.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Transaction.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessManager.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/index/B2IFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapConglomerateFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapController.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapPostCommit.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapScan.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BasePage.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InsertOperation.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/StoredPage.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/ClobReclamationTest.java

Modified: db/derby/code/trunk/java/build/org/apache/derbyBuild/lastgoodjarcontents/insane.derby.jar.lastcontents
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/build/org/apache/derbyBuild/lastgoodjarcontents/insane.derby.jar.lastcontents?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/build/org/apache/derbyBuild/lastgoodjarcontents/insane.derby.jar.lastcontents (original)
+++ db/derby/code/trunk/java/build/org/apache/derbyBuild/lastgoodjarcontents/insane.derby.jar.lastcontents Mon Nov 24 16:37:38 2014
@@ -416,6 +416,7 @@ org.apache.derby.iapi.store.raw.RowLock.
 org.apache.derby.iapi.store.raw.ScanHandle.class
 org.apache.derby.iapi.store.raw.StreamContainerHandle.class
 org.apache.derby.iapi.store.raw.Transaction.class
+org.apache.derby.iapi.store.raw.UndoHandler.class
 org.apache.derby.iapi.store.raw.Undoable.class
 org.apache.derby.iapi.store.raw.data.DataFactory.class
 org.apache.derby.iapi.store.raw.data.RawContainerHandle.class
@@ -1169,6 +1170,7 @@ org.apache.derby.impl.store.access.Cache
 org.apache.derby.impl.store.access.PC_XenaVersion.class
 org.apache.derby.impl.store.access.PropertyConglomerate.class
 org.apache.derby.impl.store.access.RAMAccessManager.class
+org.apache.derby.impl.store.access.RAMAccessUndoHandler.class
 org.apache.derby.impl.store.access.RAMTransaction.class
 org.apache.derby.impl.store.access.RAMTransactionContext.class
 org.apache.derby.impl.store.access.RllRAMAccessManager.class

Modified: db/derby/code/trunk/java/build/org/apache/derbyBuild/lastgoodjarcontents/sane.derby.jar.lastcontents
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/build/org/apache/derbyBuild/lastgoodjarcontents/sane.derby.jar.lastcontents?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/build/org/apache/derbyBuild/lastgoodjarcontents/sane.derby.jar.lastcontents (original)
+++ db/derby/code/trunk/java/build/org/apache/derbyBuild/lastgoodjarcontents/sane.derby.jar.lastcontents Mon Nov 24 16:37:38 2014
@@ -417,6 +417,7 @@ org.apache.derby.iapi.store.raw.RowLock.
 org.apache.derby.iapi.store.raw.ScanHandle.class
 org.apache.derby.iapi.store.raw.StreamContainerHandle.class
 org.apache.derby.iapi.store.raw.Transaction.class
+org.apache.derby.iapi.store.raw.UndoHandler.class
 org.apache.derby.iapi.store.raw.Undoable.class
 org.apache.derby.iapi.store.raw.data.DataFactory.class
 org.apache.derby.iapi.store.raw.data.RawContainerHandle.class
@@ -1172,6 +1173,7 @@ org.apache.derby.impl.store.access.Cache
 org.apache.derby.impl.store.access.PC_XenaVersion.class
 org.apache.derby.impl.store.access.PropertyConglomerate.class
 org.apache.derby.impl.store.access.RAMAccessManager.class
+org.apache.derby.impl.store.access.RAMAccessUndoHandler.class
 org.apache.derby.impl.store.access.RAMTransaction.class
 org.apache.derby.impl.store.access.RAMTransactionContext.class
 org.apache.derby.impl.store.access.RllRAMAccessManager.class

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/ConglomerateFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/ConglomerateFactory.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/ConglomerateFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/ConglomerateFactory.java Mon Nov 24 16:37:38 2014
@@ -27,9 +27,11 @@ import org.apache.derby.catalog.UUID;
 
 import org.apache.derby.iapi.error.StandardException;
 
+import org.apache.derby.iapi.store.access.AccessFactory;
 import org.apache.derby.iapi.store.access.ColumnOrdering;
 
 import org.apache.derby.iapi.store.raw.ContainerKey;
+import org.apache.derby.iapi.store.raw.PageKey;
 import org.apache.derby.iapi.store.raw.Transaction;
 
 import org.apache.derby.iapi.types.DataValueDescriptor;
@@ -117,4 +119,31 @@ public interface ConglomerateFactory ext
     TransactionManager      xact_mgr,
     ContainerKey            container_key)
 		throws StandardException;
+
+
+    /**
+     * Interface to be called when an undo of an insert is processed.
+     * <p>
+     * Implementer of this class provides interface to be called by the raw
+     * store when an undo of an insert is processed.  Initial implementation
+     * will be by Access layer to queue space reclaiming events if necessary
+     * when a rows is logically "deleted" as part of undo of the original
+     * insert.  This undo can happen a lot for many applications if they
+     * generate expected and handled duplicate key errors.
+     * <p>
+     * Caller may decide to call or not based on deleted row count of the
+     * page, or if overflow rows/columns are present.
+     *
+     *
+     * @param access_factory    current access_factory of the aborted insert.
+     * @param xact              transaction that is being backed out.
+     * @param page_key          page key of the aborted insert.
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+    public void insertUndoNotify(
+    AccessFactory       access_factory,
+    Transaction         xact,
+    PageKey             page_key)
+        throws StandardException;
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/TransactionManager.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/TransactionManager.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/TransactionManager.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/TransactionManager.java Mon Nov 24 16:37:38 2014
@@ -21,6 +21,7 @@
 
 package org.apache.derby.iapi.store.access.conglomerate;
 
+import org.apache.derby.iapi.store.raw.ContainerKey;
 import org.apache.derby.iapi.services.daemon.Serviceable;
 import org.apache.derby.iapi.store.access.ConglomerateController;
 import org.apache.derby.iapi.store.access.SortController;
@@ -169,4 +170,20 @@ public interface TransactionManager exte
      **/
     public Transaction getRawStoreXact()
         throws StandardException;
+
+    /**
+     * Return existing Conglomerate after doing lookup by ContainerKey
+     * <p>
+     * Throws exception if it can't find a matching conglomerate for the 
+     * ContainerKey.
+     * 
+     * @return If successful returns 
+     *
+     * @param container_key  container key of target conglomerate.
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+	public Conglomerate findExistingConglomerateFromKey(
+    ContainerKey container_key)
+		throws StandardException;
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Page.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Page.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Page.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Page.java Mon Nov 24 16:37:38 2014
@@ -1,4 +1,5 @@
 /*
+/usr/bin/mksh: p4: not found
 
    Derby - Class org.apache.derby.iapi.store.raw.Page
 
@@ -121,6 +122,18 @@ public interface Page  
      **/
 	public long getPageNumber();
 
+    /**
+     * Return the page key of this page. 
+     * <p>
+     *
+     * <BR> MT - Latched
+     *
+     * @see ContainerHandle
+     *
+	 * @return The page key of this page.
+     **/
+	public PageKey getPageKey();
+
     /**************************************************************************
      * Public Methods of This class: record handle interface.
      *     the following interfaces to page use the record Id or record handle

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java Mon Nov 24 16:37:38 2014
@@ -429,6 +429,25 @@ public interface RawStoreFactory extends
     */
     public LockFactory getLockFactory();
 
+    /**
+        Register a handler class for insert undo events.
+        <P>
+        Register a class to be called when an undo of an insert 
+        is executed.  When an undo of an event is executed by
+        the raw store UndoHandler.insertUndoNotify() will be
+        called, allowing upper level callers to execute code
+        as necessary.  The initial need is for the access layer
+        to be able to queue post commit reclaim space in the
+        case of inserts which are aborted (including the normal
+        case of inserts failed for duplicate key violations)
+        (see DERBY-4057)
+
+        @exception StandardException Standard Derby error policy
+
+    */
+    public void setUndoInsertEventHandler(
+        UndoHandler undo_handle) throws StandardException;
+
 
     /**
         Create a user transaction, almost all work within the raw store is

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Transaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Transaction.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Transaction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Transaction.java Mon Nov 24 16:37:38 2014
@@ -492,6 +492,14 @@ public interface Transaction {
 	public void addPostCommitWork(Serviceable work);
 
 	/**
+		Add to the list of post abort work that may be processed after this
+		transaction aborts.  
+
+		@param work the post commit work that is added
+	*/
+	public void addPostAbortWork(Serviceable work);
+
+	/**
 		Add to the list of post termination work that may be processed after this
 		transaction commits or aborts.
 

Added: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/UndoHandler.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/UndoHandler.java?rev=1641418&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/UndoHandler.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/UndoHandler.java Mon Nov 24 16:37:38 2014
@@ -0,0 +1,98 @@
+/*
+
+   Derby - Class org.apache.derby.iapi.store.raw.UndoHandler
+
+   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.store.raw;
+
+import org.apache.derby.iapi.store.raw.PageKey;
+import org.apache.derby.iapi.store.raw.Transaction;
+
+import org.apache.derby.iapi.error.StandardException; 
+
+
+/**
+	A class that provides interface to be called with undo of an Insert
+    happens in raw store.
+*/
+
+
+public interface UndoHandler
+{
+    /**************************************************************************
+     * Fields of the class
+     **************************************************************************
+     */
+
+    /**************************************************************************
+     * Constructors for This class:
+     **************************************************************************
+     */
+
+    /**************************************************************************
+     * Private/Protected methods of This class:
+     **************************************************************************
+     */
+
+    /**************************************************************************
+     * Public Methods of This class:
+     **************************************************************************
+     */
+    /**
+     * Interface to be called when an undo of an insert is processed.
+     * <p>
+     * Implementer of this class provides interface to be called by the raw
+     * store when an undo of an insert is processed.  Initial implementation
+     * will be by Access layer to queue space reclaiming events if necessary
+     * when a rows is logically "deleted" as part of undo of the original
+     * insert.  This undo can happen a lot for many applications if they
+     * generate expected and handled duplicate key errors.
+     * <p>
+     * It may be useful at some time to include the recordId of the deleted
+     * row, but it is not used currently by those notified.  The post commit
+     * work ultimately processes all rows on the table while
+     * it has the latch which is more efficient than one row at time per latch.
+     * <p>
+     * It is expected that notifies only happen for pages that caller
+     * is interested in.  Currently only the following aborted inserts
+     * cause a notify:
+     * o must be on a non overflow page
+     * o if all "user" rows on page are deleted a notify happens (page 1 
+     *   has a system row so on page one notifies happen if all but the first
+     *   row is deleted).
+     * o if the aborted insert row has either an overflow row or column
+     *   component then the notify is executed.
+     *
+     * @param xact      transaction that is being backed out.
+     * @param page_key  key that uniquely identifies page in question, container
+     *                  key information is embedded in the PageKey
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+    public void insertUndoNotify(
+    Transaction         xact,
+    PageKey             page_key)
+       throws StandardException;
+
+
+    /**************************************************************************
+     * Public Methods of XXXX class:
+     **************************************************************************
+     */
+}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java Mon Nov 24 16:37:38 2014
@@ -35,6 +35,7 @@ import org.apache.derby.iapi.store.raw.R
 import org.apache.derby.iapi.store.raw.StreamContainerHandle;
 import org.apache.derby.iapi.store.raw.xact.RawTransaction;
 import org.apache.derby.iapi.store.raw.Transaction;
+import org.apache.derby.iapi.store.raw.UndoHandler;
 import org.apache.derby.io.StorageFactory;
 import org.apache.derby.iapi.store.access.FileResource;
 import org.apache.derby.iapi.store.access.RowSource;
@@ -404,4 +405,29 @@ public interface DataFactory extends Cor
      * @return true if database encrypted false otherwise
      */
     public boolean databaseEncrypted();
+
+    /**
+        Register a handler class for insert undo events.
+        <P>
+        Register a class to be called when an undo of an insert 
+        is executed.  When an undo of an event is executed by
+        the raw store UndoHandler.insertUndoNotify() will be
+        called, allowing upper level callers to execute code
+        as necessary.  The initial need is for the access layer
+        to be able to queue post commit reclaim space in the
+        case of inserts which are aborted (including the normal
+        case of inserts failed for duplicate key violations)
+        (see DERBY-4057)
+        <p>
+        Currently the handler is only called on abort of inserts on
+        non-overflow pages that meet either of the following 2 
+        requirements:
+        1) the row has either overflow columns (long columns) or
+           the row columns span multiple pages (long rows).
+        2) after the action all user rows on the page are marked deleted.
+
+        @param undo_handler client code supplied undo_handle. 
+
+    */
+    public void setUndoInsertEventHandler(UndoHandler undo_handle);
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessManager.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessManager.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessManager.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessManager.java Mon Nov 24 16:37:38 2014
@@ -464,9 +464,7 @@ public abstract class RAMAccessManager
      *
 	 * @exception  StandardException  Standard exception policy.
      **/
-    /* package */ Conglomerate conglomCacheFind(
-    TransactionManager  xact_mgr,
-    long                conglomid)
+    /* package */ Conglomerate conglomCacheFind(long conglomid)
         throws StandardException
     {
         Conglomerate conglom       = null;
@@ -1041,6 +1039,11 @@ public abstract class RAMAccessManager
         rawstore = (RawStoreFactory) bootServiceModule(
             create, this, RawStoreFactory.MODULE, serviceProperties);
 
+        // initialize handler with raw store to be called in the event of
+        // aborted inserts.  Store will use the call back to reclaim space
+        // when these events happen.  See DERBY-4057.
+        rawstore.setUndoInsertEventHandler(new RAMAccessUndoHandler(this));
+
 		// Note: we also boot this module here since we may start Derby
 		// system from store access layer, as some of the unit test case,
 		// not from JDBC layer.(See

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessUndoHandler.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessUndoHandler.java?rev=1641418&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessUndoHandler.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessUndoHandler.java Mon Nov 24 16:37:38 2014
@@ -0,0 +1,141 @@
+/*
+
+   Derby - Class org.apache.derby.impl.store.access.RAMAccessUndoHandler
+
+   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.impl.store.access;
+
+import org.apache.derby.shared.common.sanity.SanityManager;
+
+import org.apache.derby.iapi.error.StandardException; 
+
+import org.apache.derby.iapi.store.access.AccessFactory;
+
+import org.apache.derby.iapi.store.access.conglomerate.ConglomerateFactory;
+import org.apache.derby.iapi.store.access.conglomerate.TransactionManager;
+
+import org.apache.derby.iapi.store.raw.PageKey;
+import org.apache.derby.iapi.store.raw.Transaction;
+import org.apache.derby.iapi.store.raw.UndoHandler;
+
+
+/**
+	A class that provides interface to be called when undo of an Insert
+    happens in raw store.
+*/
+
+
+public class RAMAccessUndoHandler implements UndoHandler
+{
+    /**************************************************************************
+     * Fields of the class
+     **************************************************************************
+     */
+    private RAMAccessManager access_factory;
+
+    /**************************************************************************
+     * Constructors for This class:
+     **************************************************************************
+     */
+    public RAMAccessUndoHandler(RAMAccessManager input_access_factory)
+    {
+        access_factory = input_access_factory;
+    }
+
+    /**************************************************************************
+     * Private/Protected methods of This class:
+     **************************************************************************
+     */
+
+    /**************************************************************************
+     * Public Methods of This class:
+     **************************************************************************
+     */
+
+    /**
+     * Interface to be called when an undo of an insert is processed.
+     * <p>
+     * Implementer of this class provides interface to be called by the raw
+     * store when an undo of an insert is processed.  Initial implementation
+     * will be by Access layer to queue space reclaiming events if necessary
+     * when a rows is logically "deleted" as part of undo of the original
+     * insert.  This undo can happen a lot for many applications if they
+     * generate expected and handled duplicate key errors.
+     * <p>
+     * It may be useful at some time to include the recordId of the deleted
+     * row, but it is not used currently by those notified.  The post commit
+     * work ultimately processes all rows on the table while
+     * it has the latch which is more efficient than one row at time per latch.
+     * <p>
+     * It is expected that notifies only happen for pages that caller
+     * is interested in.  Currently only the following aborted inserts
+     * cause a notify:
+     * o must be on a non overflow page
+     * o if all "user" rows on page are deleted a notify happens (page 1 
+     *   has a system row so on page one notifies happen if all but the first
+     *   row is deleted).
+     * o if the aborted insert row has either an overflow row or column
+     *   component then the notify is executed.
+     *
+     * @param xact      transaction that is being backed out.
+     * @param page_key  key that uniquely identifies page in question, container
+     *                  key information is embedded in the PageKey
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+    public void insertUndoNotify(
+    Transaction         xact,
+    PageKey             page_key)
+       throws StandardException
+    {
+        // from the container id, get the type of conglomerate (currently
+        // Heap or Btree), and then marshal the call to the appropriate
+        // conglomerate factory.
+        long conglom_id = page_key.getContainerId().getContainerId();
+
+        // casting to get package access to getFactoryFromConglomId, 
+        ConglomerateFactory conglom_factory = 
+            access_factory.getFactoryFromConglomId(conglom_id);
+
+        try
+        {
+            conglom_factory.insertUndoNotify(
+                    access_factory, 
+                    xact,
+                    page_key);
+
+        }
+        catch (StandardException exception)
+        {
+            // errors are likely related to abort of ddl associated
+            // with inserts.  Just ignore and don't post try and post
+            // a work event.
+
+
+            // ignore exception
+        }
+
+        return;
+    }
+
+    /**************************************************************************
+     * Public Methods of XXXX class:
+     **************************************************************************
+     */
+}

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=1641418&r1=1641417&r2=1641418&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 Mon Nov 24 16:37:38 2014
@@ -64,12 +64,12 @@ import org.apache.derby.iapi.store.acces
 
 import org.apache.derby.iapi.store.access.XATransactionController;
 
-
 import org.apache.derby.iapi.store.raw.ContainerHandle;
+import org.apache.derby.iapi.store.raw.ContainerKey;
 import org.apache.derby.iapi.store.raw.LockingPolicy;
-
-
 import org.apache.derby.iapi.store.raw.Loggable;
+import org.apache.derby.iapi.store.raw.Page;
+import org.apache.derby.iapi.store.raw.PageKey;
 import org.apache.derby.iapi.store.raw.Transaction;
 
 import org.apache.derby.iapi.types.DataValueDescriptor;
@@ -401,7 +401,7 @@ public class RAMTransaction 
 
 		if (conglomId >= 0)
         {
-            conglom = accessmanager.conglomCacheFind(this, conglomId);
+            conglom = accessmanager.conglomCacheFind(conglomId);
         }
         else
 		{
@@ -2097,6 +2097,27 @@ public class RAMTransaction 
      */
 
     /**
+     * Return existing Conglomerate after doing lookup by ContainerKey
+     * <p>
+     * Throws exception if it can't find a matching conglomerate for the 
+     * ContainerKey.
+     * 
+     * @return If successful returns 
+     *
+     * @param container_key  container key of target conglomerate.
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+	public Conglomerate findExistingConglomerateFromKey(
+    ContainerKey container_key)
+		throws StandardException
+    {
+        // in this implementation of the store conglomerate id's and 
+        // container id's are the same.
+        return(findExistingConglomerate(container_key.getContainerId()));
+	}
+
+    /**
      * Add to the list of post commit work.
      * <p>
      * Add to the list of post commit work that may be processed after this

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2IFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2IFactory.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2IFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2IFactory.java Mon Nov 24 16:37:38 2014
@@ -35,6 +35,7 @@ import org.apache.derby.shared.common.sa
 import org.apache.derby.catalog.UUID;
 import org.apache.derby.iapi.services.uuid.UUIDFactory;
 import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.store.access.AccessFactory;
 import org.apache.derby.iapi.store.access.conglomerate.Conglomerate;
 import org.apache.derby.iapi.store.access.conglomerate.ConglomerateFactory;
 import org.apache.derby.iapi.store.access.conglomerate.TransactionManager;
@@ -44,7 +45,9 @@ import org.apache.derby.iapi.store.acces
 import org.apache.derby.iapi.store.raw.ContainerKey;
 import org.apache.derby.iapi.store.raw.ContainerHandle;
 import org.apache.derby.iapi.store.raw.LockingPolicy;
+import org.apache.derby.iapi.store.raw.PageKey;
 import org.apache.derby.iapi.store.raw.RawStoreFactory;
+import org.apache.derby.iapi.store.raw.Transaction;
 
 import org.apache.derby.iapi.types.DataValueDescriptor;
 
@@ -307,6 +310,40 @@ public class B2IFactory implements Congl
         return(btree);
     }
 
+    /**
+     * Interface to be called when an undo of an insert is processed.
+     * <p>
+     * Implementer of this class provides interface to be called by the raw
+     * store when an undo of an insert is processed.  Initial implementation
+     * will be by Access layer to queue space reclaiming events if necessary
+     * when a rows is logically "deleted" as part of undo of the original
+     * insert.  This undo can happen a lot for many applications if they
+     * generate expected and handled duplicate key errors.
+     * <p>
+     * Caller may decide to call or not based on deleted row count of the
+     * page, or if overflow rows/columns are present.
+     *
+     *
+     * @param access_factory    current access_factory of the aborted insert.
+     * @param xact              transaction that is being backed out.
+     * @param page_key          page key of the aborted insert.
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+    public void insertUndoNotify(
+    AccessFactory       access_factory,
+    Transaction         xact,
+    PageKey             page_key)
+        throws StandardException
+    {
+        // Currently a no-op, btree's can reclaim space at split time. 
+        // TODO - see if it makes sense to add postAbort action if the 
+        // page has all deleted keys.  Shrinks don't work very well currently
+        // as they require table level locks, so may not help much until that
+        // issue is resolved.  see DERBY-5473
+
+    }
+
 	/*
 	** Methods of ModuleControl.
 	*/

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java Mon Nov 24 16:37:38 2014
@@ -420,6 +420,7 @@ public class Heap 
             {
                 if (SanityManager.DEBUG)
                     SanityManager.THROWASSERT(
+                        "Expected (column_id == format_ids.length)" +
                         "column_id = " + column_id +
                         "format_ids.length = " + format_ids.length +
                         "format_ids = " + format_ids);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapConglomerateFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapConglomerateFactory.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapConglomerateFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapConglomerateFactory.java Mon Nov 24 16:37:38 2014
@@ -32,8 +32,10 @@ import org.apache.derby.iapi.error.Stand
 
 import org.apache.derby.iapi.store.access.conglomerate.Conglomerate;
 import org.apache.derby.iapi.store.access.conglomerate.ConglomerateFactory;
+import org.apache.derby.impl.store.access.conglomerate.RowPosition;
 import org.apache.derby.iapi.store.access.conglomerate.TransactionManager;
 
+import org.apache.derby.iapi.store.access.AccessFactory;
 import org.apache.derby.iapi.store.access.ColumnOrdering;
 import org.apache.derby.iapi.store.access.TransactionController;
 
@@ -42,8 +44,10 @@ import org.apache.derby.iapi.store.raw.F
 import org.apache.derby.iapi.store.raw.ContainerKey;
 import org.apache.derby.iapi.store.raw.LockingPolicy;
 import org.apache.derby.iapi.store.raw.Page;
+import org.apache.derby.iapi.store.raw.PageKey;
 import org.apache.derby.iapi.store.raw.RawStoreFactory;
 import org.apache.derby.iapi.store.raw.RecordHandle;
+import org.apache.derby.iapi.store.raw.Transaction;
 
 import org.apache.derby.iapi.types.DataValueDescriptor;
 
@@ -292,6 +296,43 @@ public class HeapConglomerateFactory imp
         return((Conglomerate) control_row[0]);
     }
 
+    /**
+     * Interface to be called when an undo of an insert is processed.
+     * <p>
+     * Implementer of this class provides interface to be called by the raw
+     * store when an undo of an insert is processed.  Initial implementation
+     * will be by Access layer to queue space reclaiming events if necessary
+     * when a rows is logically "deleted" as part of undo of the original
+     * insert.  This undo can happen a lot for many applications if they
+     * generate expected and handled duplicate key errors.
+     * <p>
+     * Caller may decide to call or not based on deleted row count of the
+     * page, or if overflow rows/columns are present.
+     *
+     *
+     * @param access_factory    current access_factory of the aborted insert.
+     * @param xact              transaction that is being backed out.
+     * @param page_key          page key of the aborted insert.
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+    public void insertUndoNotify(
+    AccessFactory       access_factory,
+    Transaction         xact,
+    PageKey             page_key)
+        throws StandardException
+    {
+        // try to reclaim rows when the page is only full of deleted rows,
+        // or in the special case of the first page when all rows except the
+        // "control row" are deleted.  Or if the row we just deleted is
+        // a long row or has a long column.
+        //
+        // This logic is currently embedded in raw store InsertOperation
+        // abort code which triggers the event to notify the 
+        // HeapConglomerateFactory to post the HeapPostCommit work item.
+        xact.addPostAbortWork(new HeapPostCommit(access_factory, page_key));
+    }
+
 	/*
 	** Methods of ModuleControl.
 	*/

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapController.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapController.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapController.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapController.java Mon Nov 24 16:37:38 2014
@@ -101,8 +101,7 @@ public class HeapController 
         xact_mgr.addPostCommitWork(
             new HeapPostCommit(
                 xact_mgr.getAccessManager(), 
-                (Heap) open_conglom.getConglomerate(),
-                pos.current_page.getPageNumber()));
+                pos.current_page.getPageKey()));
     }
 
     /**************************************************************************

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapPostCommit.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapPostCommit.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapPostCommit.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapPostCommit.java Mon Nov 24 16:37:38 2014
@@ -26,6 +26,7 @@ import org.apache.derby.iapi.services.da
 import org.apache.derby.shared.common.sanity.SanityManager;
 import org.apache.derby.iapi.error.StandardException;
 
+import org.apache.derby.iapi.store.access.conglomerate.Conglomerate;
 import org.apache.derby.iapi.store.access.conglomerate.TransactionManager;
 
 import org.apache.derby.iapi.store.access.AccessFactory;
@@ -36,8 +37,10 @@ import org.apache.derby.iapi.store.acces
 import org.apache.derby.iapi.store.access.TransactionController;
 
 import org.apache.derby.iapi.store.raw.ContainerHandle;
+import org.apache.derby.iapi.store.raw.ContainerKey;
 import org.apache.derby.iapi.store.raw.LockingPolicy;
 import org.apache.derby.iapi.store.raw.Page;
+import org.apache.derby.iapi.store.raw.PageKey;
 import org.apache.derby.iapi.store.raw.RecordHandle;
 import org.apache.derby.iapi.store.raw.Transaction;
 
@@ -74,9 +77,7 @@ class HeapPostCommit implements Servicea
      */
 
     private AccessFactory access_factory  = null;
-    private Heap          heap            = null;
-    private long          page_number     = ContainerHandle.INVALID_PAGE_NUMBER;
-
+    private PageKey       page_key        = null;
 
     /**************************************************************************
      * Constructors for This class:
@@ -84,12 +85,10 @@ class HeapPostCommit implements Servicea
      */
     HeapPostCommit(
     AccessFactory   access_factory,
-    Heap            heap,
-    long            input_page_number)
+    PageKey         page_key)
     {
         this.access_factory = access_factory; 
-        this.heap           = heap; 
-        this.page_number    = input_page_number; 
+        this.page_key       = page_key; 
     }
 
     /**************************************************************************
@@ -98,13 +97,13 @@ class HeapPostCommit implements Servicea
      */
 
     /**
-     * Reclaim space taken up by committed deleted rows.
+     * Reclaim space taken of committed deleted rows or aborted inserted rows.
      * <p>
      * This routine assumes it has been called by an internal transaction which
      * has performed no work so far, and that it has an exclusive intent table 
-     * lock.  It will attempt obtain exclusive row locks on deleted rows, where
-     * successful those rows can be reclaimed as they must be "committed 
-     * deleted" rows.
+     * lock.  It will attempt obtain exclusive row locks on rows marked 
+     * deleted, where successful those rows can be reclaimed as they must be 
+     * "committed deleted" or "aborted inserted" rows.
      * <p>
      * This routine will latch the page and hold the latch due to interface
      * requirement from Page.purgeAtSlot.
@@ -127,6 +126,7 @@ class HeapPostCommit implements Servicea
 
         // wait to get the latch on the page 
         Page page = heap_control.getUserPageWait(pageno);
+
         boolean purgingDone = false;
 
         if (page != null)
@@ -213,7 +213,8 @@ class HeapPostCommit implements Servicea
                         {
                             SanityManager.DEBUG_PRINT(
                                 "HeapPostCommit", 
-                                "Calling Heap removePage().; pagenumber="+pageno+"\n");
+                                "Calling Heap removePage().; pagenumber=" +
+                                pageno + "\n");
                         }
                     }
                 }
@@ -328,6 +329,23 @@ class HeapPostCommit implements Servicea
 			//If we can not get the lock this reclamation request will 
 			//requeued.
 
+            // if does not exist will throw exception, which the code will 
+            // handle in the same way as it does heap.open failing if trying 
+            // to open a dropped container.
+
+            Conglomerate conglom = 
+                internal_xact.findExistingConglomerateFromKey(
+                    page_key.getContainerId());
+
+            if (SanityManager.DEBUG)
+            {
+                // This code can only handle Heap conglomerates.
+                SanityManager.ASSERT(conglom instanceof Heap,
+                        "Code expecting PageKey/ContainerKey of a Heap");
+            }
+
+            Heap heap = (Heap) conglom;
+
             heapcontroller = (HeapController)
                 heap.open(
                     internal_xact,
@@ -346,19 +364,20 @@ class HeapPostCommit implements Servicea
             // be reclaimed, once an "X" row lock is obtained on them.
 
             // Process all the rows on the page while holding the latch.
-            purgeCommittedDeletes(heapcontroller, this.page_number);
+            purgeCommittedDeletes(
+                heapcontroller, this.page_key.getPageNumber());
 
         }
         catch (StandardException se)
         {
             // exception might have occured either because the container got 
-            // dropper or the lock was not granted.
+            // dropped or the lock was not granted.
             // It is possible by the time this post commit work gets scheduled 
             // that the container has been dropped and that the open container 
             // call will return null - in this case just return assuming no 
             // work to be done.
 
-			// If this expcetion is because lock could not be obtained, 
+			// If this exception is because lock could not be obtained, 
             // work is requeued.
 			if (se.isLockTimeoutOrDeadlock())
 			{
@@ -388,11 +407,11 @@ class HeapPostCommit implements Servicea
                 if (requeue_work)
                     SanityManager.DEBUG_PRINT(
                         "HeapPostCommit", 
-                        "requeueing on page num = " + page_number);
+                        "requeueing on page num = " + 
+                            this.page_key.getPageNumber());
             }
         }
 
         return(requeue_work ? Serviceable.REQUEUE : Serviceable.DONE);
     }
 }
-

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=1641418&r1=1641417&r2=1641418&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 Mon Nov 24 16:37:38 2014
@@ -100,8 +100,7 @@ class HeapScan 
         xact_mgr.addPostCommitWork(
             new HeapPostCommit(
                 xact_mgr.getAccessManager(), 
-                (Heap) open_conglom.getConglomerate(),
-                pos.current_page.getPageNumber()));
+                pos.current_page.getPageKey()));
     }
 
     /**************************************************************************

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java Mon Nov 24 16:37:38 2014
@@ -48,6 +48,7 @@ import org.apache.derby.iapi.store.acces
 import org.apache.derby.iapi.store.raw.ScanHandle;
 import org.apache.derby.iapi.store.raw.RawStoreFactory;
 import org.apache.derby.iapi.store.raw.Transaction;
+import org.apache.derby.iapi.store.raw.UndoHandler;
 import org.apache.derby.iapi.store.raw.xact.RawTransaction;
 import org.apache.derby.iapi.store.raw.xact.TransactionFactory;
 import org.apache.derby.iapi.store.raw.data.DataFactory;
@@ -408,6 +409,29 @@ public final class RawStore implements R
         return xactFactory;  
     }
 
+    /**
+     * Register a handler class for insert undo events.
+     * <p>
+     * Register a class to be called when an undo of an insert is executed.  
+     * When an undo of an event is executed by the raw store 
+     * UndoHandler.insertUndoNotify() will be called, allowing upper level 
+     * callers to execute code as necessary.  The initial need is for the 
+     * access layer to be able to queue post commit reclaim space in the case 
+     * of inserts which are aborted (including the normal case of inserts 
+     * failed for duplicate key violations) (see DERBY-4057)
+     * <p>
+     *
+     * @param undo_handle Class to use to notify callers of an undo of an 
+     *                    insert.
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+    public void setUndoInsertEventHandler(
+        UndoHandler input_undo_handle) throws StandardException
+    {
+        dataFactory.setUndoInsertEventHandler(input_undo_handle);
+    }
+
 	/*
 	 * Return the module providing XAresource interface to the transaction
      * table.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java Mon Nov 24 16:37:38 2014
@@ -56,9 +56,11 @@ import org.apache.derby.iapi.store.raw.C
 import org.apache.derby.iapi.store.raw.ContainerKey;
 import org.apache.derby.iapi.store.raw.LockingPolicy;
 import org.apache.derby.iapi.store.raw.Page;
+import org.apache.derby.iapi.store.raw.PageKey;
 import org.apache.derby.iapi.store.raw.RawStoreFactory;
 import org.apache.derby.iapi.store.raw.StreamContainerHandle;
 import org.apache.derby.iapi.store.raw.Transaction;
+import org.apache.derby.iapi.store.raw.UndoHandler;
 import org.apache.derby.iapi.store.raw.xact.RawTransaction;
 
 import org.apache.derby.iapi.store.access.RowSource;
@@ -73,6 +75,7 @@ import org.apache.derby.iapi.reference.A
 import org.apache.derby.iapi.reference.Property;
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.util.ByteArray;
+
 import org.apache.derby.iapi.util.InterruptStatus;
 import org.apache.derby.iapi.services.io.FileUtil;
 import org.apache.derby.iapi.util.ReuseFactory;
@@ -226,6 +229,10 @@ public class BaseDataFileFactory
     private File            backupRoot;
     private String[]        bfilelist;
 
+
+    // Class to use to notify upon undo of deletes
+    private UndoHandler undo_handler = null;
+
 	/*
 	** Constructor
 	*/
@@ -1367,6 +1374,57 @@ public class BaseDataFileFactory
 	** Implementation specific methods
 	*/
 
+    /**
+     * Register a handler class for insert undo events.
+     * <p>
+     * Register a class to be called when an undo of an insert is executed.  
+     * When an undo of an event is executed by the raw store 
+     * UndoHandler.insertUndoNotify() will be called, allowing upper level 
+     * callers to execute code as necessary.  The initial need is for the 
+     * access layer to be able to queue post commit reclaim space in the case 
+     * of inserts which are aborted (including the normal case of inserts 
+     * failed for duplicate key violations) (see DERBY-4057)
+     * <p>
+     *
+     * @param undo_handle Class to use to notify callers of an undo of an 
+     *                    insert.
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+    public void setUndoInsertEventHandler(
+        UndoHandler input_undo_handle)
+    {
+        undo_handler = input_undo_handle;
+    }
+    
+    /**
+     * Notify through set handler that an undo of an insert has happened.
+     * <p>
+     * When an undo of an event is executed by the raw store 
+     * UndoHandler.insertUndoNotify() will be called, allowing upper level 
+     * callers to execute code as necessary.  The initial need is for the 
+     * access layer to be able to queue post commit reclaim space in the case 
+     * of inserts which are aborted (including the normal case of inserts 
+     * failed for duplicate key violations) (see DERBY-4057)
+     * Longer descrption of routine.
+     * <p>
+     *
+     * @param rxact     raw transaction of the aborted insert.
+     * @param page_key  page key of the aborted insert.
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+    protected void insertUndoNotify(
+    RawTransaction  rxact,
+    PageKey         page_key)
+        throws StandardException
+    {
+        if (undo_handler != null)
+        {
+            undo_handler.insertUndoNotify(rxact, page_key);
+        }
+    }
+
 	public int getIntParameter(
     String      parameterName, 
     Properties  properties, 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BasePage.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BasePage.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BasePage.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BasePage.java Mon Nov 24 16:37:38 2014
@@ -353,6 +353,16 @@ abstract class BasePage implements Page,
 		return identity.getPageNumber();
 	}
 
+	/** @see Page#getPageIdentity */
+	public final PageKey getPageKey() {
+		if (SanityManager.DEBUG) {
+			SanityManager.ASSERT(isLatched(), "page is not latched.");
+			SanityManager.ASSERT(identity != null, "identity is null.");
+		}
+
+		return identity;
+	}
+
 	public final RecordHandle getRecordHandle(int recordId) {
 		if (SanityManager.DEBUG) {
 			SanityManager.ASSERT(isLatched());
@@ -1462,6 +1472,8 @@ abstract class BasePage implements Page,
      * to ask whether a post commit should be queued to try to reclaim space
      * after the delete commits.  
      * <p>
+     * Also used by access methods after undo of an insert.
+     * <p>
      * Will return true if the number of non-deleted rows on the page is
      * <= "num_non_deleted_rows".  For instance 0 means schedule reclaim
      * only if all rows are deleted, 1 if all rows but one are deleted.  
@@ -1496,16 +1508,28 @@ abstract class BasePage implements Page,
 
         boolean ret_val = false;
 
-        if (internalNonDeletedRecordCount() <= num_non_deleted_rows)
+        if (!isOverflowPage())
         {
-            ret_val = true;
-        }
-        else 
-        {
-            if (!entireRecordOnPage(slot_just_deleted))
+            // only return true for non-overflow pages.  Overflow pages
+            // will be reclaimed as part of reclaiming the rows on the
+            // head pages.
+
+            if (internalNonDeletedRecordCount() <= num_non_deleted_rows)
             {
+                // all user rows on this page are marked deleted, so 
+                // reclaim this page.
                 ret_val = true;
             }
+            else 
+            {
+                // always reclaim if there rows span more than one page,
+                // this picks up long rows and blob/clob columns.  This
+                // tells us likely to reclaim 1 or more pages.
+                if (!entireRecordOnPage(slot_just_deleted))
+                {
+                    ret_val = true;
+                }
+            }
         }
 
         return(ret_val);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InsertOperation.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InsertOperation.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InsertOperation.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InsertOperation.java Mon Nov 24 16:37:38 2014
@@ -207,11 +207,12 @@ public final class InsertOperation exten
 					undoPage.getPageNumber());
 		}
 
+        RawTransaction rxact = (RawTransaction)xact;
+
 		if ((insertFlag & Page.INSERT_UNDO_WITH_PURGE) != 0)
 		{
 			undoPage.purgeRecord(CLRInstant, slot, undoRecordId);
 
-			RawTransaction rxact = (RawTransaction)xact;
 
 			// If we purged the last row off an overflow page, reclaim that
 			// page - we have to do this post transaction termination because we
@@ -221,14 +222,31 @@ public final class InsertOperation exten
 				undoPage.isOverflowPage() && undoPage.recordCount() == 0)
 			{
 				ReclaimSpace work = 
-					new ReclaimSpace(ReclaimSpace.PAGE, (PageKey)undoPage.getIdentity(),
-									 rxact.getDataFactory(), true /* service ASAP */);
+					new ReclaimSpace(
+                            ReclaimSpace.PAGE, 
+                            (PageKey)undoPage.getIdentity(),
+                            rxact.getDataFactory(), 
+                            true /* service ASAP */);
+
 				rxact.addPostTerminationWork(work);
 			}
 		}
 		else
 		{
 			undoPage.setDeleteStatus(CLRInstant, slot, true);
+
+
+			if (rxact.handlesPostTerminationWork() && 
+                !undoPage.isOverflowPage())
+            {
+                if (undoPage.shouldReclaimSpace(
+                        undoPage.getPageNumber() == 1 ? 1 : 0, slot))
+                {
+                    ((BaseDataFileFactory) rxact.getDataFactory()).insertUndoNotify(
+                        rxact,
+                        undoPage.getPageKey());
+                }
+            }
 		}
 
 		undoPage.setAuxObject(null);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/StoredPage.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/StoredPage.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/StoredPage.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/StoredPage.java Mon Nov 24 16:37:38 2014
@@ -2599,8 +2599,14 @@ public class StoredPage extends CachedPa
                         ", total slotsInUse = " + slotsInUse);
                 }
 
-                SanityManager.ASSERT(recordHeader.getFirstField() == 0,
-                     "Head row piece should start at field 0 but is not");
+                if (recordHeader.getFirstField() != 0)
+                {
+                    SanityManager.THROWASSERT(
+                        "Head row piece should start at field 0 but is not," + 
+                        ", current slot = " + slot + 
+                        ", total slotsInUse = " + slotsInUse + 
+                        "page = " + this);
+                }
             }
 
             int numberFields = recordHeader.getNumberFields();

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java Mon Nov 24 16:37:38 2014
@@ -221,10 +221,12 @@ public class Xact extends RawTransaction
 
 	private Stack<SavePoint>			savePoints;	// stack of SavePoint objects.
 
-	protected List<Serviceable>   		postCommitWorks; // a list of post commit work
-	protected List<Serviceable>		    postTerminationWorks; // work to be done after
-												  // transaction terminates,
-												  // commit or abort
+	protected List<Serviceable> postCommitWorks; // a list of post commit work
+	protected List<Serviceable> postAbortWorks;  // a list of post abort work
+	protected List<Serviceable> postTerminationWorks; // work to be done after
+												      // transaction terminates,
+												      // commit or abort
+                                                      
 	private boolean			recoveryTransaction;  // this transaction is being
 												  // used by recovery
 
@@ -1023,6 +1025,7 @@ public class Xact extends RawTransaction
 		{
 			postCommitWorks.clear();
 		}
+       
 
 		// Now do post termination work - must do this after the rollback is
 		// complete because the rollback itself may generate postTermination
@@ -1274,11 +1277,28 @@ public class Xact extends RawTransaction
 		if (recoveryTransaction)
 			return;
 
+
 		if (postCommitWorks == null)
 			postCommitWorks = new ArrayList<Serviceable>(1);
 		postCommitWorks.add(work);
 	}
 
+	/**
+		Add to the list of post abort work that may be processed after this
+		transaction aborts.  
+
+		@param work the post commit work that is added
+	*/
+	public void addPostAbortWork(Serviceable work)
+	{
+		if (recoveryTransaction)
+			return;
+
+		if (postAbortWorks == null)
+			postAbortWorks = new ArrayList<Serviceable>(1);
+		postAbortWorks.add(work);
+	}
+
 	public void addPostTerminationWork(Serviceable work)
 	{
 		if (recoveryTransaction)
@@ -2045,26 +2065,16 @@ public class Xact extends RawTransaction
 
 	}
 
-
-	private final void postTermination() throws StandardException
-	{
-		// move all the postTermination work to the postCommit queue
-		int count = (postTerminationWorks == null) ? 
-			0 : postTerminationWorks.size(); 
-
-		for (int i = 0; i < count; i++)
-			addPostCommitWork(postTerminationWorks.get(i));
-
-		if (count > 0)
-			postTerminationWorks.clear();
-
+    private void transferPostCommitorAbortWork(List <Serviceable> work_list)
+        throws StandardException
+    {
 
 		// if there are post commit work to be done, transfer them to the
 		// daemon.  The log is flushed, all locks released and the
 		// transaction has ended at this point.
-		if (postCommitWorks != null && !postCommitWorks.isEmpty())
+		if (work_list != null && !work_list.isEmpty())
 		{
-			int pcsize = postCommitWorks.size();
+			int pcsize = work_list.size();
 			
 			// do we want to do post commit work with this transaction object?
 			if (doPostCommitWorkInTran())
@@ -2073,36 +2083,43 @@ public class Xact extends RawTransaction
 				{
 					inPostCommitProcessing = true;
 
-					// to avoid confusion, copy the post commit work to an array if this
-					// is going to do some work now
+					// to avoid confusion, copy the post commit or abort work 
+                    // to an array if this is going to do some work now
 					Serviceable[] work = new Serviceable[pcsize];
-					work = (Serviceable[])postCommitWorks.toArray(work);
+					work = (Serviceable[])work_list.toArray(work);
 
-					// clear this for post commit processing to queue its own post
-					// commit works - when it commits, it will send all its post
-					// commit request to the daemon instead of dealing with it here.
-					postCommitWorks.clear();
-
-					//All the post commit work that is part  of the database creation
-					//should be done on this thread immediately.
-					boolean doWorkInThisThread = xactFactory.inDatabaseCreation();
+					// clear this for post commit or abort processing to queue 
+                    // its own post commit works - when it commits, it will 
+                    // send all its post commit request to the daemon instead 
+                    // of dealing with it here.
+					work_list.clear();
+
+					//All the post commit or abort work that is part of the 
+                    //database creation should be done on this thread 
+                    //immediately.
+					boolean doWorkInThisThread = 
+                        xactFactory.inDatabaseCreation();
 
 					for (int i = 0; i < pcsize; i++)
 					{
 
 						//process work that should be done immediately or
 						//when we  are in still in database creattion.
-						//All the other work should be submitted 
-						//to the post commit thread to be processed asynchronously
+						//All the other work should be submitted to the post 
+                        //commit thread to be processed asynchronously
+
 						if (doWorkInThisThread || work[i].serviceImmediately())
 						{
 							try
 							{
 								// this may cause other post commit work to be
-								// added.  when that transaction commits, those
+								// added.  when that transaction commits, that
 								// work will be transfered to the daemon
-								if (work[i].performWork(xc.getContextManager()) == Serviceable.DONE)
+								if (work[i].performWork(xc.getContextManager())
+                                        == Serviceable.DONE)
+                                {
 									work[i] = null;
+                                }
 
 								// if REQUEUE, leave it on for the postcommit
 								// daemon to handle
@@ -2112,7 +2129,9 @@ public class Xact extends RawTransaction
 								// don't try to service this again
 								work[i] = null;
 
-								// try to handle it here.  If we fail, then let the error percolate.
+								// try to handle it here.  
+                                // If we fail, then let the error percolate.
+
 								xc.cleanupOnError(se);
 							}
 						}
@@ -2121,11 +2140,15 @@ public class Xact extends RawTransaction
 						// requeueing, send it off.   Note that this is one case
 						// where a REQUEUE ends up in the high priority queue.
 						// Unfortunately, there is no easy way to tell.  If the
-						// Servicable is well mannered, it can change itself from
-						// serviceASAP to not serviceASAP if it returns REQUEUE.
+						// Servicable is well mannered, it can change itself 
+                        // from serviceASAP to not serviceASAP if it returns 
+                        // REQUEUE.
+
 						if (work[i] != null)
 						{
-							boolean needHelp = xactFactory.submitPostCommitWork(work[i]);
+							boolean needHelp = 
+                                xactFactory.submitPostCommitWork(work[i]);
+
 							work[i] = null;
 							if (needHelp)
 								doWorkInThisThread = true;
@@ -2137,8 +2160,8 @@ public class Xact extends RawTransaction
 					inPostCommitProcessing = false;
 
 					// if something untoward happends, clear the queue.
-					if (postCommitWorks != null)
-						postCommitWorks.clear();
+					if (work_list != null)
+						work_list.clear();
 				}
 
 			}
@@ -2149,15 +2172,37 @@ public class Xact extends RawTransaction
 				// work submitting other post commit work)
 				for (int i = 0; i < pcsize; i++)
 				{
-					// SanityManager.DEBUG_PRINT("PostTermination",postCommitWorks.elementAt((i)).toString());
-					xactFactory.submitPostCommitWork((Serviceable)postCommitWorks.get((i)));
+					// SanityManager.DEBUG_PRINT("PostTermination",work_list.elementAt((i)).toString());
+					xactFactory.submitPostCommitWork(
+                        (Serviceable)work_list.get((i)));
 				}
 			}
 
-			postCommitWorks.clear();
+			work_list.clear();
 
 		}
 
+    }
+
+
+	private final void postTermination() throws StandardException
+	{
+		// move all the postTermination work to the postCommit queue
+		int count = (postTerminationWorks == null) ? 
+			0 : postTerminationWorks.size(); 
+
+		for (int i = 0; i < count; i++)
+			addPostCommitWork(postTerminationWorks.get(i));
+
+		if (count > 0)
+			postTerminationWorks.clear();
+
+        // transfer post commit work
+        transferPostCommitorAbortWork(postCommitWorks);
+
+        // transfer post abort work
+        transferPostCommitorAbortWork(postAbortWorks);
+
         // any backup blocking operations (like unlogged ops) in this 
         // transaction are done with post commit/abort work that needs be
         // done in the same trasaction,  unblock the backup.

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/ClobReclamationTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/ClobReclamationTest.java?rev=1641418&r1=1641417&r2=1641418&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/ClobReclamationTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/ClobReclamationTest.java Mon Nov 24 16:37:38 2014
@@ -145,6 +145,37 @@ public class ClobReclamationTest extends
         JDBC.assertFullResultSet(rs, new String[][] { { "" + expectedFree } });
     }
 
+    private void checkNumFreePagesMax(
+    String  table, 
+    int     expectedFreeMax) throws SQLException {        
+
+        // Check the space table 
+        // Should not have grown.
+
+        PreparedStatement ps = 
+            prepareStatement(
+                  "SELECT NUMFREEPAGES FROM "
+                + " new org.apache.derby.diag.SpaceTable('APP',?) t"
+                + " WHERE CONGLOMERATENAME = ?");
+
+        ps.setString(1, table);
+        ps.setString(2, table);
+        ResultSet rs = ps.executeQuery();
+
+        // get 1 row, it has the num free page count and make sure it is
+        // lower than the passed in maximum.
+        rs.next();
+
+        int numfreerows = rs.getInt(1);
+
+        assertTrue(
+            "Fail -- numfreerows:" + numfreerows + 
+            " > expectedFreeMax: " + expectedFreeMax, 
+            numfreerows < expectedFreeMax); 
+        rs.close();
+        ps.close();
+    }
+
     private static void fiveHundredUpdates(Connection conn,
             String updateString, int key, boolean lockTable) throws SQLException {
         PreparedStatement ps = conn
@@ -199,7 +230,7 @@ public class ClobReclamationTest extends
      * 
      * @throws SQLException
      */
-    public void xtestReclamationOnRollback() throws SQLException {
+    public void testReclamationOnRollback() throws SQLException {
         setAutoCommit(false);
         String insertString = Formatters.repeatChar("a", 33000);
         PreparedStatement ps = prepareStatement("INSERT INTO CLOBTAB2 VALUES(?,?)");
@@ -243,8 +274,14 @@ public class ClobReclamationTest extends
         ps.close();
 
         // until DERBY-4057 fixed expect space to be 2 pages per row plus
-        // 1 head page per container.
-        checkNumAllocatedPages("CLOBTAB3", (numrows * 2) + 1);
+        // 1 head page per container.  DERBY-4057 has been fixed so allocated
+        // pages is showing up as 1 in my runs.  Am a bit worried that because
+        // this is thread and post commit dependent that we may have to play
+        // with the "expected" allocated a little to allow for post commit
+        // on some machines not running fast enough.  For now just setting
+        // to 1 as the head page will remain allocated, all the rest should
+        // be free after post commit reclaim.
+        checkNumAllocatedPages("CLOBTAB3", 1);
 
         // expect most free pages to get used by subsequent inserts.  Only 
         // free pages should be from the last remaining aborted insert of
@@ -252,10 +289,23 @@ public class ClobReclamationTest extends
         // head page of the chain: (sizeof(clob) / page size) - 1
         // Derby should default to 32k page size for any table with a clob in
         // it.
- 
-        // (clob length / page size ) + 
+
+        // after fix for DERBY-4057 work gets queued immediately after the 
+        // rollback, but also the next insert happens at same time.  On
+        // my machine I am getting 12 pages free which looks like space from
+        // the first abort  gets used for the 3rd insert, space from second
+        // abort gets used for the 4th insert ...  Running full test suite
+        // on my machine got 14 free pages which is still very good considering
+        // before change there were 1000 allocated pages and now there is 1
+        // allocated and 14 free.  There is a timing 
+        // issue with test and maybe a sleep of some sort is necessary after
+        // the rollback.  
+
+        // declaring correct run if only max if free pages from 5 pages worth
+        // of free space remains.
+        // ((clob length / page size ) * 5) + 
         //     1 page for int divide round off - 1 for the head page.
-        checkNumFreePages("CLOBTAB3", (clob_length / 32000) + 1 - 1);
+        checkNumFreePagesMax("CLOBTAB3", ((clob_length / 32000) * 5) + 1 - 1);
         commit();
 
         // running inplace compress should reclaim all the remaining aborted
@@ -288,6 +338,7 @@ public class ClobReclamationTest extends
             protected void decorateSQL(Statement s) throws SQLException {
                 if (SanityManager.DEBUG) {
                     SanityManager.DEBUG_SET("DaemonTrace");
+                    SanityManager.DEBUG_SET("verbose_heap_post_commit");
                 }
 
                 Connection conn = s.getConnection();