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 da...@apache.org on 2014/04/29 02:23:53 UTC

svn commit: r1590849 [1/3] - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/sql/conn/ engine/org/apache/derby/iapi/sql/dictionary/ engine/org/apache/derby/iapi/types/ engine/org/apache/derby/impl/sql/compile/ engine/org/apache/derby/impl/sq...

Author: dag
Date: Tue Apr 29 00:23:52 2014
New Revision: 1590849

URL: http://svn.apache.org/r1590849
Log:
DERBY-532 Support deferrable constraints

Deferred foreign key constraint. Patch derby-532-fk-7.

The approach taken for deferring foreign keys is similar to that taken
for the other constraints: when we detect a violation inserting or
updating the referring table, and when detecting a violation when
deleting or updating the referenced table (only when we have ON DELETE
(or UPDATE) NO ACTION), we save the key in a temporary table instead
of throwing an exception. At check time, typically on commit, we
revisit first the supporting index of referencing table to see if
there might still be a problem. If that key is (still) present, we
must also check the corresponding index in the referenced table. If
that is found, all is good. Otherwise we throw.

Patch details:

M       java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java
M       java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java

Move logic related to constraints checking to
DeferredConstraintsMemory. 

M       java/engine/org/apache/derby/impl/sql/execute/DeferredConstraintsMemory.java

New logic from LanguageConnectionContext; added logic for the case of
foreign key constraints violation memory and checking. Make old "if"s
on object type object oriented: new interface methods in
ValidationInfo: possiblyValidateOnReturn and validateConstraint which
are implemented differently for each constraint type.

M       java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java
M       java/engine/org/apache/derby/impl/sql/execute/UpdateResultSet.java
M       java/engine/org/apache/derby/impl/sql/execute/ConstraintConstantAction.java

More fk info to bulk insert; smaller refactorings.

M       java/engine/org/apache/derby/iapi/sql/dictionary/ReferencedKeyConstraintDescriptor.java

Change the method hasNonSelfReferencingFK to getNonSelfReferencingFK;
we need to get at them, see AlterTableConstantAction for truncate.

M       java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java

Refactored logic to DeferredConstraintsMemory#compressOrTruncate.
Allow truncate (if deferred and NO_ACTION constraints only) of
deferenced table.

M       java/engine/org/apache/derby/iapi/types/SQLBoolean.java
M       java/engine/org/apache/derby/impl/sql/execute/IndexChanger.java
M       java/engine/org/apache/derby/impl/sql/execute/ProjectRestrictResultSet.java

Minor refactorings.

M       java/engine/org/apache/derby/impl/sql/execute/FKInfo.java

Extended with information about deferred constraints; conglomerate ids and constrain ids.

M       java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java

More information collection to support extended FKInfo, see above.

M       java/engine/org/apache/derby/impl/sql/compile/TableElementList.java

Extra predicates to avoid logic for unique and primary key constraints to inferere with
deferred foreign keys.

M       java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java

Remove foreign keys as "not supported", minor refactorings.

M       java/engine/org/apache/derby/impl/sql/execute/DeleteResultSet.java

Added arguments, refactorings.

M       java/engine/org/apache/derby/impl/sql/execute/ForeignKeyRIChecker.java

This is where we actually defer when we see a fk violation at insert/update of referencing table.

M       java/engine/org/apache/derby/impl/sql/execute/GenericRIChecker.java

Minor interface changes: new member variable: lcc

M       java/engine/org/apache/derby/impl/sql/execute/RIBulkChecker.java

This is where we actually defer when we see a fk violation at bulk insert
into referencing table.

M       java/engine/org/apache/derby/impl/sql/execute/RISetChecker.java

Minor interface changes; added parameters.

M       java/engine/org/apache/derby/impl/sql/execute/ReferencedKeyRIChecker.java

This is where we actually defer when we see a fk violation at delete or update of
a row in the referenced table.

M       java/engine/org/apache/derby/impl/sql/execute/SetConstraintsConstantAction.java

Make SET CONSTRAINTS work also for foreign keys.

M       java/shared/org/apache/derby/shared/common/reference/SQLState.java
M       java/engine/org/apache/derby/loc/messages.xml

New error messages.

M       java/testing/org/apache/derbyTesting/functionTests/tests/lang/ConstraintCharacteristicsTest.java

Extended existing deferrable constraints to also work for foreign constraints.

A       java/testing/org/apache/derbyTesting/functionTests/tests/lang/ForeignKeysDeferrableTest.java

New tests only for foreign key constraints.

M       java/testing/org/apache/derbyTesting/functionTests/tests/upgradeTests/Changes10_11.java

Remove asserts for not implemented on deferrable foreign key
constraints, check basic sanity iff hard upgraded.

M       java/testing/org/apache/derbyTesting/junit/BaseJDBCTestCase.java

Added assertCommitError; minor changes to support fk testing.



Added:
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/ForeignKeysDeferrableTest.java
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ReferencedKeyConstraintDescriptor.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/SQLBoolean.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableElementList.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ConstraintConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeferredConstraintsMemory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeleteResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/FKInfo.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ForeignKeyRIChecker.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericRIChecker.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/IndexChanger.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ProjectRestrictResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RIBulkChecker.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RISetChecker.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ReferencedKeyRIChecker.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/SetConstraintsConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml
    db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/ConstraintCharacteristicsTest.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/upgradeTests/Changes10_11.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/junit/BaseJDBCTestCase.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java Tue Apr 29 00:23:52 2014
@@ -45,7 +45,6 @@ import org.apache.derby.iapi.sql.execute
 import org.apache.derby.iapi.sql.execute.CursorActivation;
 import org.apache.derby.iapi.sql.execute.ExecutionStmtValidator;
 import org.apache.derby.iapi.sql.execute.RunTimeStatistics;
-import org.apache.derby.iapi.store.access.BackingStoreHashtable;
 import org.apache.derby.iapi.store.access.TransactionController;
 import org.apache.derby.iapi.types.DataValueFactory;
 import org.apache.derby.impl.sql.execute.DeferredConstraintsMemory;
@@ -1309,40 +1308,43 @@ public interface LanguageConnectionConte
                                  UUID constraintId,
                                  boolean deferred) throws StandardException;
 
+
     /**
-     * Get the constraint mode set, if any.
+     * Determines if a unique or primary key constraint currently has deferred
+     * mode.
      *
-     * @param a         Activation
-     * @param conglomId The conglomerate id of the backing index
-     * @return         {@code true} if the constraint mode
-     *                  for this constraint/index is effectively
-     *                  deferred, {@code false} if it is immediate.
-     * @throws StandardException standard error policy
+     * @param sc       The session context for which we are asking the status
+     * @param indexCID The conglomerate id of the supporting index of the
+     *                 constraint.
+     * @return         {@code true} if the constraint is deferred
+     * @throws StandardException
+     *                 Standard error policy
      */
-    public boolean isEffectivelyDeferred(Activation a, long conglomId)
+    public boolean isEffectivelyDeferred(SQLSessionContext sc, long indexCID)
             throws StandardException;
 
     /**
-     * Get the constraint mode set, if any.
+     * Determines if a check or foreign key constraint has deferred
+     * mode.
      *
-     * @param a         Activation
-     * @param constraintId The constraint id
-     * @return         {@code true} if the constraint mode
-     *                  for this constraint/index is effectively
-     *                  deferred, {@code false} if it is immediate.
-     * @throws StandardException standard error policy
+     * @param sc           The SQL session context for which we are asking the status
+     * @param constraintId The constraint id we are inquiring about.
+     * @return             {@code true} if the constraint is deferred
+     * @throws StandardException
+     *                     Standard error policy
      */
-    public boolean isEffectivelyDeferred(Activation a, UUID constraintId)
+    public boolean isEffectivelyDeferred(SQLSessionContext sc, UUID constraintId)
             throws StandardException;
 
     /**
-     * Set the constraint mode for all deferrable constraints to
-     * {@code deferred}.
-     * If {@code deferred} is {@code false}, to immediate checking,
-     * if {@code true} to deferred checking.
+     * Set the constraint mode of all deferrable constraints to the value of
+     * {@code deferred}. If the value is {@code false}, this method might
+     * throw with a constraint violation error, i.e. if some constraint
+     * has deferred mode before this call and had seen violations.
      *
-     * @param a        Activation
-     * @param deferred The new constraint mode
+     * @param a         The activation
+     * @param deferred  The value which holds the constraint mode
+     * @throws          StandardException Standard error policy
      */
     public void setDeferredAll(Activation a, boolean deferred)
             throws StandardException;
@@ -1376,4 +1378,11 @@ public interface LanguageConnectionConte
      */
     public void forgetDeferredConstraintsData(long conglomId)
             throws StandardException;
+
+    /**
+     * Get the SQL session context of the given activation.
+     * @param activation The activation
+     * @return           The SQL session object
+     */
+    public SQLSessionContext getCurrentSQLSessionContext(Activation activation);
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ReferencedKeyConstraintDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ReferencedKeyConstraintDescriptor.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ReferencedKeyConstraintDescriptor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ReferencedKeyConstraintDescriptor.java Tue Apr 29 00:23:52 2014
@@ -150,16 +150,16 @@ public class ReferencedKeyConstraintDesc
 
 
 	/**
-	 * Am I referenced by a FK on another table?
+     * Am I referenced by a FK on another table? Return the list of those
+     * foreign constraints.
 	 * @param type ConstraintDescriptor.(ENABLED|DISABLED|ALL)
-	 * @return	true/false
+     * @return  list of constraints
 	 * @exception StandardException on error
 	 */
-	public boolean hasNonSelfReferencingFK(int type) 
+    public ConstraintDescriptorList getNonSelfReferencingFK(int type)
 		throws StandardException
 	{
-
-		boolean hasNonSelfReferenceFk = false;
+        ConstraintDescriptorList result = new ConstraintDescriptorList();
 
 		if (SanityManager.DEBUG)
 		{
@@ -167,10 +167,8 @@ public class ReferencedKeyConstraintDesc
 		}
 	
 		ForeignKeyConstraintDescriptor fkcd;
-		// Get a full list of referencing keys,
-		ConstraintDescriptorList cdl = getForeignKeyConstraints(type);
 
-        for (ConstraintDescriptor cd : cdl)
+        for (ConstraintDescriptor cd : getForeignKeyConstraints(type))
 		{
 			if (! (cd instanceof ForeignKeyConstraintDescriptor))
 			{
@@ -180,11 +178,10 @@ public class ReferencedKeyConstraintDesc
 			fkcd = (ForeignKeyConstraintDescriptor) cd;
 			if(!(fkcd.getTableId().equals(getTableId())))
 			{
-				hasNonSelfReferenceFk = true;
-				break;
+                result.add(fkcd);
 			}
 		}
-		return hasNonSelfReferenceFk;
+        return result;
 	}
 
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/SQLBoolean.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/SQLBoolean.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/SQLBoolean.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/SQLBoolean.java Tue Apr 29 00:23:52 2014
@@ -885,7 +885,8 @@ public final class SQLBoolean
             final UUID constrId = (UUID)ps.getSavedObject(savedUUIDIdx);
             final LanguageConnectionContext lcc =
                 a.getLanguageConnectionContext();
-            final boolean isDeferred = lcc.isEffectivelyDeferred(a, constrId);
+            final boolean isDeferred = lcc.isEffectivelyDeferred(
+                   lcc.getCurrentSQLSessionContext(a), constrId);
 
             if (!isDeferred) {
                 throw StandardException.newException(

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java Tue Apr 29 00:23:52 2014
@@ -906,6 +906,8 @@ abstract class DMLModStatementNode exten
 		ConstraintDescriptorList			activeList = dd.getActiveConstraintDescriptors(cdl);
 		int[]								rowMap = getRowMap(readColsBitSet, td);
         int[]                               raRules;
+        boolean[]                           deferrable;
+        UUID[]                              fkIds;
 		ArrayList<String>              refTableNames = new ArrayList<String>(1);
 		ArrayList<Long>               refIndexConglomNum = new ArrayList<Long>(1);
 		ArrayList<Integer>            refActions = new ArrayList<Integer>(1);
@@ -926,13 +928,16 @@ abstract class DMLModStatementNode exten
 				type = FKInfo.FOREIGN_KEY;
 				refcd = ((ForeignKeyConstraintDescriptor)cd).getReferencedConstraint();
 				uuids = new UUID[1];
+                deferrable = new boolean[1];
+                fkIds = new UUID[1];
 				conglomNumbers = new long[1];
 				fkNames = new String[1];
 				isSelfReferencingFK = new boolean[1];
 				raRules = new int[1];
-				fkSetupArrays(dd, (ForeignKeyConstraintDescriptor)cd, 
-						0, uuids, conglomNumbers, 
-						fkNames, isSelfReferencingFK, raRules);
+                fkSetupArrays(
+                    dd, (ForeignKeyConstraintDescriptor)cd,
+                    0, uuids, conglomNumbers,
+                    fkNames, isSelfReferencingFK, raRules, deferrable, fkIds);
 
 				// oops, get the right constraint name -- for error
 				// handling we want the FK name, not refcd name
@@ -958,6 +963,8 @@ abstract class DMLModStatementNode exten
 				}
 
 				uuids = new UUID[size];
+                deferrable = new boolean[size];
+                fkIds = new UUID[size];
 				fkNames = new String[size];
 				conglomNumbers = new long[size];
 				isSelfReferencingFK = new boolean[size];
@@ -971,9 +978,11 @@ abstract class DMLModStatementNode exten
 				{
                     ForeignKeyConstraintDescriptor fkcd =
                         (ForeignKeyConstraintDescriptor) fkcdl.elementAt(inner);
-					fkSetupArrays(dd, fkcd,
-								inner, uuids, conglomNumbers, fkNames,
-								isSelfReferencingFK, raRules);
+                    fkSetupArrays(
+                        dd, fkcd,
+                        inner, uuids, conglomNumbers, fkNames,
+                        isSelfReferencingFK, raRules, deferrable, fkIds);
+
 					if((raRules[inner] == StatementType.RA_CASCADE) || 
 					   (raRules[inner] ==StatementType.RA_SETNULL))
 					{
@@ -1007,8 +1016,10 @@ abstract class DMLModStatementNode exten
 			ConglomerateDescriptor pkIndexConglom = pktd.getConglomerateDescriptor(pkuuid);
 
 			TableDescriptor refTd = cd.getTableDescriptor();
+
             fkList.add(new FKInfo(
 									fkNames,							// foreign key names
+                                    cd.getSchemaDescriptor().getSchemaName(),
 									refTd.getName(),				// table being modified
 									statementType,						// INSERT|UPDATE|DELETE
 									type,								// FOREIGN_KEY|REFERENCED_KEY
@@ -1020,7 +1031,9 @@ abstract class DMLModStatementNode exten
 									remapReferencedColumns(cd, rowMap),	// column referened by key
 									dd.getRowLocationTemplate(getLanguageConnectionContext(), refTd),
 									                                // row location template  for table being modified 
-									raRules));	// referential action rules
+                                    raRules,// referential action rules
+                                    deferrable,
+                                    fkIds));
 
 		}
 		
@@ -1064,12 +1077,16 @@ abstract class DMLModStatementNode exten
 		long[]								conglomNumbers,
 		String[]							fkNames,
 		boolean[]							isSelfReferencingFK,
-		int[]                               raRules
+        int[]                               raRules,
+        boolean[]                           isDeferrable,
+        UUID[]                              fkIds
 	)
 		throws StandardException
 	{
 		fkNames[index] = fkcd.getConstraintName();
 		uuids[index] = fkcd.getIndexId();
+        isDeferrable[index] = fkcd.deferrable();
+        fkIds[index] = fkcd.getUUID();
 		conglomNumbers[index] = fkcd.getIndexConglomerateDescriptor(dd).getConglomerateNumber();
 		isSelfReferencingFK[index] = fkcd.isSelfReferencingFK();
 		if(statementType == StatementType.DELETE)

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableElementList.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableElementList.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableElementList.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableElementList.java Tue Apr 29 00:23:52 2014
@@ -1054,12 +1054,19 @@ class TableElementList extends QueryTree
 						constraintType, dd);
 				} 
                 else 
-                {
+                {   // PRIMARY KEY, FOREIGN KEY
+                    // For foreign key constraint we do no mark the
+                    // index as deferrable; since checking isn't done on
+                    // duplicate keys there.
 					indexAction = genIndexAction(
 						forCreateTable,
 						constraintDN.requiresUniqueIndex(), false,
-                        cChars[0], // deferrable
-                        cChars[1], // initiallyDeferred?
+                        cChars[0] &                     // deferrable ?
+                                (constraintType !=
+                                 DataDictionary.FOREIGNKEY_CONSTRAINT),
+                        cChars[1] &                     // initiallyDeferred ?
+                                (constraintType !=
+                                 DataDictionary.FOREIGNKEY_CONSTRAINT),
 						null, constraintDN,
 						columnNames, true, tableSd, tableName,
 						constraintType, dd);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java Tue Apr 29 00:23:52 2014
@@ -87,9 +87,6 @@ import org.apache.derby.impl.sql.Generic
 import org.apache.derby.impl.sql.GenericStatement;
 import org.apache.derby.impl.sql.compile.CompilerContextImpl;
 import org.apache.derby.impl.sql.execute.AutoincrementCounter;
-import org.apache.derby.impl.sql.execute.DeferredConstraintsMemory;
-import org.apache.derby.impl.sql.execute.DeferredConstraintsMemory.CheckInfo;
-import org.apache.derby.impl.sql.execute.DeferredConstraintsMemory.UniquePkInfo;
 import org.apache.derby.impl.sql.execute.DeferredConstraintsMemory.ValidationInfo;
 import org.apache.derby.shared.common.sanity.SanityManager;
 /**
@@ -3792,76 +3789,16 @@ public class GenericLanguageConnectionCo
             return;
         }
 
-        DataDictionary dd = getDataDictionary();
-
         // Check all constraints that were deferred inside the routine
         // but whose constraint mode is immediate on the outside. If
         // any of these violate the constraints, roll back.
-        Set<Map.Entry<Long, ValidationInfo>> es = deferredHashTables.entrySet();
-
-        for (Map.Entry<Long, ValidationInfo> e : es) {
-
-            if (e.getValue() instanceof UniquePkInfo) {
-
-                final long indexCID = e.getKey().longValue();
-
-                boolean effectivelyDeferred =
-                    effectivelyDeferred(caller, indexCID);
-
-                if (effectivelyDeferred ) {
-                    // the constraint is also deferred in the calling context
-                    continue;
-                }
-
-                doValidateUniquePKConstraint(
-                    e.getKey().longValue(),
-                    (UniquePkInfo)e.getValue(),
-                    true);
-
-            } else if (e.getValue() instanceof CheckInfo) {
-
-                final long baseTableCID = e.getKey().longValue();
-                CheckInfo ci = (CheckInfo)e.getValue();
-
-                // check if any of the constraints involved is immediate on
-                // the outside
-                boolean allEffectivelyDeferred = true;
-
-                for (UUID uid : ci.getCulprints()) {
-                    if (!effectivelyDeferred(caller, uid) &&
-                        effectivelyDeferred(nested, uid)) {
-
-                        // at least one check constraint changed back
-                        // from being deferred to immediate, so check
-                        // all immediates
-
-                        // FIXME: could be optimized if we knew
-                        // exactly which constraints failed under the
-                        // deferred regime: that might save us from
-                        // checking in a few cases.
-                        allEffectivelyDeferred = false;
-                        break;
-                    }
-                }
-
-                if (allEffectivelyDeferred) {
-                    continue;
-                }
-
-                doValidateUniqueCheckConstraints(
-                    baseTableCID,
-                    null,
-                    (CheckInfo)e.getValue(),
-                    true);
-            } else {
-                if (SanityManager.DEBUG) {
-                    SanityManager.NOTREACHED();
-                }
-            }
+        for (Map.Entry<Long, ValidationInfo> e : deferredHashTables.entrySet())
+        {
+            e.getValue().possiblyValidateOnReturn(e, this, nested, caller);
         }
     }
 
-    private boolean effectivelyDeferred(SQLSessionContext sc, long indexCID)
+    public boolean isEffectivelyDeferred(SQLSessionContext sc, long indexCID)
             throws StandardException {
 
         Boolean deferred = sc.isDeferred(indexCID);
@@ -3884,7 +3821,7 @@ public class GenericLanguageConnectionCo
         return effectivelyDeferred;
     }
 
-    private boolean effectivelyDeferred(SQLSessionContext sc, UUID constraintId)
+    public boolean isEffectivelyDeferred(SQLSessionContext sc, UUID constraintId)
             throws StandardException {
 
         final Boolean deferred = sc.isDeferred(constraintId);
@@ -4053,21 +3990,6 @@ public class GenericLanguageConnectionCo
 
     }
 
-    public boolean isEffectivelyDeferred(
-            final Activation a,
-            long conglomId) throws StandardException {
-
-        return effectivelyDeferred(getCurrentSQLSessionContext(a), conglomId);
-    }
-
-    public boolean isEffectivelyDeferred(
-            final Activation a,
-            final UUID constraintId) throws StandardException {
-
-        return effectivelyDeferred(getCurrentSQLSessionContext(a),
-                                   constraintId);
-    }
-
     public void checkIntegrity() throws StandardException {
         validateDeferredConstraints(true);
         clearDeferreds();
@@ -4089,9 +4011,6 @@ public class GenericLanguageConnectionCo
         getCurrentSQLSessionContext().resetConstraintModes();
     }
 
-    /**
-     * {@inheritDoc}
-     */
     public void setDeferredAll(final Activation a, final boolean deferred)
             throws StandardException {
         if (!deferred) {
@@ -4104,9 +4023,6 @@ public class GenericLanguageConnectionCo
             Boolean.valueOf(deferred));
     }
 
-    /**
-     * {@inheritDoc}
-     */
     public HashMap<Long, ValidationInfo> getDeferredHashTables() {
         if (deferredHashTables == null) {
             deferredHashTables = new HashMap<Long, ValidationInfo>();
@@ -4126,20 +4042,8 @@ public class GenericLanguageConnectionCo
                 deferredHashTables.entrySet();
 
         for (Map.Entry<Long, ValidationInfo> e : es) {
-            if (e.getValue() instanceof UniquePkInfo) {
-                doValidateUniquePKConstraint(e.getKey().longValue(),
-                        (UniquePkInfo)e.getValue(),
-                        rollbackOnError);
-            } else if (e.getValue() instanceof CheckInfo) {
-                doValidateUniqueCheckConstraints(e.getKey().longValue(),
-                        null,
-                        (CheckInfo)e.getValue(),
-                        rollbackOnError);
-            } else {
-                if (SanityManager.DEBUG) {
-                    SanityManager.NOTREACHED();
-                }
-            }
+            final long cid = e.getKey().longValue();
+            e.getValue().validateConstraint(this, cid, null, rollbackOnError);
         }
     }
 
@@ -4156,37 +4060,7 @@ public class GenericLanguageConnectionCo
             return;
         }
 
-        if (vi instanceof CheckInfo) {
-            doValidateUniqueCheckConstraints(
-                conglomCID, constraintId, (CheckInfo)vi, false);
-        } else if (vi instanceof UniquePkInfo) {
-            doValidateUniquePKConstraint(conglomCID, (UniquePkInfo)vi, false);
-        } else {
-            if (SanityManager.DEBUG) {
-                SanityManager.NOTREACHED();
-            }
-        }
-
+        vi.validateConstraint(this, conglomCID, constraintId, false);
         deferredHashTables.remove(conglomCID);
     }
-
-    private void doValidateUniquePKConstraint(
-        final long indexCID,
-        final UniquePkInfo ui,
-        final boolean rollbackOnError) throws StandardException {
-
-        DeferredConstraintsMemory.validateUniquePK(
-                this, indexCID, ui.infoRows, rollbackOnError);
-    }
-
-    private void doValidateUniqueCheckConstraints(
-        final long baseTableCID,
-        final UUID constraintId,
-        final CheckInfo ci,
-        final boolean rollbackOnError) throws StandardException {
-
-        DeferredConstraintsMemory.validateCheck(
-                this, baseTableCID, constraintId, ci, rollbackOnError);
-    }
-
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java Tue Apr 29 00:23:52 2014
@@ -22,10 +22,8 @@
 package org.apache.derby.impl.sql.execute;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Properties;
-
 import org.apache.derby.catalog.DefaultInfo;
 import org.apache.derby.catalog.Dependable;
 import org.apache.derby.catalog.DependableFinder;
@@ -37,7 +35,6 @@ import org.apache.derby.iapi.error.Stand
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.services.io.FormatableBitSet;
 import org.apache.derby.iapi.services.io.StreamStorable;
-import org.apache.derby.shared.common.sanity.SanityManager;
 import org.apache.derby.iapi.sql.Activation;
 import org.apache.derby.iapi.sql.PreparedStatement;
 import org.apache.derby.iapi.sql.ResultSet;
@@ -57,14 +54,15 @@ import org.apache.derby.iapi.sql.diction
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
 import org.apache.derby.iapi.sql.dictionary.DefaultDescriptor;
 import org.apache.derby.iapi.sql.dictionary.DependencyDescriptor;
+import org.apache.derby.iapi.sql.dictionary.ForeignKeyConstraintDescriptor;
 import org.apache.derby.iapi.sql.dictionary.IndexLister;
 import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
 import org.apache.derby.iapi.sql.dictionary.ReferencedKeyConstraintDescriptor;
+import org.apache.derby.iapi.sql.dictionary.SPSDescriptor;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
 import org.apache.derby.iapi.sql.dictionary.StatisticsDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TriggerDescriptor;
-import org.apache.derby.iapi.sql.dictionary.SPSDescriptor;
 import org.apache.derby.iapi.sql.execute.ConstantAction;
 import org.apache.derby.iapi.sql.execute.ExecIndexRow;
 import org.apache.derby.iapi.sql.execute.ExecRow;
@@ -86,6 +84,7 @@ import org.apache.derby.iapi.util.IdUtil
 import org.apache.derby.iapi.util.StringUtil;
 import org.apache.derby.impl.sql.compile.ColumnDefinitionNode;
 import org.apache.derby.impl.sql.compile.StatementNode;
+import org.apache.derby.shared.common.sanity.SanityManager;
 
 /**
  *	This class  describes actions that are ALWAYS performed for an
@@ -316,24 +315,8 @@ class AlterTableConstantAction extends D
         boolean						tableScanned = false;
 
         if (compressTable || truncateTable) {
-            final HashMap<Long, DeferredConstraintsMemory.ValidationInfo> vis =
-                    lcc.getDeferredHashTables();
-            td = dd.getTableDescriptor(tableId);
-            final DeferredConstraintsMemory.ValidationInfo vi =
-                    vis.get(td.getHeapConglomerateId());
-
-            if (td == null) {
-                throw StandardException.newException(
-                    SQLState.LANG_TABLE_NOT_FOUND_DURING_EXECUTION, tableName);
-            }
-
-            if (vi != null &&
-                vi instanceof DeferredConstraintsMemory.CheckInfo) {
-                // We can not use row locations when re-visiting offending
-                // rows in this table, since we are truncating or compressing.
-                ((DeferredConstraintsMemory.CheckInfo)vi).
-                    setInvalidatedRowLocations();
-            }
+            DeferredConstraintsMemory.compressOrTruncate(
+                    lcc, tableId, tableName);
         }
 
         //Following if is for inplace compress. Compress using temporary
@@ -2438,18 +2421,34 @@ class AlterTableConstantAction extends D
 		}
 
 
-		//truncate table is not allowed if there are any tables referencing it.
-		//except if it is self referencing.
-		ConstraintDescriptorList cdl = dd.getConstraintDescriptors(td);
-		for(int index = 0; index < cdl.size(); index++)
-		{
-			ConstraintDescriptor cd = cdl.elementAt(index);
+        // Truncate table is not allowed if there are any tables referencing it.
+        // except if it is self referencing, or if the constraint is deferred
+        // and the ON DELETE action is NO ACTION.
+        for(ConstraintDescriptor cd : dd.getConstraintDescriptors(td)) {
 			if (cd instanceof ReferencedKeyConstraintDescriptor)
 			{
-				ReferencedKeyConstraintDescriptor rfcd = (ReferencedKeyConstraintDescriptor) cd;
-				if(rfcd.hasNonSelfReferencingFK(ConstraintDescriptor.ENABLED))
-				{
-					throw StandardException.newException(SQLState.LANG_NO_TRUNCATE_ON_FK_REFERENCE_TABLE,td.getName());
+                final ReferencedKeyConstraintDescriptor rfcd =
+                    (ReferencedKeyConstraintDescriptor)cd;
+
+                for (ConstraintDescriptor fkcd :
+                     rfcd.getNonSelfReferencingFK(ConstraintDescriptor.ENABLED))
+                {
+                    final ForeignKeyConstraintDescriptor fk =
+                            (ForeignKeyConstraintDescriptor)fkcd;
+
+                    if (fk.deferrable() &&
+                        lcc.isEffectivelyDeferred(
+                                lcc.getCurrentSQLSessionContext(activation),
+                                fk.getUUID()) &&
+                        fk.getRaDeleteRule() == StatementType.RA_NOACTION) {
+                        // Allowed. We will update the indexCID later, see
+                        // updateIndex, so we know where to look when
+                        // checking time comes.
+                    } else {
+                        throw StandardException.newException(
+                            SQLState.LANG_NO_TRUNCATE_ON_FK_REFERENCE_TABLE,
+                            td.getName());
+                    }
 				}
 			}
 		}
@@ -2771,6 +2770,13 @@ class AlterTableConstantAction extends D
 
 		// Drop the old conglomerate
 		tc.dropConglomerate(indexConglomerateNumbers[index]);
+
+        DeferredConstraintsMemory.updateIndexCIDs(
+                lcc,
+                truncateTable,
+                indexConglomerateNumbers[index],
+                newIndexCongloms[index]);
+
 	}
 
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ConstraintConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ConstraintConstantAction.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ConstraintConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ConstraintConstantAction.java Tue Apr 29 00:23:52 2014
@@ -22,12 +22,12 @@
 package org.apache.derby.impl.sql.execute;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import org.apache.derby.catalog.UUID;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.services.io.FormatableBitSet;
+import org.apache.derby.iapi.sql.Activation;
 import org.apache.derby.iapi.sql.PreparedStatement;
 import org.apache.derby.iapi.sql.ResultSet;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
@@ -36,7 +36,6 @@ import org.apache.derby.iapi.sql.diction
 import org.apache.derby.iapi.sql.dictionary.ReferencedKeyConstraintDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.execute.ExecRow;
-import org.apache.derby.iapi.store.access.BackingStoreHashtable;
 import org.apache.derby.iapi.store.access.ConglomerateController;
 import org.apache.derby.iapi.store.access.GroupFetchScanController;
 import org.apache.derby.iapi.store.access.ScanController;
@@ -44,7 +43,6 @@ import org.apache.derby.iapi.store.acces
 import org.apache.derby.iapi.types.DataValueDescriptor;
 import org.apache.derby.iapi.types.NumberDataValue;
 import org.apache.derby.impl.sql.execute.DeferredConstraintsMemory.CheckInfo;
-import org.apache.derby.impl.sql.execute.DeferredConstraintsMemory.ValidationInfo;
 import org.apache.derby.impl.store.access.heap.HeapRowLocation;
 import org.apache.derby.shared.common.sanity.SanityManager;
 /**
@@ -152,6 +150,7 @@ public abstract class ConstraintConstant
 	 */
 	static void validateFKConstraint
 	(
+        Activation                          activation,
 		TransactionController				tc,
 		DataDictionary						dd,
 		ForeignKeyConstraintDescriptor		fk,
@@ -229,12 +228,22 @@ public abstract class ConstraintConstant
                         ScanController.GT             	// stopSearchOp 
                         );
 
-			RIBulkChecker riChecker = new RIBulkChecker(refScan, 
-										fkScan, 
-										indexTemplateRow, 	
-										true, 				// fail on 1st failure
-										(ConglomerateController)null,
-										(ExecRow)null);
+            RIBulkChecker riChecker = new RIBulkChecker(
+                    activation,
+                    refScan,
+                    fkScan,
+                    indexTemplateRow,
+                    true,               // fail on 1st failure
+                    (ConglomerateController)null,
+                    (ExecRow)null,
+                    fk.getTableDescriptor().getSchemaName(),
+                    fk.getTableDescriptor().getName(),
+                    fk.getUUID(),
+                    fk.deferrable(),
+                    fk.getIndexConglomerateDescriptor(dd).
+                        getConglomerateNumber(),
+                    refcd.getIndexConglomerateDescriptor(dd).
+                        getConglomerateNumber());
 
 			int numFailures = riChecker.doCheck();
 			if (numFailures > 0)
@@ -340,6 +349,7 @@ public abstract class ConstraintConstant
                         // violating rows, so for now, just pretend we know one,
                         // then invalidate the row location information forcing
                         // full table check at validation time
+                        CheckInfo newCi[] = new CheckInfo[1];
                         DeferredConstraintsMemory.rememberCheckViolations(
                                 lcc,
                                 td.getHeapConglomerateId(),
@@ -347,12 +357,9 @@ public abstract class ConstraintConstant
                                 td.getName(),
                                 null,
                                 violatingConstraints,
-                                new HeapRowLocation() /* dummy */);
-                        HashMap<Long, ValidationInfo>
-                                hashTables = lcc.getDeferredHashTables();
-                        CheckInfo ci = (CheckInfo)hashTables.get(
-                                Long.valueOf(td.getHeapConglomerateId()));
-                        ci.setInvalidatedRowLocations();
+                                new HeapRowLocation() /* dummy */,
+                                newCi);
+                        newCi[0].setInvalidatedRowLocations();
 
                     } else {
                         throw StandardException.newException(

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java Tue Apr 29 00:23:52 2014
@@ -295,8 +295,7 @@ public class CreateConstraintConstantAct
                 dd.checkVersion(DataDictionary.DD_VERSION_DERBY_10_11,
                                 "DEFERRED CONSTRAINTS");
 
-                if (constraintType == DataDictionary.FOREIGNKEY_CONSTRAINT ||
-                    constraintType == DataDictionary.NOTNULL_CONSTRAINT ||
+                if (constraintType == DataDictionary.NOTNULL_CONSTRAINT ||
                     !characteristics[2] /* not enforced */) {
 
                     // Remove when feature DERBY-532 is completed
@@ -391,7 +390,8 @@ public class CreateConstraintConstantAct
 				if ( (! forCreateTable) && 
 					 dd.activeConstraint( conDesc ) )
 				{
-					validateFKConstraint(tc, 
+                    validateFKConstraint(activation,
+                                         tc,
 										 dd, 
 										 (ForeignKeyConstraintDescriptor)conDesc, 
 										 referencedConstraint,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeferredConstraintsMemory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeferredConstraintsMemory.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeferredConstraintsMemory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeferredConstraintsMemory.java Tue Apr 29 00:23:52 2014
@@ -25,15 +25,19 @@ import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import org.apache.derby.catalog.UUID;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.services.io.FormatableBitSet;
+import org.apache.derby.iapi.sql.Activation;
 import org.apache.derby.iapi.sql.PreparedStatement;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
+import org.apache.derby.iapi.sql.conn.SQLSessionContext;
 import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
 import org.apache.derby.iapi.sql.dictionary.ConstraintDescriptor;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
+import org.apache.derby.iapi.sql.dictionary.ForeignKeyConstraintDescriptor;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.execute.ExecRow;
@@ -144,6 +148,7 @@ final public class DeferredConstraintsMe
      * @param violatingCheckConstraints offending constraint(s)
      * @param offendingRow the duplicate row to be saved in the hash table
      *                  for later checking
+     * @param result OUT parameter: the allocated CheckInfo
      * @return the hash table (for caching by client to minimize lookups)
      * @throws StandardException standard error policy
      */
@@ -154,7 +159,8 @@ final public class DeferredConstraintsMe
             final String tableName,
                   BackingStoreHashtable deferredCheckViolations,
             final List<UUID> violatingCheckConstraints,
-            final RowLocation offendingRow) throws StandardException {
+            final RowLocation offendingRow,
+            CheckInfo[] result) throws StandardException {
 
         if (violatingCheckConstraints.isEmpty()) {
             return null;
@@ -176,15 +182,16 @@ final public class DeferredConstraintsMe
                 // offending row
                 deferredCheckViolations =
                         makeDeferredHashTable(lcc.getTransactionExecute(), 1);
-
-                hashTables.put(Long.valueOf(basetableCID),
-                               new CheckInfo(deferredCheckViolations,
+                CheckInfo ci = new CheckInfo(deferredCheckViolations,
                                    schemaName,
                                    tableName,
-                                   violatingCheckConstraints));
+                                   violatingCheckConstraints);
+                hashTables.put(Long.valueOf(basetableCID), ci);
+                result[0] = ci;
             } else {
                 vi.addCulprits(violatingCheckConstraints);
                 deferredCheckViolations = vi.infoRows;
+                result[0] = vi;
             }
         }
 
@@ -196,6 +203,74 @@ final public class DeferredConstraintsMe
     }
 
 
+    public static Enumeration<Object> getDeferredCheckConstraintLocations(
+            Activation activation,
+            long validatingBaseTableCID) throws StandardException {
+
+        CheckInfo ci = (DeferredConstraintsMemory.CheckInfo)activation.
+                getLanguageConnectionContext().
+                getDeferredHashTables().get(
+                    Long.valueOf(validatingBaseTableCID));
+        return ci.infoRows.elements();
+    }
+
+    /**
+     * Make note of a violated foreign key constraint, i.e. the referenced
+     * key is not present
+     *
+     * @param lcc the language connection context
+     * @param deferredRowsHashTable cached client copy
+     * @param siCID the conglomerate ID of the supporting index of the FK
+     * @param rtCID the conglomerate id of the index of the referenced table
+     * @param fkId the UUID of the foreign key constraint
+     * @param indexRow the row in the supporting index which contains
+     *        the key which is not present in the referenced index.
+     * @param schemaName the schema of the table
+     * @param tableName the table being modified that has a FK.
+     * @return value to cache
+     * @throws StandardException
+     */
+    public static BackingStoreHashtable rememberFKViolation(
+            final LanguageConnectionContext lcc,
+                  BackingStoreHashtable deferredRowsHashTable,
+            final long siCID,
+            final long rtCID,
+            final UUID fkId,
+            final DataValueDescriptor[] indexRow,
+            String schemaName,
+            String tableName) throws StandardException {
+
+        if (deferredRowsHashTable == null) {
+            // Use the backing hashtable for this index's deferred rows
+            // by checking the transaction's map of such backing hash
+            // tables (one per index conglomerate).  Use it if it
+            // exists, else make a new one.
+
+            final HashMap<Long, ValidationInfo> hashTables =
+                lcc.getDeferredHashTables();
+            final ValidationInfo vi = hashTables.get(Long.valueOf(siCID));
+
+            if (vi == null) {
+                deferredRowsHashTable = makeDeferredHashTable(
+                    lcc.getTransactionExecute(), indexRow.length);
+
+                hashTables.put(
+                    Long.valueOf(siCID),
+                    new ForeignKeyInfo(deferredRowsHashTable, fkId, rtCID,
+                                       schemaName, tableName));
+            } else {
+                deferredRowsHashTable = vi.infoRows;
+            }
+        }
+
+        DataValueDescriptor[] hashRowArray =
+            new DataValueDescriptor[indexRow.length];
+        System.arraycopy(indexRow, 0, hashRowArray, 0, indexRow.length);
+        deferredRowsHashTable.putRow(true, hashRowArray, null);
+
+        return deferredRowsHashTable;
+    }
+
     /**
      * After having belatedly learned the identity of the conglomerate, we now
      * associate the conglomerate id information with the saved duplicates
@@ -209,6 +284,13 @@ final public class DeferredConstraintsMe
     public static void associateDuplicatesWithConglomerate(
             final LanguageConnectionContext lcc,
             final long indexCID) {
+        updateKey(lcc, UNDEFINED_CONGLOMERATE, indexCID);
+    }
+
+    private static void updateKey(
+        LanguageConnectionContext lcc,
+        long oldCID,
+        long newCID) {
 
         final HashMap<Long, ValidationInfo> hashTables =
                     lcc.getDeferredHashTables();
@@ -217,17 +299,16 @@ final public class DeferredConstraintsMe
             return; // no duplicates recorded in this transaction
         }
 
-        final ValidationInfo ht = hashTables.remove(
-                Long.valueOf(UNDEFINED_CONGLOMERATE));
+        final ValidationInfo ht = hashTables.remove(Long.valueOf(oldCID));
 
         if (ht != null) {
-            hashTables.put(indexCID, ht);
-        } // else no duplicates recorded for this index
+            hashTables.put(newCID, ht);
+        }
     }
 
     /**
      * The conglomerate id for an index with deferred row checking needs
-     * updating in the memory if the underlying index is rebuilt, for example
+     * updating in the memory if the underlying index is rebuilt
      * on bulk insert for import.
      *
      * @param lcc the language connection context needed to find the
@@ -240,19 +321,7 @@ final public class DeferredConstraintsMe
             final long oldIndexCID,
             final long newIndexCID) {
 
-        final HashMap<Long, ValidationInfo> hashTables =
-                    lcc.getDeferredHashTables();
-
-        if (hashTables == null) {
-            return; // no duplicates recorded in this transaction
-        }
-
-        final ValidationInfo ht = hashTables.remove(
-                Long.valueOf(oldIndexCID));
-
-        if (ht != null) {
-            hashTables.put(Long.valueOf(newIndexCID), ht);
-        }
+        updateKey(lcc, oldIndexCID, newIndexCID);
     }
 
     private static BackingStoreHashtable makeDeferredHashTable(
@@ -281,246 +350,94 @@ final public class DeferredConstraintsMe
                 false);
     }
 
-    private static BackingStoreHashtable makeDeferredCheck(
-            final TransactionController tc,
-            final int cols) throws StandardException {
-
-        // key: all columns (these are index rows)
-        final int[] keyCols = new int[cols];
-
-        for (int i = 0; i < cols; i++) {
-            keyCols[i] = i;
-        }
-
-        return new BackingStoreHashtable(
-                tc,
-                null,
-                keyCols,
-                true, // remove duplicates: no need for more copies:
-                      // one is enough to know what to look for on commit
-                -1,
-                HashScanResultSet.DEFAULT_MAX_CAPACITY,
-                HashScanResultSet.DEFAULT_INITIAL_CAPACITY,
-                HashScanResultSet.DEFAULT_MAX_CAPACITY,
-                false,
-                false);
-    }
-
     /**
-     * Validate one primary key or unique constraint
+     * Update the conglomerate ids of supporting indexes, when they change,
+     * under truncate and compress.
      *
-     * @param lcc       The language connection context
-     * @param indexCID  The conglomerate id of the index backing the constraint
-     * @param ht        Cached saved rows if any
-     * @param rollbackOnError {@code true} if we should roll back the
-     *                  transaction if we see a violation of the constraint
+     * @param lcc       the language connection context
+     * @param truncate  if {@code true} we are in a TRUNCATE TABLE context
+     * @param oldCID    the old conglomerate id of a supporting index
+     * @param newCID    the old conglomerate id of a supporting index
      * @throws StandardException
      */
-    public static void validateUniquePK(
-            final LanguageConnectionContext lcc,
-            final long indexCID,
-            final BackingStoreHashtable ht,
-            final boolean rollbackOnError) throws StandardException {
+    public static void updateIndexCIDs(
+            LanguageConnectionContext lcc,
+            boolean truncate,
+            long oldCID,
+            long newCID) throws StandardException {
+
+        // Handle reference to the old index conglomerate ID for
+        // deferred constraints
+        final HashMap<Long, ValidationInfo> vis = lcc.getDeferredHashTables();
+
+        if (vis == null) {
+            return;
+        }
+
+        if (truncate) {
+            // Invalidate any deferred constraints information based on this
+            // index.
+            final ValidationInfo vi =  vis.get(oldCID);
+
+            if (vi != null &&
+                    (vi instanceof UniquePkInfo ||
+                     vi instanceof ForeignKeyInfo)) {
+                lcc.forgetDeferredConstraintsData(oldCID);
+            }
+        } else {
+            final ValidationInfo vi =  vis.get(oldCID);
 
-        final TransactionController tc = lcc.getTransactionExecute();
-        final Enumeration<?> e = ht.elements();
+            if (vi != null && vi instanceof UniquePkInfo){
+                updateKey(lcc, oldCID, newCID);
+            }
+        }
 
-        while (e.hasMoreElements()) {
-            final DataValueDescriptor[] key =
-                    (DataValueDescriptor[])e.nextElement();
-
-            // FIXME: This is not very efficient: we could sort the rows in the
-            // hash table, and then check all rows using a single scan.
-            ScanController indexSC = null;
-            boolean sawException = false;
-
-            try {
-                indexSC = tc.openScan(
-                    indexCID,
-                    false,
-                    0, // read only
-                    TransactionController.MODE_RECORD,
-                    TransactionController.ISOLATION_READ_COMMITTED_NOHOLDLOCK,
-                    (FormatableBitSet)null, // retrieve all fields
-                    key,
-                    ScanController.GE, // startSearchOp
-                    null,
-                    key,
-                    ScanController.GT);
+        // Update conglomerate information for deferred foreign keys involved
+        // with this index re-creation.
+        for (Map.Entry<Long, ValidationInfo> e : vis.entrySet()) {
+            final ValidationInfo vi = e.getValue();
 
-                if (indexSC.next()) {
-                    if (indexSC.next()) {
-                        // two matching rows found, constraint violated
-                        final DataDictionary dd = lcc.getDataDictionary();
-                        final ConglomerateDescriptor cd =
-                                dd.getConglomerateDescriptor(indexCID);
-                        final TableDescriptor td =
-                                dd.getTableDescriptor(cd.getTableID());
-                        final ConstraintDescriptor conDesc =
-                                dd.getConstraintDescriptor(td, cd.getUUID());
+            if (vi instanceof ForeignKeyInfo) {
+                final ForeignKeyInfo fki = (ForeignKeyInfo)vi;
 
-                        throw StandardException.newException(
-                          rollbackOnError ?
-                            SQLState.LANG_DEFERRED_DUPLICATE_KEY_CONSTRAINT_T :
-                            SQLState.LANG_DEFERRED_DUPLICATE_KEY_CONSTRAINT_S,
-                          conDesc.getConstraintName(),
-                          td.getName());
-                    } // else exactly one row contains key: OK
-                } else {
-                    // No rows contain key: OK, must have been deleted later
-                    // in transaction, or we got here due to pessimistic
-                    // assumption on a timeout while checking on the insert.
-                }
-            } catch (StandardException se) {
-                sawException = true;
-                throw se;
-            } finally {
-                // Clean up resource usage
-                try {
-                    if (indexSC != null) {
-                        indexSC.close();
-                    }
-                } catch (StandardException ie) {
-                    if (!sawException) {
-                        throw ie;
-                    } // else: can't let it shadow preceding exception
+                if (fki.getRtCID() == oldCID) {
+                    fki.updateRtCID(newCID);
+
+                } else if (e.getKey().longValue() == oldCID) {
+                    updateKey(lcc, oldCID, newCID);
                 }
             }
         }
+
     }
 
 
-    /**
-     * Validate one or more CHECK constraints on a table.
-     * <p>
-     * Implementation note: We remember violations for a row by storing its row
-     * location in a disk based hash table, similar to what we do for the index
-     * location for primary key and unique constraints. As far as which
-     * constraints caused which violations, constraints are only presently
-     * remembered as having caused "one or more violations", i.e. for any
-     * violating row, we do not know at checking time exactly which constraint
-     * caused a problem for that exact row. So, for any given constraint which
-     * was violated in the transaction we visit all rows which had one or more
-     * violations and check again. This could be improved upon by remembering
-     * for each violating row the exact set of constraint(s) that saw a
-     * violation. Still, this is much faster than a full table scan in most use
-     * cases.  We use a special internal query option to achieve this.  The row
-     * locations may not be usable if Derby does a compress or otherwise makes
-     * them invalid. In that case we resort to a full table scan.
-     * @see ValidateCheckConstraintResultSet
-     *
-     * @param lcc          The language connection context
-     * @param baseTableCID The conglomerate id of the base table for which
-     *                     we want to validate check constraints.
-     * @param constraintId If not {@code null}, check only for this
-     *                     constraint.  This is used when switching mode to
-     *                     immediate.  If {@code null}, we check all check
-     *                     constraints, i.e.  at commit or if we have {@code
-     *                     SET CONSTRAINT ALL IMMEDIATE}.
-     * @param ci           The constraints info for the table we need
-     *                     to perform the check on.
-     * @param rollbackOnError If {@code true} and a violation occurs, throw
-     *                     and exception that will cause rollback.
-     * @throws StandardException
-     *                     Default error policy
-     */
-    public static void validateCheck(
-            final LanguageConnectionContext lcc,
-            final long baseTableCID,
-            final UUID constraintId,
-            final CheckInfo ci,
-            final boolean rollbackOnError) throws StandardException {
-
-        final TransactionController tc = lcc.getTransactionExecute();
-        final DataDictionary dd = lcc.getDataDictionary();
-        final SchemaDescriptor sd = dd.getSchemaDescriptor(
-                ci.schemaName, tc, true);
-
-        if (sd == null) {
-            if (SanityManager.DEBUG) {
-                // dropping of a schema shouold drop any tables and their
-                // constraints, which in turn should drop any deferred
-                // constraint memory of them.
-                SanityManager.NOTREACHED();
-            } else {
-                return;
-            }
-        }
+    public static void compressOrTruncate(
+            LanguageConnectionContext lcc,
+            UUID tableId,
+            String tableName) throws StandardException {
 
-        final TableDescriptor td = dd.getTableDescriptor(ci.tableName, sd, tc);
+        final HashMap<Long, DeferredConstraintsMemory.ValidationInfo> vis =
+                lcc.getDeferredHashTables();
+        final TableDescriptor td =
+                lcc.getDataDictionary().getTableDescriptor(tableId);
+        final DeferredConstraintsMemory.ValidationInfo vi =
+                vis.get(td.getHeapConglomerateId());
 
         if (td == null) {
-            if (SanityManager.DEBUG) {
-                // dropping of a table shouold drop any
-                // constraints, which in turn should drop any deferred
-                // constraint memory of them. Renaming of a table with
-                // constrants is not presently allowed. FIXME: use UUID
-                // instead of string here, more stable reference.
-                SanityManager.NOTREACHED();
-            }
-        } else {
-            for (UUID id : ci.culprits) {
-                if (constraintId == null || constraintId.equals(id)) {
-                    final ConstraintDescriptor cd =
-                        dd.getConstraintDescriptor(id);
-                    final StringBuilder checkStmt = new StringBuilder();
-                    checkStmt.append("SELECT 1 FROM ");
-                    checkStmt.append(td.getQualifiedName());
-
-                    // If a compress has happened in this transaction, we can't
-                    // trust the rowLocations, so make a full table scan. If
-                    // not, we optimize by using a special result set type
-                    // which utilized the saved away row locations for the
-                    // offending rows, so we only visit those when checking.
-                    // I.e. other rows are known to be good a priori.
-                    if (!ci.isInvalidated()) {
-                        checkStmt.append(
-                            " --DERBY-PROPERTIES joinStrategy=nestedLoop, " +
-                                "                    index=null, " +
-                                "                    validateCheckConstraint=");
-                        checkStmt.append(Long.toString(baseTableCID));
-                        checkStmt.append('\n');
-                    }
-
-                    checkStmt.append(" WHERE NOT(");
-                    checkStmt.append(cd.getConstraintText());
-                    checkStmt.append(')');
-
-                    BasicNoPutResultSetImpl rs = null;
-                    final PreparedStatement ps = lcc.prepareInternalStatement(
-                        lcc.getDefaultSchema(),
-                        checkStmt.toString(),
-                        true,
-                        true);
-                    try {
-                        rs = (BasicNoPutResultSetImpl)ps.execute(
-                            ps.getActivation(lcc, false), false, 0L);
-                        final ExecRow row = rs.getNextRowCore();
-
-                        if (row != null) {
-                            //check constraint violated
+            throw StandardException.newException(
+                    SQLState.LANG_TABLE_NOT_FOUND_DURING_EXECUTION, tableName);
+        }
 
-                            throw StandardException.newException(
-                                rollbackOnError ?
-                                    SQLState.LANG_DEFERRED_CHECK_CONSTRAINT_T :
-                                    SQLState.LANG_DEFERRED_CHECK_CONSTRAINT_S,
-                                cd.getConstraintName(),
-                                td.getName(),
-                                cd.getConstraintText());
-                        }
-                    } finally {
-                        if (rs != null) {
-                            try {
-                                rs.close();
-                            } catch (StandardException e) {}
-                        }
-                    }
-                }
-            }
+        if (vi != null &&
+                vi instanceof DeferredConstraintsMemory.CheckInfo) {
+            // We can not use row locations when re-visiting offending
+            // rows in this table, since we are truncating or compressing.
+            ((DeferredConstraintsMemory.CheckInfo)vi).
+                    setInvalidatedRowLocations();
         }
     }
 
-
     /**
      * Class hierarchy carrying the information we need to validate
      * some deferred constraint. For unique and primary key constraints, we
@@ -533,15 +450,140 @@ final public class DeferredConstraintsMe
         public ValidationInfo(final BackingStoreHashtable infoRows) {
             this.infoRows = infoRows;
         }
+
+        public abstract void possiblyValidateOnReturn(
+                Map.Entry<Long, ValidationInfo> e,
+                LanguageConnectionContext lcc,
+                SQLSessionContext nested,
+                SQLSessionContext caller) throws StandardException;
+
+        public abstract void validateConstraint(
+                LanguageConnectionContext lcc,
+                long conglomerateId,
+                UUID constraintId,
+                boolean rollbackOnError) throws StandardException;
     }
 
     /**
      * Info needed for unique and primary key constraints
      */
-    public static class UniquePkInfo extends ValidationInfo {
+    private static class UniquePkInfo extends ValidationInfo {
+
         public UniquePkInfo(final BackingStoreHashtable infoRows) {
             super(infoRows);
         }
+
+        public final void possiblyValidateOnReturn(
+                Map.Entry<Long, ValidationInfo> e,
+                LanguageConnectionContext lcc,
+                SQLSessionContext nested,
+                SQLSessionContext caller) throws StandardException {
+
+                final long indexCID = e.getKey().longValue();
+
+                if (lcc.isEffectivelyDeferred(caller, indexCID)) {
+                    // the constraint is also deferred in the calling context
+                    return;
+                }
+
+                validateUniquePK(lcc, indexCID, e.getValue().infoRows, true);
+        }
+
+        /**
+         * Validate one primary key or unique constraint
+         *
+         * @param lcc       The language connection context
+         * @param indexCID The conglomerate id of the index backing the
+         *                 constraint
+         * @param constraintId Not used by this constraint type
+         * @param rollbackOnError {@code true} if we should roll back the
+         *                  transaction if we see a violation of the constraint
+         * @throws StandardException
+         */
+        public final void validateConstraint(
+                LanguageConnectionContext lcc,
+                long indexCID,
+                UUID constraintId,
+                boolean rollbackOnError) throws StandardException {
+
+            validateUniquePK(
+                    lcc, indexCID, this.infoRows, rollbackOnError);
+        }
+
+        private static void validateUniquePK(
+                final LanguageConnectionContext lcc,
+                final long indexCID,
+                final BackingStoreHashtable ht,
+                final boolean rollbackOnError) throws StandardException {
+
+            final TransactionController tc = lcc.getTransactionExecute();
+            final Enumeration<?> e = ht.elements();
+
+            while (e.hasMoreElements()) {
+                final DataValueDescriptor[] key =
+                        (DataValueDescriptor[])e.nextElement();
+
+                // FIXME: This is not very efficient: we could sort the rows in
+                // the hash table, and then check all rows using a single scan.
+                ScanController indexSC = null;
+                boolean sawException = false;
+
+                try {
+                    indexSC = tc.openScan(
+                        indexCID,
+                        false,
+                        0, // read only
+                        TransactionController.MODE_RECORD,
+                        TransactionController.ISOLATION_READ_COMMITTED_NOHOLDLOCK,
+                        (FormatableBitSet)null, // retrieve all fields
+                        key,
+                        ScanController.GE, // startSearchOp
+                        null,
+                        key,
+                        ScanController.GT);
+
+                    if (indexSC.next()) {
+                        if (indexSC.next()) {
+                            // two matching rows found, constraint violated
+                            final DataDictionary dd = lcc.getDataDictionary();
+                            final ConglomerateDescriptor cd =
+                                dd.getConglomerateDescriptor(indexCID);
+                            final TableDescriptor td =
+                                dd.getTableDescriptor(cd.getTableID());
+                            final ConstraintDescriptor conDesc =
+                                dd.getConstraintDescriptor(td, cd.getUUID());
+
+                            throw StandardException.newException(
+                                rollbackOnError ?
+                                SQLState.
+                                    LANG_DEFERRED_DUPLICATE_KEY_CONSTRAINT_T :
+                                SQLState.
+                                    LANG_DEFERRED_DUPLICATE_KEY_CONSTRAINT_S,
+                                conDesc.getConstraintName(),
+                                td.getName());
+                        } // else exactly one row contains key: OK
+                    } else {
+                        // No rows contain key: OK, must have been deleted later
+                        // in transaction, or we got here due to pessimistic
+                        // assumption on a timeout while checking on the insert.
+                    }
+                } catch (StandardException se) {
+                    sawException = true;
+                    throw se;
+                } finally {
+                    // Clean up resource usage
+                    try {
+                        if (indexSC != null) {
+                            indexSC.close();
+                        }
+                    } catch (StandardException ie) {
+                        if (!sawException) {
+                            throw ie;
+                        } // else: can't let it shadow preceding exception
+                    }
+                }
+            }
+        }
     }
 
     /**
@@ -584,5 +626,357 @@ final public class DeferredConstraintsMe
         public List<UUID> getCulprints() {
             return culprits;
         }
+
+        public void possiblyValidateOnReturn(
+                Map.Entry<Long, ValidationInfo> e,
+                LanguageConnectionContext lcc,
+                SQLSessionContext nested,
+                SQLSessionContext caller) throws StandardException {
+
+            final long baseTableCID = e.getKey().longValue();
+
+            // check if any of the constraints involved is immediate on
+            // the outside
+            boolean allEffectivelyDeferred = true;
+
+            for (UUID uid : getCulprints()) {
+                if (!lcc.isEffectivelyDeferred(caller, uid) &&
+                        lcc.isEffectivelyDeferred(nested, uid)) {
+
+                    // at least one check constraint changed back
+                    // from being deferred to immediate, so check
+                    // all immediates
+
+                    // FIXME: could be optimized if we knew
+                    // exactly which constraints failed under the
+                    // deferred regime: that might save us from
+                    // checking in a few cases.
+                    allEffectivelyDeferred = false;
+                    break;
+                }
+            }
+
+            if (allEffectivelyDeferred) {
+                return;
+            }
+
+            validateCheck(lcc, baseTableCID, null, true);
+        }
+
+        /**
+         * Validate one or more CHECK constraints on a table.
+         * <p>
+         * Implementation note: We remember violations for a row by storing its
+         * row location in a disk based hash table, similar to what we do for
+         * the index location for primary key and unique constraints. As far as
+         * which constraints caused which violations, constraints are only
+         * presently remembered as having caused "one or more violations",
+         * i.e. for any violating row, we do not know at checking time exactly
+         * which constraint caused a problem for that exact row. So, for any
+         * given constraint which was violated in the transaction we visit all
+         * rows which had one or more violations and check again. This could be
+         * improved upon by remembering for each violating row the exact set of
+         * constraint(s) that saw a violation. Still, this is much faster than
+         * a full table scan in most use cases.  We use a special internal
+         * query option to achieve this.  The row locations may not be usable
+         * if Derby does a compress or otherwise makes them invalid. In that
+         * case we resort to a full table scan.</p>
+         * @see ValidateCheckConstraintResultSet
+         *
+         * @param lcc          The language connection context
+         * @param baseTableCID The conglomerate id of the base table for which
+         *                     we want to validate check constraints.
+         * @param constraintId If not {@code null}, check only for this
+         *                     constraint.  This is used when switching mode to
+         *                     immediate.  If {@code null}, we check all check
+         *                     constraints, i.e.  at commit or if we have {@code
+         *                     SET CONSTRAINT ALL IMMEDIATE}.
+         * @param rollbackOnError If {@code true} and a violation occurs, throw
+         *                     and exception that will cause rollback.
+         * @throws StandardException
+         *                     Default error policy
+         */
+        public final void validateConstraint(
+                LanguageConnectionContext lcc,
+                long baseTableCID,
+                UUID constraintId,
+                boolean rollbackOnError) throws StandardException {
+
+            validateCheck(lcc, baseTableCID, constraintId, rollbackOnError);
+        }
+
+        private void validateCheck(
+                final LanguageConnectionContext lcc,
+                final long baseTableCID,
+                final UUID constraintId,
+                final boolean rollbackOnError) throws StandardException {
+
+            final TransactionController tc = lcc.getTransactionExecute();
+            final DataDictionary dd = lcc.getDataDictionary();
+            final SchemaDescriptor sd = dd.getSchemaDescriptor(
+                    schemaName, tc, true);
+
+            if (sd == null) {
+                if (SanityManager.DEBUG) {
+                    // dropping of a schema shouold drop any tables and their
+                    // constraints, which in turn should drop any deferred
+                    // constraint memory of them.
+                    SanityManager.NOTREACHED();
+                } else {
+                    return;
+                }
+            }
+
+            final TableDescriptor td = dd.getTableDescriptor(tableName, sd, tc);
+
+            if (td == null) {
+                if (SanityManager.DEBUG) {
+                    // dropping of a table shouold drop any
+                    // constraints, which in turn should drop any deferred
+                    // constraint memory of them. Renaming of a table with
+                    // constrants is not presently allowed. FIXME: use UUID
+                    // instead of string here, more stable reference.
+                    SanityManager.NOTREACHED();
+                }
+            } else {
+                for (UUID id : culprits) {
+                    if (constraintId == null || constraintId.equals(id)) {
+                        final ConstraintDescriptor cd =
+                                dd.getConstraintDescriptor(id);
+                        final StringBuilder checkStmt = new StringBuilder();
+                        checkStmt.append("SELECT 1 FROM ");
+                        checkStmt.append(td.getQualifiedName());
+
+                        // If a compress has happened in this transaction, we
+                        // can't trust the rowLocations, so make a full table
+                        // scan. If not, we optimize by using a special result
+                        // set type which utilized the saved away row locations
+                        // for the offending rows, so we only visit those when
+                        // checking.  I.e. other rows are known to be good a
+                        // priori.
+                        if (!isInvalidated()) {
+                            checkStmt.append(
+                               " --DERBY-PROPERTIES joinStrategy=nestedLoop, " +
+                               "                    index=null, " +
+                               "                    validateCheckConstraint=");
+                            checkStmt.append(Long.toString(baseTableCID));
+                            checkStmt.append('\n');
+                        }
+
+                        checkStmt.append(" WHERE NOT(");
+                        checkStmt.append(cd.getConstraintText());
+                        checkStmt.append(')');
+
+                        BasicNoPutResultSetImpl rs = null;
+                        final PreparedStatement ps =
+                            lcc.prepareInternalStatement(
+                                lcc.getDefaultSchema(),
+                                checkStmt.toString(),
+                                true,
+                                true);
+                        try {
+                            rs = (BasicNoPutResultSetImpl)ps.execute(
+                                    ps.getActivation(lcc, false), false, 0L);
+                            final ExecRow row = rs.getNextRowCore();
+
+                            if (row != null) {
+                                //check constraint violated
+
+                                throw StandardException.newException(
+                                   rollbackOnError ?
+                                     SQLState.LANG_DEFERRED_CHECK_CONSTRAINT_T :
+                                     SQLState.LANG_DEFERRED_CHECK_CONSTRAINT_S,
+                                   cd.getConstraintName(),
+                                   td.getQualifiedName(),
+                                   cd.getConstraintText());
+                            }
+                        } finally {
+                            if (rs != null) {
+                                try {
+                                    rs.close();
+                                } catch (StandardException e) {}
+                            }
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    private static class ForeignKeyInfo extends ValidationInfo {
+        /**
+         * the UUID of the foreign constraint
+         */
+        private final UUID fkId;
+
+        /**
+         * The conglomerate id of the index of the referenced table
+         */
+        private long rtCID;
+
+        final private String schemaName;
+        final private String tableName;
+
+        public ForeignKeyInfo(
+                final BackingStoreHashtable infoRows,
+                UUID fkId,
+                long rtCID,
+                String schemaName,
+                String tableName) {
+
+            super(infoRows);
+            this.fkId = fkId;
+            this.rtCID = rtCID;
+            this.tableName = tableName;
+            this.schemaName = schemaName;
+        }
+
+        public UUID getFkId() {
+            return fkId;
+        }
+
+        public void updateRtCID(long rtCID) {
+            this.rtCID = rtCID;
+        }
+
+        public long getRtCID() {
+            return rtCID;
+        }
+
+        public void possiblyValidateOnReturn(
+                Map.Entry<Long, ValidationInfo> e,
+                LanguageConnectionContext lcc,
+                SQLSessionContext nested,
+                SQLSessionContext caller) throws StandardException {
+
+            if (lcc.isEffectivelyDeferred(caller, getFkId())) {
+                // the constraint is also deferred in the calling context
+                return;
+            }
+
+            final long indexCID = e.getKey().longValue();
+            validateForeignKey(lcc, indexCID, true);
+        }
+
+        public final void validateConstraint(
+                LanguageConnectionContext lcc,
+                long conglomerateId,
+                UUID constraintId,
+                boolean rollbackOnError) throws StandardException {
+
+            validateForeignKey(lcc, conglomerateId, rollbackOnError);
+
+        }
+
+        private void validateForeignKey(
+            LanguageConnectionContext lcc,
+            long indexCID,
+            boolean rollbackOnError) throws StandardException {
+
+            // First check if the offending row is still present,
+            // if so, check that the referenced row exists. To do this we open
+            // two index scans below.
+
+            TransactionController tc = lcc.getTransactionExecute();
+
+            final Enumeration<?> e = infoRows.elements();
+
+            while (e.hasMoreElements()) {
+                final DataValueDescriptor[] key =
+                        (DataValueDescriptor[])e.nextElement();
+
+                // FIXME: This is not very efficient: we could sort the rows in
+                // the hash table, and then check all rows using a single scan.
+                ScanController indexSC = null;
+                boolean violation = false;
+
+                long[] cids = new long[]{indexCID, getRtCID()};
+
+                for (int idx = 0; idx < 2; idx++) {
+                    boolean sawException = false;
+
+                    try {
+                        indexSC = tc.openScan(
+                                cids[idx],
+                                false,
+                                0, // read only
+                                TransactionController.MODE_RECORD,
+                                TransactionController.ISOLATION_READ_COMMITTED,
+                                (FormatableBitSet)null, // retrieve all fields
+                                key,
+                                ScanController.GE, // startSearchOp
+                                null,
+                                key,
+                                ScanController.GT);
+
+                        if (idx == 0) {
+                            if (indexSC.next()) {
+                                // The row with the PK still exists, so we need
+                                // to check the referenced table's index
+                            } else {
+                                // No rows contain key: OK, must have been
+                                // deleted later in transaction, or we got here
+                                // due to pessimistic assumption on a timeout
+                                // while checking on the insert.  In any case,
+                                // no need to check the referenced key, so
+                                // leave.
+                                break;
+                            }
+                        } else {
+                            if (indexSC.next()) {
+                                // We found the referenced key, all is good
+                            } else {
+                                // We didn't find it and we know it is present
+                                // as a PK, so we have a violation.
+                                violation = true;
+                            }
+                        }
+                    } catch (StandardException se) {
+                        sawException = true;
+                        throw se;
+                    } finally {
+                        // Clean up resource usage
+                        try {
+                            if (indexSC != null) {
+                                indexSC.close();
+                            }
+                        } catch (StandardException ie) {
+                            if (!sawException) {
+                                throw ie;
+                            } // else: can't let it shadow preceding exception
+                        }
+                    }
+                }
+
+                if (violation) {
+                    final DataDictionary dd = lcc.getDataDictionary();
+
+                    final SchemaDescriptor sd =
+                            dd.getSchemaDescriptor(schemaName, tc, true);
+
+                    final TableDescriptor td  =
+                            dd.getTableDescriptor(tableName, sd, tc);
+
+                    final ForeignKeyConstraintDescriptor cd =
+                            (ForeignKeyConstraintDescriptor)dd.
+                                    getConstraintDescriptor(getFkId());
+
+                    final ConstraintDescriptor rcd = dd.getConstraintDescriptor(
+                            cd.getReferencedConstraintId());
+
+                    final TableDescriptor rtd = rcd.getTableDescriptor();
+
+                    throw StandardException.newException(
+                            rollbackOnError ?
+                                    SQLState.LANG_DEFERRED_FK_CONSTRAINT_T :
+                                    SQLState.LANG_DEFERRED_FK_CONSTRAINT_S,
+                            cd.getConstraintName(),
+                            td.getQualifiedName(),
+                            rcd.getConstraintName(),
+                            rtd.getQualifiedName(),
+                            RowUtil.toString(key));
+                }
+            }
+        }
     }
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeleteResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeleteResultSet.java?rev=1590849&r1=1590848&r2=1590849&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeleteResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeleteResultSet.java Tue Apr 29 00:23:52 2014
@@ -313,7 +313,7 @@ class DeleteResultSet extends DMLWriteRe
 		{
 			if (fkChecker == null)
 			{
-				fkChecker = new RISetChecker(tc, fkInfoArray);
+                fkChecker = new RISetChecker(lcc, tc, fkInfoArray);
 			}
 			else
 			{
@@ -389,7 +389,7 @@ class DeleteResultSet extends DMLWriteRe
 			{
 				if (fkChecker != null)
 				{
-					fkChecker.doPKCheck(row, false);
+                    fkChecker.doPKCheck(activation, row, false);
 				}
 
 				baseRowLocation = 
@@ -556,7 +556,8 @@ class DeleteResultSet extends DMLWriteRe
                 ExecRow defRLRow;
                 while ((defRLRow = rs.getNextRow()) != null)
 				{
-                    fkChecker.doPKCheck(defRLRow, restrictCheckOnly);
+                    fkChecker.doPKCheck(
+                        activation, defRLRow, restrictCheckOnly);
 				}
 			} finally
 			{