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 2013/11/25 21:30:46 UTC

svn commit: r1545394 [1/3] - in /db/derby/code/trunk/java: engine/org/apache/derby/catalog/ engine/org/apache/derby/catalog/types/ engine/org/apache/derby/iapi/sql/conn/ engine/org/apache/derby/iapi/sql/dictionary/ engine/org/apache/derby/iapi/store/ac...

Author: dag
Date: Mon Nov 25 20:30:45 2013
New Revision: 1545394

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

Patch derby-532-post-scan-4 implements basic support for deferred
constraints for PRIMARY KEY and UNIQUE constraints. Deferrable
constraints are not enabled by default yet; one needs to set a
property to try the feature: "derby.constraintsTesting".

This patch enables deferred constraints for:

    a) primary key constraints
    b) unique constraint with not nullable columns
    c) unique constraint with nullable columns

by new logic in insertion and sorts.

The patch includes relaxing the constraint at insertion and update
time, as well as adding a constraint to an existing table. 

Derby treats constraints a) and b) the same, and in the code these are
marked as "unique" when they are not deferrable (as in existing code).

Constraint type c) is currently marked as
"uniqueWithDuplicateNulls". Insert/update of these is implemented in
the BTree by including the RowLocation of the base row in the set of
keys in the index row (DERBY-3330). This makes them trivially unique,
but there is an extra code path in BTreeController that checks
neighbor rows for duplicates, and only allows insertion if the key
contains a null. When adding a constraint to an existing table, these
are handled by a specially crafted sorter
(UniqueWithDuplicateNullsMergeSort).

The implementation of insert/update of deferrable indexes is based on
a similar approach, i.e. by backing with a non-unique index, and checking 
duplicates in the language layer, notably IndexChanger.

In IndexChanger, after inserting a row, we check if it is unique by
performing a scan of the BTree. A time-out here leads to a pessimistic
assumption that there is a duplicate. Duplicate key values are saved
until checking time (usually commit time), when a new scan is
performed to validate the uniqueness property.

[This means a) and b) if deferrable are no longer marked "unique"].

Deferrable indexes are not shared.

If there are duplicates and we have deferred constraint mode (a
dynamic session property), we save the duplicate index row in a disk
based hash table (DeferredDuplicates#rememberDuplicate).

For a) and b), constraints which are deferrable are marked as
"uniqueDeferrable" and "hasDeferrableChecking". Constraints of type c)
which are deferrable are marked "uniqueWithDuplicateNulls" and
"hasDeferrableChecking". These marks determines the code paths
used. Note that existing indexes and non-deferrable constraint do not
get a new code path, which should preserve correctness and performance
of those.

Now, with these markers in place, deferral of checks happens in three
places:

    {{ IndexChanger#insertAndCheckDups}}

    {{CreateIndexConstantAction#executeConstantAction +
     MergeSort#compare and UniqueWithDuplicateNullsMergeSort#compare }}

    {{InsertResultSet#setUpAllSorts}}


The former is active for deferral under INSERT and UPDATE. The middle
when adding a deferrable constraint to an existing table, when we sort
existing rows detecting any duplicates. The last is used when importing
rows.

At transaction commit (1), or when the constraint mode for a deferred
constraint is changed back to immediate (2), we validate the
constraint (DeferredDuplicates#validate) by replaying the hash table
and scanning the index for the duplicate index rows to ascertain there
are none, or else we have an error: transaction or statement severity
respectively for (1) and (2).

The constraint mode is a SQL session level variable, and inside
routines (nested connections), we push this on the stack. This means
change of the constraint mode inside nested connections will be popped
on routine exit. If, as part of this, a constraint changes from
deferred to immediate mode, we also validate it for correctness. If
this fail, the transaction rolls back
We needed to do this from a newly introduced method,
GenericLanguageConnectionContext#popNestedSessionContext. This
pops the SQL session context.
That hook is called from GenericPreparedStatement#executeStmt. As a
part of this effort, we also renamed #setupNestedSessionContext to
#pushNestedSessionContext.

The patch also adds support for checking deferred constraints in
xa_prepare and xa_commit (.., true), cf. specification attached to the
JIRA issue.

Concurrency: if a transaction gets a lock time-out when trying to
establish if a row just inserted is a duplicate (another transaction
may have just inserted a row with a similar index key), we use a
pessimistics assumption and add that key to the set of keys to be
checked at commit time. If a key can't be grabbed then, a time-out is
thrown. We plan to add an optimized scan to avoid waiting for the lock
at insertion time, cf DERBY-6419.

The "not enforced" feature is not yet implemented in this patch.

Several new test cases been added to ConstraintCharacteristicsTest to
test these basic behaviors.


Added:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeferredDuplicates.java
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/catalog/IndexDescriptor.java
    db/derby/code/trunk/java/engine/org/apache/derby/catalog/types/IndexDescriptorImpl.java
    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/conn/SQLSessionContext.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/StatementContext.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/IndexRowGenerator.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/SortObserver.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedConnection.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/load/Import.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericPreparedStatement.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/TabInfoImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/CreateIndexNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SetConstraintsNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/StaticMethodCallNode.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/compile/sqlgrammar.jj
    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/conn/SQLSessionContextImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterConstraintConstantAction.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/BaseActivation.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BasicSortObserver.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/CreateIndexConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DDLSingleTableConstantAction.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/GenericConstantActionFactory.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/MaterializedResultSet.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/RowChangerImpl.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/TemporaryRowHolderImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UniqueIndexSortObserver.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UniqueWithDuplicateNullsIndexSortObserver.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/sort/MergeSort.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/sort/UniqueWithDuplicateNullsMergeSort.java
    db/derby/code/trunk/java/engine/org/apache/derby/jdbc/EmbedXAResource.java
    db/derby/code/trunk/java/engine/org/apache/derby/jdbc/XATransactionState.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/LockTableTest.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/junit/SupportFilesSetup.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_SortController.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/catalog/IndexDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/catalog/IndexDescriptor.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/catalog/IndexDescriptor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/catalog/IndexDescriptor.java Mon Nov 25 20:30:45 2013
@@ -30,6 +30,59 @@ package org.apache.derby.catalog;
  * the table on which the index is defined.
  * That information is available 
  * in the columns NAME and TABLEID of the table SYS.SYSCONGLOMERATES.
+ * <p>
+ * Whereas non-deferrable constraints are backed by UNIQUE indexes,
+ * deferrable constraints are backed by non-unique indexes. The duplicate
+ * checking on inserts and updates for deferrable constraints are handled at
+ * the language level, not by the store level. The following table shows
+ * the correspondence between the constraint types and the index attributes
+ * used:
+ * <ul>
+ *  <li>Non-deferrable PRIMARY KEY and UNIQUE NOT NULL on all constraint
+ *     columns
+ *  <pre>
+ *                            \  Value  | Number of index columns | Check
+ *   Attribute                 \        | in physical BTree key   | in
+ *   --------------------------------------------------------------------
+ *   unique                     | true  | N - 1 (row location     |
+ *   isUniqueWithDuplicateNulls | false |        not part of key) | Store
+ *   uniqueDeferrable           | false |                         | Btree
+ *   hasDeferrableChecking      | false |                         | code
+ *  </pre>
+ *  <li>Non-deferrable UNIQUE, where at least one constraint column is
+ *      nullable.
+ *  <pre>
+ *                            \  Value  | Number of index columns | Check
+ *   Attribute                 \        | in physical BTree key   | in
+ *   ------------------------------------------------------------ -------
+ *   unique                     | false | N                       |
+ *   isUniqueWithDuplicateNulls | true  |                         | Store
+ *   uniqueDeferrable           | false |                         | Btree
+ *   hasDeferrableChecking      | false |                         | code
+ *  </pre>
+ *  <li>Deferrable PRIMARY KEY and UNIQUE NOT NULL on all constraint
+ *     columns
+ *  <pre>
+ *                            \  Value  | Number of index columns | Check
+ *   Attribute                 \        | in physical BTree key   | in
+ *   ------------------------------------------------------------ -------
+ *   unique                     | false | N                       |
+ *   isUniqueWithDuplicateNulls | false |                         | Lang.
+ *   uniqueDeferrable           | true  |                         | code
+ *   hasDeferrableChecking      | true  |                         |
+ *  </pre>
+ *  <li>Deferrable UNIQUE, where at least one constraint column is
+ *      nullable.
+ *  <pre>
+ *                            \  Value  | Number of index columns | Check
+ *   Attribute                 \        | in physical BTree key   | in
+ *   ------------------------------------------------------------ -------
+ *   unique                     | false | N                       |
+ *   isUniqueWithDuplicateNulls | true  |                         | Lang.
+ *   uniqueDeferrable           | false |                         | code
+ *   hasDeferrableChecking      | true  |                         |
+ *  </pre>
+ *  </ul>
  */
 public interface IndexDescriptor
 {
@@ -43,6 +96,22 @@ public interface IndexDescriptor
 	 */
 	boolean			isUniqueWithDuplicateNulls();
 
+    /**
+     * The index represents a PRIMARY KEY or a UNIQUE NOT NULL constraint which
+     * is deferrable.
+     * {@code true} implies {@code isUnique() == false} and
+     * {@code isUniqueWithDuplicateNulls() == false} and
+     * {@code hasDeferrableChecking() == true}.
+
+     * @return {@code true} if the index represents such a constraint
+     */
+    boolean isUniqueDeferrable();
+
+    /**
+     * Returns true if the index is used to support a deferrable constraint.
+     */
+    boolean hasDeferrableChecking();
+
 	/**
 	 * Returns an array of column positions in the base table.  Each index
 	 * column corresponds to a column position in the base table, except

Modified: db/derby/code/trunk/java/engine/org/apache/derby/catalog/types/IndexDescriptorImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/catalog/types/IndexDescriptorImpl.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/catalog/types/IndexDescriptorImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/catalog/types/IndexDescriptorImpl.java Mon Nov 25 20:30:45 2013
@@ -37,7 +37,14 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.io.IOException;
 
-/** @see org.apache.derby.iapi.sql.dictionary.IndexRowGenerator */
+/**
+ * See also {@link org.apache.derby.iapi.sql.dictionary.IndexRowGenerator}.
+ * <p>
+ * For a description of how deferrable and non-deferrable constraints
+ * are backed differently, including the meaning of the
+ * boolean attributes used here, see {@link
+ * org.apache.derby.catalog.IndexDescriptor}.
+ */
 public class IndexDescriptorImpl implements IndexDescriptor, Formatable
 {
 	/********************************************************
@@ -48,7 +55,7 @@ public class IndexDescriptorImpl impleme
 	**	also write/read them with the writeExternal()/readExternal()
 	**	methods.
 	**
-	**	If, inbetween releases, you add more fields to this class,
+    **  If, in between releases, you add more fields to this class,
 	**	then you should bump the version number emitted by the getTypeFormatId()
 	**	method.
 	**
@@ -66,6 +73,20 @@ public class IndexDescriptorImpl impleme
     //will act like a unique index.
 	private boolean     isUniqueWithDuplicateNulls;
 
+    /**
+     * The index represents a PRIMARY KEY or a UNIQUE NOT NULL constraint which
+     * is deferrable.
+     * {@code true} implies {@code isUnique == false} and
+     * {@code isUniqueWithDuplicateNulls == false} and
+     * {@code hasDeferrableChecking == true}.
+     */
+    private boolean     isUniqueDeferrable;
+
+    /**
+     * The index represents a constraint which is deferrable.
+     */
+    private boolean     hasDeferrableChecking;
+
 	/**
      * Constructor for an IndexDescriptorImpl
      * 
@@ -79,6 +100,11 @@ public class IndexDescriptorImpl impleme
      *                              isUniqueWithDuplicateNulls is set to true the
      *                              index will allow duplicate nulls but for
      *                              non null keys will act like a unique index.
+     * @param isUniqueDeferrable    True means the index represents a PRIMARY
+     *                              KEY or a UNIQUE NOT NULL constraint which
+     *                              is deferrable.
+     * @param hasDeferrableChecking True if this index supports a deferrable
+     *                              constraint.
      * @param baseColumnPositions	An array of column positions in the base
      * 								table.  Each index column corresponds to a
      * 								column position in the base table.
@@ -92,6 +118,8 @@ public class IndexDescriptorImpl impleme
 	public IndexDescriptorImpl(String indexType,
 								boolean isUnique,
 								boolean isUniqueWithDuplicateNulls,
+                                boolean isUniqueDeferrable,
+                                boolean hasDeferrableChecking,
 								int[] baseColumnPositions,
 								boolean[] isAscending,
 								int numberOfOrderedColumns)
@@ -99,6 +127,8 @@ public class IndexDescriptorImpl impleme
 		this.indexType = indexType;
 		this.isUnique = isUnique;
 		this.isUniqueWithDuplicateNulls = isUniqueWithDuplicateNulls;
+        this.isUniqueDeferrable = isUniqueDeferrable;
+        this.hasDeferrableChecking = hasDeferrableChecking;
 		this.baseColumnPositions = ArrayUtil.copy( baseColumnPositions );
 		this.isAscending = ArrayUtil.copy( isAscending );
 		this.numberOfOrderedColumns = numberOfOrderedColumns;
@@ -119,6 +149,29 @@ public class IndexDescriptorImpl impleme
 		return isUniqueWithDuplicateNulls;
 	}
 
+    /**
+     *
+     * @return  {@code true} is the index supports a deferrable constraint
+     */
+    public boolean hasDeferrableChecking()
+    {
+        return hasDeferrableChecking;
+    }
+
+    /**
+     * The index represents a PRIMARY KEY or a UNIQUE NOT NULL constraint which
+     * is deferrable.
+     * {@code true} implies {@code #isUnique() == false} and
+     * {@code #isUniqueWithDuplicateNulls() == false} and
+     * {@code #hasDeferrableChecking() == true}.
+     *
+     * @return {@code true} is the index supports such a constraint
+     */
+    public boolean isUniqueDeferrable()
+    {
+        return isUniqueDeferrable;
+    }
+
 	/** @see IndexDescriptor#isUnique */
 	public boolean isUnique()
 	{
@@ -208,10 +261,14 @@ public class IndexDescriptorImpl impleme
 	{
 		StringBuffer	sb = new StringBuffer(60);
 
-		if (isUnique)
+        if (isUnique || isUniqueDeferrable)
 			sb.append("UNIQUE ");
 		else if (isUniqueWithDuplicateNulls)
-			sb.append ("UNIQUE WITH DUPLICATE NULLS");
+            sb.append ("UNIQUE WITH DUPLICATE NULLS ");
+
+        if (hasDeferrableChecking) {
+            sb.append(" DEFERRABLE CHECKING ");
+        }
 
 		sb.append(indexType);
 
@@ -260,6 +317,22 @@ public class IndexDescriptorImpl impleme
                                     "isUniqueWithDuplicateNulls");
 		else
 			isUniqueWithDuplicateNulls = false;
+
+        // hasDeferrableChecking won't be present if the index
+        // was created in old versions (< 10_11).
+        if (fh.containsKey("hasDeferrableChecking")) {
+            hasDeferrableChecking = fh.getBoolean("hasDeferrableChecking");
+        } else {
+            hasDeferrableChecking = false;
+        }
+
+        // isUniqueDeferrable won't be present if the index
+        // was created in old versions (< 10_11).
+        if (fh.containsKey("isUniqueDeferrable")) {
+            isUniqueDeferrable = fh.getBoolean("isUniqueDeferrable");
+        } else {
+            isUniqueDeferrable = false;
+        }
 	}
 
 	/**
@@ -282,6 +355,8 @@ public class IndexDescriptorImpl impleme
 		//write the new attribut older versions will simply ignore it
 		fh.putBoolean("isUniqueWithDuplicateNulls", 
                                         isUniqueWithDuplicateNulls);
+        fh.putBoolean("hasDeferrableChecking", hasDeferrableChecking);
+        fh.putBoolean("isUniqueDeferrable", isUniqueDeferrable);
         out.writeObject(fh);
 	}
 

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=1545394&r1=1545393&r2=1545394&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 Mon Nov 25 20:30:45 2013
@@ -21,37 +21,33 @@
 
 package org.apache.derby.iapi.sql.conn;
 
-import org.apache.derby.iapi.services.context.Context;
-import org.apache.derby.iapi.services.io.FormatableBitSet;
-import org.apache.derby.iapi.sql.compile.OptTrace;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.derby.catalog.UUID;
 import org.apache.derby.iapi.db.Database;
+import org.apache.derby.iapi.db.TriggerExecutionContext;
 import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.services.context.Context;
+import org.apache.derby.iapi.services.io.FormatableBitSet;
+import org.apache.derby.iapi.sql.Activation;
+import org.apache.derby.iapi.sql.LanguageFactory;
+import org.apache.derby.iapi.sql.ParameterValueSet;
+import org.apache.derby.iapi.sql.PreparedStatement;
+import org.apache.derby.iapi.sql.compile.ASTVisitor;
 import org.apache.derby.iapi.sql.compile.CompilerContext;
-import org.apache.derby.iapi.sql.dictionary.DataDictionary;
-import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
-import org.apache.derby.iapi.sql.dictionary.RoleGrantDescriptor;
-import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
+import org.apache.derby.iapi.sql.compile.OptTrace;
 import org.apache.derby.iapi.sql.compile.OptimizerFactory;
-import org.apache.derby.iapi.types.DataValueFactory;
-
-import org.apache.derby.iapi.sql.compile.ASTVisitor;
 import org.apache.derby.iapi.sql.depend.Provider;
+import org.apache.derby.iapi.sql.dictionary.DataDictionary;
+import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
+import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.execute.ConstantAction;
 import org.apache.derby.iapi.sql.execute.CursorActivation;
-import org.apache.derby.iapi.sql.execute.ExecutionContext;
 import org.apache.derby.iapi.sql.execute.ExecutionStmtValidator;
-import org.apache.derby.iapi.sql.Activation;
-import org.apache.derby.iapi.sql.LanguageFactory;
-import org.apache.derby.iapi.sql.PreparedStatement;
-import org.apache.derby.iapi.sql.ParameterValueSet;
-
-import org.apache.derby.iapi.store.access.TransactionController;
-import org.apache.derby.iapi.db.TriggerExecutionContext;
 import org.apache.derby.iapi.sql.execute.RunTimeStatistics;
-import org.apache.derby.catalog.UUID;
-
-import java.util.Map;
-import java.util.AbstractMap;
+import org.apache.derby.iapi.store.access.BackingStoreHashtable;
+import org.apache.derby.iapi.store.access.TransactionController;
+import org.apache.derby.iapi.types.DataValueFactory;
 
 /**
  * LanguageConnectionContext keeps the result sets,
@@ -1118,32 +1114,43 @@ public interface LanguageConnectionConte
 	 * section 4.27.3, since this gives rise to a nested connection.
 	 * <p>
 	 * Called from generated code, see
-	 * {@code StaticMethodCallNode#generateSetupNestedSessionContext}.
+     * {@link org.apache.derby.impl.sql.compile.StaticMethodCallNode#generatePushNestedSessionContext}.
 	 * <p>
 	 * The new SQL session context is also set in the current statement
 	 * context (of the invocation).
 	 *
-	 * @see org.apache.derby.impl.sql.compile.StaticMethodCallNode#generateSetupNestedSessionContext
+     * @see org.apache.derby.impl.sql.compile.StaticMethodCallNode#generatePushNestedSessionContext
 	 * @see StatementContext#getSQLSessionContext
 	 * @see #setupSubStatementSessionContext
 	 *
 	 * @param a activation of the statement which performs the call.
      * @param definersRights if the method should run with definer's rights
      * @param definer authorization id of the definer
+     * @throws StandardException standard error policy
 	 */
-    public void setupNestedSessionContext(Activation a,
-                                          boolean definersRights,
-                                          String definer)
+    public void pushNestedSessionContext(Activation a,
+                                        boolean definersRights,
+                                        String definer)
             throws StandardException;
 
+    /**
+     * If returning from a routine that can execute SQL, perform any
+     * actions needed when popping the SQL session context.
+     *
+     * @param a activation
+     * @throws StandardException standard error policy
+     */
+    public void popNestedSessionContext(Activation a) throws StandardException;
+
 	/**
 	 * Get the value of top level session context of the top level connection.
+     * @return the requested session context
 	 */
 	public SQLSessionContext getTopLevelSQLSessionContext();
 
 	/**
 	 * Used when a statement as part of its operation executes an other
-	 * statement. In contrast to setupNestedSessionContext, the activation (for
+     * statement. In contrast to pushNestedSessionContext, the activation (for
 	 * the substatement) just inherits the current session context from the
 	 * parent statements activation, it does <b>not</b> push a new copy on the
 	 * stack of session contexts.
@@ -1160,7 +1167,7 @@ public interface LanguageConnectionConte
 	 *         and {@code deleteRow}.
 	 *     <li>During trigger body execution.
 	 * </ul>
-	 * @see #setupNestedSessionContext
+     * @see #pushNestedSessionContext
 	 */
     public void setupSubStatementSessionContext(Activation a)
             throws StandardException;
@@ -1268,4 +1275,70 @@ public interface LanguageConnectionConte
      */
     public void setReferencedColumnMap(TableDescriptor td,
                                        FormatableBitSet map);
+
+    /**
+     * Set the constraint mode for this constraint/index to {@code deferred}.
+     * If {@code deferred} is {@code false}, to immediate checking,
+     * if {@code true} to deferred checking.
+     *
+     * @param a         Activation
+     * @param conglomId The conglomerate id of the backing index for the
+     *                  constraint .
+     * @param deferred  The new constraint mode
+     * @throws StandardException
+     */
+    public void setDeferred(Activation a, long conglomId, boolean deferred)
+            throws StandardException;
+
+    /**
+     * Get the constraint mode set, if any.
+     *
+     * @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
+     */
+    public boolean isEffectivelyDeferred(Activation a, long conglomId)
+            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.
+     *
+     * @param a        Activation
+     * @param deferred The new constraint mode
+     */
+    public void setDeferredAll(Activation a, boolean deferred)
+            throws StandardException;
+
+    /**
+     * Get the set of disk backed hash tables containing any index rows
+     * saved for deferred constraints in this transaction, keyed by the
+     * conglomerate id.
+     * @return the set
+     */
+    HashMap<Long, BackingStoreHashtable> getDeferredHashTables();
+
+    /**
+     * Check that deferred constraints are valid, if not roll back the
+     * transaction.
+     *
+     * @throws StandardException
+     */
+    public void checkIntegrity() throws StandardException;
+
+    /**
+     * Forget any violating rows for the deferred constraint backed by
+     * {@code indexCID}, if any. Typically used when an index gets dropped
+     * and/or recreated.
+     *
+     * @param indexCID The conglomerate identifier of the backing index
+     * @throws StandardException
+     */
+    public void invalidateDeferredConstraintsData(long indexCID)
+            throws StandardException;
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/SQLSessionContext.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/SQLSessionContext.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/SQLSessionContext.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/SQLSessionContext.java Mon Nov 25 20:30:45 2013
@@ -22,6 +22,7 @@
 package org.apache.derby.iapi.sql.conn;
 
 import java.lang.String;
+import java.util.HashMap;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
 
 /**
@@ -63,7 +64,7 @@ import org.apache.derby.iapi.sql.diction
  * context. Since the same dynamic call context is involved, this
  * seems correct.
  *
- * @see org.apache.derby.iapi.sql.conn.LanguageConnectionContext#setupNestedSessionContext
+ * @see LanguageConnectionContext#pushNestedSessionContext
  */
 
 public interface SQLSessionContext {
@@ -97,4 +98,67 @@ public interface SQLSessionContext {
      * Get the schema of this SQL connection context
      */
     public SchemaDescriptor getDefaultSchema();
+
+    /**
+     * Get a handle to the session's constraint modes.
+     * The caller is responsible for any cloning needed.
+     * @return constraint modes map
+     */
+    public HashMap<Long, Boolean> getConstraintModes();
+
+    /**
+     * Initialize a inferior session context with the constraint mode map
+     * of the parent session context.
+     * @param hm constraint mode map
+     */
+    public void setConstraintModes(HashMap<Long, Boolean> hm);
+
+    /**
+     * Set the constraint mode for this constraint/index to {@code deferred}.
+     * If {@code deferred} is {@code false}, to immediate checking,
+     * if {@code true} to deferred checking.
+     *
+     * @param conglomId The conglomerate id of the backing index for the
+     *                  constraint .
+     * @param deferred  The new constraint mode
+     */
+    public void setDeferred(long conglomId, boolean deferred);
+
+    /**
+     * Return {@code Boolean.TRUE} if the constraint mode for this
+     * constraint/index has been set to deferred, {@code Boolean.FALSE} if
+     * it has been set to immediate.  Any ALL setting is considered also.
+     * If the constraint mode hasn't been set for this constraint,
+     * return {@code null}. The constraint mode is the effectively the initial
+     * constraint mode in this case.
+     */
+    public Boolean isDeferred(long conglomId);
+
+    /**
+     * Clear deferred information for this transaction.
+     */
+    public void resetConstraintModes();
+
+    /**
+     * Set the constraint mode for all deferrable constraints to
+     * {@code deferred}.
+     * If {@code deferred} is {@code false}, set to immediate checking,
+     * if {@code true} to deferred checking.
+     * {@code null} is allowed: it means no ALL setting exists.
+     *
+     * @param deferred the mode to set
+     */
+    public void setDeferredAll(Boolean deferred);
+
+    /**
+     * Get state of DEFERRED ALL setting.
+     *
+     * @return {@code True} is deferred all constraint mode has been
+     *         set for this session context.
+     *         {@code False} is deferred immediate has been set for this
+     *         session context.
+     *         {@code null} means no ALL setting has been made for this context
+     */
+    public Boolean getDeferredAll();
+
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/StatementContext.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/StatementContext.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/StatementContext.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/StatementContext.java Mon Nov 25 20:30:45 2013
@@ -261,7 +261,8 @@ public interface StatementContext extend
 
 
 	/**
-	 * Get the current SQL session context
+     * Get the current SQL session context.
+     * @return the current SQL session context
 	 */
 	public SQLSessionContext getSQLSessionContext();
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java Mon Nov 25 20:30:45 2013
@@ -715,24 +715,21 @@ public abstract class ConstraintDescript
             // Bug 4307
             // We need to get the conglomerate descriptors from the 
             // dd in case we dropped other constraints in a cascade operation. 
-             ConglomerateDescriptor[]conglomDescs =
-                 dd.getConglomerateDescriptors(getConglomerateId());
+            ConglomerateDescriptor[]conglomDescs =
+                    dd.getConglomerateDescriptors(getConglomerateId());
 
-            if (conglomDescs.length != 0)
-            {
-                // Typically there is only one ConglomerateDescriptor
-                // for a given UUID, but due to an old bug
-                // there may be more than one. If there is more
-                // than one then which one is remvoed does not
-                // matter since they will all have the same critical
-                // information since they point to the same physical index.
-                for (int i = 0; i < conglomDescs.length; i++)
-                {
-                    if (conglomDescs[i].isConstraint())
-                    {
-                        newBackingConglomCD = conglomDescs[i].drop(lcc, table);
-                        break;
-                    }
+            // Typically there is only one ConglomerateDescriptor
+            // for a given UUID, but due to an old bug
+            // there may be more than one. If there is more
+            // than one then which one is remvoed does not
+            // matter since they will all have the same critical
+            // information since they point to the same physical index.
+            for (ConglomerateDescriptor cd : conglomDescs) {
+                if (cd.isConstraint()) {
+                    lcc.invalidateDeferredConstraintsData(
+                            cd.getConglomerateNumber());
+                    newBackingConglomCD = cd.drop(lcc, table);
+                    break;
                 }
             }
         }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/IndexRowGenerator.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/IndexRowGenerator.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/IndexRowGenerator.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/IndexRowGenerator.java Mon Nov 25 20:30:45 2013
@@ -44,48 +44,17 @@ import org.apache.derby.iapi.types.DataT
 /**
  * This class extends IndexDescriptor for internal use by the
  * DataDictionary.
+ * <p>
+ * For a description of how deferrable and non-deferrable constraints
+ * are backed differently, including the meaning of the
+ * boolean attributes used here, see {@link
+ * org.apache.derby.catalog.IndexDescriptor}.
  */
 public class IndexRowGenerator implements IndexDescriptor, Formatable
 {
 	private IndexDescriptor	id;
 	private ExecutionFactory ef;
 
-	/**
-	 * Constructor for an IndexRowGeneratorImpl
-	 *
-	 * @param indexType		The type of index
-	 * @param isUnique		True means the index is unique
-	 * @param baseColumnPositions	An array of column positions in the base
-	 *								table.  Each index column corresponds to a
-	 *								column position in the base table.
-	 * @param isAscending	An array of booleans telling asc/desc on each
-	 *						column.
-	 * @param numberOfOrderedColumns	In the future, it will be possible
-	 *									to store non-ordered columns in an
-	 *									index.  These will be useful for
-	 *									covered queries.
-	 */
-	public IndexRowGenerator(String indexType,
-								boolean isUnique,
-								int[] baseColumnPositions,
-								boolean[] isAscending,
-								int numberOfOrderedColumns)
-	{
-		id = new IndexDescriptorImpl(
-                        indexType,
-                        isUnique, //default uniqueWithDuplicateNulls to false
-                        false,
-                        baseColumnPositions,
-                        isAscending,
-                        numberOfOrderedColumns);
-
-		if (SanityManager.DEBUG)
-		{
-			SanityManager.ASSERT(baseColumnPositions != null,
-				"baseColumnPositions are null");
-		}
-	}
-        
     /**
      * Constructor for an IndexRowGeneratorImpl
      * 
@@ -93,6 +62,11 @@ public class IndexRowGenerator implement
      * @param isUnique		True means the index is unique
      * @param isUniqueWithDuplicateNulls means the index is almost unique
      *                              i.e. unique only for non null keys
+     * @param isUniqueDeferrable    True means the index represents a PRIMARY
+     *                              KEY or a UNIQUE NOT NULL constraint which
+     *                              is deferrable.
+     * @param hasDeferrableChecking True if the index is used to back a
+     *                              deferrable constraint
      * @param baseColumnPositions	An array of column positions in the base
      * 								table.  Each index column corresponds to a
      * 								column position in the base table.
@@ -106,6 +80,8 @@ public class IndexRowGenerator implement
 	public IndexRowGenerator(String indexType,
 								boolean isUnique,
 								boolean isUniqueWithDuplicateNulls,
+                                boolean isUniqueDeferrable,
+                                boolean hasDeferrableChecking,
 								int[] baseColumnPositions,
 								boolean[] isAscending,
 								int numberOfOrderedColumns)
@@ -113,6 +89,8 @@ public class IndexRowGenerator implement
 		id = new IndexDescriptorImpl(indexType,
 									isUnique,
 									isUniqueWithDuplicateNulls,
+                                    isUniqueDeferrable,
+                                    hasDeferrableChecking,
 									baseColumnPositions,
 									isAscending,
 									numberOfOrderedColumns);
@@ -304,6 +282,19 @@ public class IndexRowGenerator implement
 	{
 		return id.isUniqueWithDuplicateNulls();
 	}
+
+    public boolean hasDeferrableChecking()
+    {
+        return id.hasDeferrableChecking();
+    }
+
+
+    public boolean isUniqueDeferrable()
+    {
+        return id.isUniqueDeferrable();
+    }
+
+
 	/** @see IndexDescriptor#isUnique */
 	public boolean isUnique()
 	{

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/SortObserver.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/SortObserver.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/SortObserver.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/SortObserver.java Mon Nov 25 20:30:45 2013
@@ -128,4 +128,34 @@ public interface SortObserver
 
 	public DataValueDescriptor[] getArrayClone()
 		throws StandardException;
+
+    /**
+     * Overridden by subclasses that observe sorters with uniqueness checking.
+     * @return true if the index's constraint is deferrable. Any SortObserver
+     * implementations that implement uniqueness checking need to keep track of
+     * this information.
+     */
+    public boolean deferrable();
+
+    /**
+     * Overridden by subclasses that observe sorters with uniqueness checking.
+     * @return true if constraint mode of the index's constraint is effectively
+     * deferred. Any SortObserver
+     * implementations that implement uniqueness checking need to keep track of
+     * this information.
+     */
+    public boolean deferred();
+
+    /**
+     * Overridden by subclasses that observe sorters with uniqueness checking.
+     * Will be called by sorters iff deferrable() and deferred() and
+     * uniqueness violation, so implementations that sometimes return
+     * true to these must implement this method to save duplicate
+     * information till commit time.
+     *
+     * @param row data of offending key
+     * @throws StandardException standard error policy
+     */
+    public void rememberDuplicate(DataValueDescriptor[] row)
+            throws StandardException;
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedConnection.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedConnection.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedConnection.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedConnection.java Mon Nov 25 20:30:45 2013
@@ -48,10 +48,10 @@ import org.apache.derby.iapi.sql.conn.La
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
 import org.apache.derby.iapi.store.access.XATransactionController;
 import org.apache.derby.iapi.store.access.TransactionController;
+import org.apache.derby.iapi.store.access.XATransactionController;
 
 import org.apache.derby.iapi.store.replication.master.MasterFactory;
 import org.apache.derby.iapi.store.replication.slave.SlaveFactory;
-
 import java.io.IOException;
 
 import java.security.Permission;
@@ -86,6 +86,7 @@ import java.util.HashMap;
 import java.util.Properties;
 import java.util.Iterator;
 import java.util.concurrent.Executor;
+import javax.transaction.xa.XAException;
 
 import org.apache.derby.iapi.jdbc.EngineLOB;
 import org.apache.derby.iapi.jdbc.FailedProperties40;
@@ -3005,8 +3006,12 @@ public class EmbedConnection implements 
 
     /**
      * Do not use this method directly use XATransactionState.xa_prepare
-     * instead because it also maintains/cancels the timout task which is
+     * instead because it also maintains/cancels the timeout task which is
      * scheduled to cancel/rollback the global transaction.
+     *
+     * @return One of {@link org.apache.derby.iapi.store.access.XATransactionController#XA_OK} or
+     *         {@link org.apache.derby.iapi.store.access.XATransactionController#XA_RDONLY}
+     * @throws java.sql.SQLException
      */
 	public final int xa_prepare() throws SQLException {
 
@@ -3019,6 +3024,13 @@ public class EmbedConnection implements 
 				XATransactionController tc = 
                     (XATransactionController)lcc.getTransactionExecute();
 
+                try {
+                    lcc.checkIntegrity();
+                } catch (StandardException e) {
+                    lcc.xaRollback();
+                    throw e;
+                }
+
 				int ret = tc.xa_prepare();
 
 				if (ret == XATransactionController.XA_RDONLY)

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/load/Import.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/load/Import.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/load/Import.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/load/Import.java Mon Nov 25 20:30:45 2013
@@ -56,7 +56,7 @@ public class Import extends ImportAbstra
     private String inputFileName;
 
 	/**
-	 * Constructior to Invoke Import from a select statement 
+     * Constructor to Invoke Import from a select statement
 	 * @param inputFileName	 The URL of the ASCII file from which import will happen
      * @exception SQLException on error
 	 */

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericPreparedStatement.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericPreparedStatement.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericPreparedStatement.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericPreparedStatement.java Mon Nov 25 20:30:45 2013
@@ -47,6 +47,7 @@ import org.apache.derby.iapi.sql.ResultD
 import org.apache.derby.iapi.sql.ResultSet;
 import org.apache.derby.iapi.sql.Statement;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
+import org.apache.derby.iapi.sql.conn.SQLSessionContext;
 import org.apache.derby.iapi.sql.conn.StatementContext;
 import org.apache.derby.iapi.sql.depend.DependencyManager;
 import org.apache.derby.iapi.sql.depend.Provider;
@@ -453,7 +454,11 @@ recompileOutOfDatePlan:
 				statementContext.clearSavePoint();
 			}
 
-			lccToUse.popStatementContext(statementContext, null);					
+            lccToUse.popStatementContext(statementContext, null);
+
+            if (activation.getSQLSessionContextForChildren() != null) {
+                lccToUse.popNestedSessionContext(activation);
+            }
 
 			if (activation.isSingleExecution() && resultSet.isClosed())
 			{

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java Mon Nov 25 20:30:45 2013
@@ -8971,26 +8971,14 @@ public final class	DataDictionaryImpl
 
         IndexRowGenerator irg = null;
 
-        if (softwareVersion.checkVersion(
-                DataDictionary.DD_VERSION_DERBY_10_4,null)) 
-        {
-            irg = new IndexRowGenerator(
+        irg = new IndexRowGenerator(
                 "BTREE", ti.isIndexUnique(indexNumber),
                 false,
+                false,
+                false,
                 baseColumnPositions,
                 isAscending,
                 baseColumnPositions.length);
-        }
-        else 
-        {
-            //older version of Data Disctionary
-            //use old constructor
-            irg = new IndexRowGenerator (
-                "BTREE", ti.isIndexUnique(indexNumber),
-                baseColumnPositions,
-                isAscending,
-                baseColumnPositions.length);
-        }
 
 		// For now, assume that all index columns are ordered columns
 		ti.setIndexRowGenerator(indexNumber, irg);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/TabInfoImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/TabInfoImpl.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/TabInfoImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/TabInfoImpl.java Mon Nov 25 20:30:45 2013
@@ -522,8 +522,8 @@ class TabInfoImpl
 													   heapLocation,
 													   row );
 
-				insertRetCode = 
-                    indexControllers[ ictr ].insert(indexableRow.getRowArray());
+                insertRetCode = indexControllers[ ictr ].insert(
+                        indexableRow.getRowArray());
 
 				if ( insertRetCode == ConglomerateController.ROWISDUPLICATE )
 				{

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/CreateIndexNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/CreateIndexNode.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/CreateIndexNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/CreateIndexNode.java Mon Nov 25 20:30:45 2013
@@ -270,7 +270,10 @@ class CreateIndexNode extends DDLStateme
 		return getGenericConstantActionFactory().getCreateIndexConstantAction(
                     false, // not for CREATE TABLE
                     unique,
-                    false, //its not a UniqueWithDuplicateNulls Index
+                    false, // it's not a UniqueWithDuplicateNulls Index
+                    false, // it's not a constraint, so its checking
+                           // is not deferrable
+                    false, // initialltDeferred: N/A
                     indexType,
                     sd.getSchemaName(),
                     indexName.getTableName(),

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SetConstraintsNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SetConstraintsNode.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SetConstraintsNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SetConstraintsNode.java Mon Nov 25 20:30:45 2013
@@ -25,12 +25,7 @@ import java.util.List;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.services.context.ContextManager;
 import org.apache.derby.shared.common.sanity.SanityManager;
-import org.apache.derby.iapi.sql.compile.CompilerContext;
-import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
-import org.apache.derby.iapi.sql.dictionary.ConstraintDescriptorList;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
-import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
-import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.execute.ConstantAction;
 
 /**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/StaticMethodCallNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/StaticMethodCallNode.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/StaticMethodCallNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/StaticMethodCallNode.java Mon Nov 25 20:30:45 2013
@@ -38,6 +38,7 @@ import org.apache.derby.iapi.services.co
 import org.apache.derby.shared.common.sanity.SanityManager;
 import org.apache.derby.iapi.sql.compile.CompilerContext;
 import org.apache.derby.iapi.sql.conn.Authorizer;
+import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 import org.apache.derby.iapi.sql.dictionary.AliasDescriptor;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
 import org.apache.derby.iapi.types.DataTypeDescriptor;
@@ -843,13 +844,13 @@ class StaticMethodCallNode extends Metho
 	 * procedure or function which needs a nested SQL session
 	 * context (only needed for those which can contain SQL).
 	 *
-	 * The generated code calls setupNestedSessionContext.
-	 * @see org.apache.derby.iapi.sql.conn.LanguageConnectionContext#setupNestedSessionContext
+     * The generated code calls pushNestedSessionContext.
+     * @see LanguageConnectionContext#pushNestedSessionContext
 	 *
 	 * @param acb activation class builder
 	 * @param mb  method builder
 	 */
-    private void generateSetupNestedSessionContext(
+    private void generatePushNestedSessionContext(
         ActivationClassBuilder acb,
         MethodBuilder mb,
         boolean hadDefinersRights,
@@ -857,7 +858,7 @@ class StaticMethodCallNode extends Metho
 
 		// Generates the following Java code:
 		// ((Activation)this).getLanguageConnectionContext().
-		//       setupNestedSessionContext((Activation)this);
+        //       pushNestedSessionContext((Activation)this);
 
 		acb.pushThisAsActivation(mb);
 		mb.callMethod(VMOpcode.INVOKEINTERFACE, null,
@@ -867,7 +868,7 @@ class StaticMethodCallNode extends Metho
         mb.push(hadDefinersRights);
         mb.push(definer);
 		mb.callMethod(VMOpcode.INVOKEINTERFACE, null,
-					  "setupNestedSessionContext",
+                      "pushNestedSessionContext",
                       "void", 3);
 	}
 
@@ -1145,7 +1146,7 @@ class StaticMethodCallNode extends Metho
 			// If no SQL, there is no need to setup a nested session
 			// context.
 			if (sqlAllowed != RoutineAliasInfo.NO_SQL) {
-                generateSetupNestedSessionContext(
+                generatePushNestedSessionContext(
                     (ActivationClassBuilder) acb,
                     mb,
                     routineInfo.hasDefinersRights(),

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=1545394&r1=1545393&r2=1545394&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 Mon Nov 25 20:30:45 2013
@@ -1044,6 +1044,8 @@ class TableElementList extends QueryTree
 						forCreateTable,
 						unique,
                         uniqueWithDuplicateNulls,
+                        cChars[0], // deferrable?
+                        cChars[1], // initiallyDeferred?
 						null, constraintDN,
 						columnNames, true, tableSd, tableName,
 						constraintType, dd);
@@ -1053,6 +1055,8 @@ class TableElementList extends QueryTree
 					indexAction = genIndexAction(
 						forCreateTable,
 						constraintDN.requiresUniqueIndex(), false,
+                        cChars[0], // deferrable
+                        cChars[1], // initiallyDeferred?
 						null, constraintDN,
 						columnNames, true, tableSd, tableName,
 						constraintType, dd);
@@ -1176,6 +1180,10 @@ class TableElementList extends QueryTree
      *                                      column in the key has a null value,
      *                                      no checking is done and insert will
      *                                      always succeed.
+     * @param hasDeferrableChecking         True if index is used to back a
+     *                                      deferrable constraint
+     * @param initiallyDeferred             True means the deferrable constraint
+     *                                      has deferred mode
      * @param indexName	                    The type of index (BTREE, for 
      *                                      example)
      * @param cdn
@@ -1192,6 +1200,8 @@ class TableElementList extends QueryTree
     boolean                     forCreateTable,
     boolean                     isUnique,
     boolean                     isUniqueWithDuplicateNulls,
+    boolean                     hasDeferrableChecking,
+    boolean                     initiallyDeferred,
     String                      indexName,
     ConstraintDefinitionNode    cdn,
     String[]                    columnNames,
@@ -1235,6 +1245,8 @@ class TableElementList extends QueryTree
                     forCreateTable, 
                     isUnique, 
                     isUniqueWithDuplicateNulls,
+                    hasDeferrableChecking,
+                    initiallyDeferred,
                     "BTREE", // indexType
                     sd.getSchemaName(),
                     indexName,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj Mon Nov 25 20:30:45 2013
@@ -245,13 +245,15 @@ public class SQLParser
 		upgraded to the required level to use this functionality. Used to
 		disallow SQL statements that would leave on-disk formats that would
 		not be understood by a engine that matches the current upgrade level
-		of the database. Throws an exception if the database is not a the required level.
+        of the database. Throws an exception if the database is not at
+        the required level.
 		<P>
 		Typically used for CREATE statements at the parser level. Called usually just
 		before the node is created, or can be called in just a partial syntax fragment
 		
 		@param version Data Dictionary major version (DataDictionary.DD_ constant)
 		@param feature SQL Feature name, for error text.
+        @throws If the database is not at the required level.
 	*/
 	private boolean checkVersion(int version, String feature) throws StandardException
 	{
@@ -3144,6 +3146,8 @@ setConstraintsStatement() throws Standar
         }
     )
     {
+        checkVersion(
+            DataDictionary.DD_VERSION_DERBY_10_11, "DEFERRED CONSTRAINTS");
         return new SetConstraintsNode(constraints, initiallyDeferred, cm);
     }
 }

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=1545394&r1=1545393&r2=1545394&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 Mon Nov 25 20:30:45 2013
@@ -21,76 +21,76 @@
 
 package org.apache.derby.impl.sql.conn;
 
-import org.apache.derby.iapi.services.context.ContextImpl;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.WeakHashMap;
+import org.apache.derby.catalog.UUID;
+import org.apache.derby.iapi.db.Database;
+import org.apache.derby.iapi.db.TriggerExecutionContext;
+import org.apache.derby.iapi.error.ExceptionSeverity;
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.reference.ContextId;
+import org.apache.derby.iapi.reference.Limits;
+import org.apache.derby.iapi.reference.Property;
+import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.services.cache.CacheManager;
-
-import org.apache.derby.impl.sql.compile.CompilerContextImpl;
-import org.apache.derby.impl.sql.execute.AutoincrementCounter;
-import org.apache.derby.impl.sql.GenericPreparedStatement;
-import org.apache.derby.impl.sql.GenericStatement;
-
-import org.apache.derby.iapi.services.property.PropertyUtil;
+import org.apache.derby.iapi.services.cache.Cacheable;
+import org.apache.derby.iapi.services.context.Context;
+import org.apache.derby.iapi.services.context.ContextImpl;
 import org.apache.derby.iapi.services.context.ContextManager;
+import org.apache.derby.iapi.services.io.FormatableBitSet;
+import org.apache.derby.iapi.services.loader.GeneratedClass;
 import org.apache.derby.iapi.services.monitor.Monitor;
-import org.apache.derby.shared.common.sanity.SanityManager;
+import org.apache.derby.iapi.services.property.PropertyUtil;
 import org.apache.derby.iapi.services.stream.HeaderPrintWriter;
-import org.apache.derby.iapi.services.loader.GeneratedClass;
-import org.apache.derby.iapi.services.cache.Cacheable;
-import org.apache.derby.iapi.services.io.FormatableBitSet;
-import org.apache.derby.iapi.db.Database;
-import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.sql.Activation;
+import org.apache.derby.iapi.sql.LanguageFactory;
+import org.apache.derby.iapi.sql.ParameterValueSet;
+import org.apache.derby.iapi.sql.PreparedStatement;
+import org.apache.derby.iapi.sql.ResultSet;
+import org.apache.derby.iapi.sql.compile.ASTVisitor;
 import org.apache.derby.iapi.sql.compile.CompilerContext;
-import org.apache.derby.iapi.sql.compile.OptimizerFactory;
 import org.apache.derby.iapi.sql.compile.OptTrace;
-import org.apache.derby.iapi.sql.compile.ASTVisitor;
+import org.apache.derby.iapi.sql.compile.OptimizerFactory;
+import org.apache.derby.iapi.sql.compile.TypeCompilerFactory;
 import org.apache.derby.iapi.sql.conn.Authorizer;
-import org.apache.derby.iapi.error.ExceptionSeverity;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionFactory;
-import org.apache.derby.iapi.sql.conn.StatementContext;
 import org.apache.derby.iapi.sql.conn.SQLSessionContext;
+import org.apache.derby.iapi.sql.conn.StatementContext;
+import org.apache.derby.iapi.sql.depend.DependencyManager;
+import org.apache.derby.iapi.sql.depend.Provider;
 import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
 import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptorList;
+import org.apache.derby.iapi.sql.dictionary.ConstraintDescriptor;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
+import org.apache.derby.iapi.sql.dictionary.RoleGrantDescriptor;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
-import org.apache.derby.iapi.sql.dictionary.RoleGrantDescriptor;
-import org.apache.derby.iapi.types.DataValueFactory;
-import org.apache.derby.iapi.sql.compile.TypeCompilerFactory;
-import org.apache.derby.iapi.sql.depend.DependencyManager;
-import org.apache.derby.iapi.sql.depend.Provider;
-import org.apache.derby.iapi.reference.SQLState;
-import org.apache.derby.iapi.reference.Limits;
 import org.apache.derby.iapi.sql.execute.ConstantAction;
 import org.apache.derby.iapi.sql.execute.CursorActivation;
 import org.apache.derby.iapi.sql.execute.ExecPreparedStatement;
 import org.apache.derby.iapi.sql.execute.ExecutionStmtValidator;
-import org.apache.derby.iapi.sql.Activation;
-import org.apache.derby.iapi.sql.LanguageFactory;
-import org.apache.derby.iapi.sql.PreparedStatement;
-import org.apache.derby.iapi.sql.ResultSet;
-import org.apache.derby.iapi.sql.ParameterValueSet;
-
+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.store.access.XATransactionController;
 import org.apache.derby.iapi.transaction.TransactionControl;
+import org.apache.derby.iapi.types.DataValueFactory;
 import org.apache.derby.iapi.util.IdUtil;
 import org.apache.derby.iapi.util.InterruptStatus;
-
-import org.apache.derby.catalog.UUID;
-import org.apache.derby.iapi.sql.execute.RunTimeStatistics;
-import org.apache.derby.iapi.db.TriggerExecutionContext;
-import org.apache.derby.iapi.reference.Property;
-
-import java.util.List;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.IdentityHashMap;
-import java.util.WeakHashMap;
-import java.util.Iterator;
-import java.util.Map;
-import org.apache.derby.iapi.reference.ContextId;
-import org.apache.derby.iapi.services.context.Context;
+import org.apache.derby.impl.sql.GenericPreparedStatement;
+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.DeferredDuplicates;
+import org.apache.derby.shared.common.sanity.SanityManager;
 
 /**
  * LanguageConnectionContext keeps the pool of prepared statements,
@@ -219,7 +219,7 @@ public class GenericLanguageConnectionCo
      * The top SQL session context stack frame (SQL 2003, section
      * 4.37.3), is kept in topLevelSSC. For nested session contexts,
      * the SQL session context is held by the activation of the
-     * calling statement, cf. setupNestedSessionContext and it is
+     * calling statement, cf. pushNestedSessionContext and it is
      * accessible through the current statement context
      * (compile-time), or via the current activation (execution-time).
      * @see GenericLanguageConnectionContext#getTopLevelSQLSessionContext
@@ -311,6 +311,13 @@ public class GenericLanguageConnectionCo
      */
     private WeakHashMap<TableDescriptor,FormatableBitSet> referencedColumnMap;
 
+    /**
+     * The set of disk backed hash tables containing any index rows
+     * saved for deferred constraints in this transaction, keyed by the
+     * conglomerate id. Checked at commit time, then discarded.
+     */
+    private HashMap<Long, BackingStoreHashtable> deferredHashTables;
+
     /*
        constructor
     */
@@ -1481,6 +1488,8 @@ public class GenericLanguageConnectionCo
                     SQLState.LANG_NO_COMMIT_IN_NESTED_CONNECTION);
         }
 
+        checkIntegrity();
+
         // Log commit to error log, if appropriate
         if (logStatementText)
         {
@@ -1724,6 +1733,8 @@ public class GenericLanguageConnectionCo
             throw StandardException.newException(SQLState.LANG_NO_ROLLBACK_IN_NESTED_CONNECTION);
         }
 
+        clearDeferreds();
+
         // Log rollback to error log, if appropriate
         if (logStatementText)
         {
@@ -2438,7 +2449,7 @@ public class GenericLanguageConnectionCo
      *
      * Inherit SQL session state a priori (statementContext will get
      * its own SQL session state if this statement executes a call,
-     * cf. setupNestedSessionContext.
+     * cf. pushNestedSessionContext.
 
      * @param isAtomic whether this statement is atomic or not
      * @param isForReadOnly whether this statement is for a read only resultset
@@ -3624,7 +3635,9 @@ public class GenericLanguageConnectionCo
      *
      * @param activation the activation
      */
-    private SQLSessionContext getCurrentSQLSessionContext(Activation activation) {
+    public SQLSessionContext getCurrentSQLSessionContext(
+        Activation activation) {
+
         SQLSessionContext curr;
 
         Activation parent = activation.getParentActivation();
@@ -3669,9 +3682,9 @@ public class GenericLanguageConnectionCo
 
 
     /**
-     * @see LanguageConnectionContext#setupNestedSessionContext(Activation a, boolean definersRights, String definer)
+     * {@inheritDoc}
      */
-    public void setupNestedSessionContext(
+    public void pushNestedSessionContext(
         Activation a,
         boolean definersRights,
         String definer) throws StandardException {
@@ -3736,6 +3749,10 @@ public class GenericLanguageConnectionCo
             sc.setDefaultSchema(getDefaultSchema(a));
         }
 
+        final SQLSessionContext ssc = getCurrentSQLSessionContext(a);
+        sc.setDeferredAll(ssc.getDeferredAll());
+        sc.setConstraintModes(ssc.getConstraintModes());
+
         StatementContext stmctx = getStatementContext();
 
         // Since the statement is an invocation (iff push=true), it will now be
@@ -3754,6 +3771,74 @@ public class GenericLanguageConnectionCo
         stmctx.setSQLSessionContext(sc);
     }
 
+    /**
+     * @param a {@inheritDoc}
+     * @throws StandardException {@inheritDoc}
+     */
+    public void popNestedSessionContext(Activation a) throws StandardException {
+        SQLSessionContext nested = a.getSQLSessionContextForChildren();
+        SQLSessionContext caller = getCurrentSQLSessionContext(a);
+
+        compareConstraintModes(nested, caller);
+    }
+
+    private void compareConstraintModes(
+            SQLSessionContext nested,
+            SQLSessionContext caller) throws StandardException {
+
+        if (deferredHashTables == null) {
+            // Nothing to do
+            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, BackingStoreHashtable>> es =
+                deferredHashTables.entrySet();
+
+        for (Map.Entry<Long, BackingStoreHashtable> e : es) {
+            final long indexCID = e.getKey().longValue();
+
+            boolean effectivelyDeferred = effectivelyDeferred(caller, indexCID);
+
+            if (effectivelyDeferred ) {
+                // the constraint is also deferred in the calling context
+                continue;
+            }
+            // The constraint must have been deferred inside the routine
+            if (SanityManager.DEBUG) {
+                SanityManager.ASSERT(effectivelyDeferred(nested, indexCID));
+            }
+
+            doValidateConstraint(e.getKey().longValue(), e.getValue(), true);
+        }
+    }
+
+    private boolean effectivelyDeferred(SQLSessionContext sc, long indexCID)
+            throws StandardException {
+
+        Boolean deferred = sc.isDeferred(indexCID);
+        boolean effectivelyDeferred;
+        final DataDictionary dd = getDataDictionary();
+
+        if (deferred != null) {
+            effectivelyDeferred = deferred.booleanValue();
+        } else {
+            // no explicit setting applicable, use initial constraint mode
+            final ConglomerateDescriptor cd =
+                    dd.getConglomerateDescriptor(indexCID);
+            final TableDescriptor td =
+                    dd.getTableDescriptor(cd.getTableID());
+            final ConstraintDescriptor conDesc =
+                    dd.getConstraintDescriptor(td, cd.getUUID());
+            effectivelyDeferred = conDesc.initiallyDeferred();
+        }
+
+        return effectivelyDeferred;
+    }
 
     /**
      * @see LanguageConnectionContext#setupSubStatementSessionContext(Activation a)
@@ -3784,7 +3869,7 @@ public class GenericLanguageConnectionCo
     public SQLSessionContext createSQLSessionContext() {
         return new SQLSessionContextImpl(
             getInitialDefaultSchemaDescriptor(),
-            getSessionUserId() /* a priori */);
+            getSessionUserId()); /* a priori */
     }
 
     /**
@@ -3857,4 +3942,107 @@ public class GenericLanguageConnectionCo
                                        FormatableBitSet map) {
         referencedColumnMap.put(td, map);
     }
+
+    /**
+     * {@inheritDoc}
+     */
+    public void setDeferred(Activation a, long conglomId, boolean deferred)
+            throws StandardException {
+        if (!deferred) {
+            // Moving to immediate, check whats done in this transaction first
+            validateDeferredConstraint(conglomId);
+        }
+        getCurrentSQLSessionContext(a).setDeferred(conglomId, deferred);
+    }
+
+    public boolean isEffectivelyDeferred(Activation a, long conglomId)
+            throws StandardException {
+        return effectivelyDeferred(getCurrentSQLSessionContext(a), conglomId);
+    }
+
+
+    public void checkIntegrity() throws StandardException {
+        validateDeferredConstraints(true);
+        clearDeferreds();
+    }
+
+    public void invalidateDeferredConstraintsData(long indexCID)
+            throws StandardException {
+        if (deferredHashTables != null &&
+                deferredHashTables.containsKey(Long.valueOf(indexCID))) {
+            deferredHashTables.remove(Long.valueOf(indexCID));
+        }
+    }
+
+    /**
+     * Clear deferred information for this transaction.
+     */
+    private void clearDeferreds() {
+        deferredHashTables = null;
+        getCurrentSQLSessionContext().resetConstraintModes();
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    public void setDeferredAll(Activation a, boolean deferred)
+            throws StandardException {
+        if (!deferred) {
+            validateDeferredConstraints(false);
+
+            // No violations, so reset the memory
+            deferredHashTables = null;
+        }
+        getCurrentSQLSessionContext(a).setDeferredAll(
+            Boolean.valueOf(deferred));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    public HashMap<Long, BackingStoreHashtable> getDeferredHashTables() {
+        if (deferredHashTables == null) {
+            deferredHashTables = new HashMap<Long, BackingStoreHashtable>();
+        }
+        return deferredHashTables;
+    }
+
+    private void validateDeferredConstraints(boolean rollbackOnError)
+            throws StandardException {
+        if (deferredHashTables == null) {
+            // Nothing to do
+            return;
+        }
+
+        Set<Map.Entry<Long, BackingStoreHashtable>> es =
+                deferredHashTables.entrySet();
+
+        for (Map.Entry<Long, BackingStoreHashtable> e : es) {
+            doValidateConstraint(e.getKey().longValue(),
+                                 e.getValue(),
+                                 rollbackOnError);
+        }
+    }
+
+
+    private void validateDeferredConstraint(long indexCID)
+            throws StandardException {
+        BackingStoreHashtable ht = null;
+
+        if (deferredHashTables == null ||
+            (ht = deferredHashTables.get(indexCID)) == null) {
+            // Nothing to do
+            return;
+        }
+        doValidateConstraint(indexCID, ht, false);
+        deferredHashTables.remove(indexCID);
+    }
+
+    private void doValidateConstraint(
+        long indexCID,
+        BackingStoreHashtable ht,
+        boolean rollbackOnError) throws StandardException {
+
+        DeferredDuplicates.validate(tran, indexCID, this, ht, rollbackOnError);
+    }
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/SQLSessionContextImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/SQLSessionContextImpl.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/SQLSessionContextImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/SQLSessionContextImpl.java Mon Nov 25 20:30:45 2013
@@ -21,7 +21,7 @@
 
 package org.apache.derby.impl.sql.conn;
 
-import java.lang.String;
+import java.util.HashMap;
 import org.apache.derby.iapi.sql.conn.SQLSessionContext;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
 
@@ -31,10 +31,30 @@ public class SQLSessionContextImpl imple
     private String currentRole;
     private SchemaDescriptor currentDefaultSchema;
 
-    public SQLSessionContextImpl (SchemaDescriptor sd, String currentUser) {
+    /**
+     * Maps a conglomerate id (key) into a Boolean for deferrable constraints.
+     * There is a 1-1 correspondence for these backing indexes, they are not
+     * shared). If the Boolean value is {@code FALSE}, we have immediate
+     * checking, if it is {@code TRUE} we have deferred checking. Cf. SQL
+     * SET CONSTRAINT.
+     */
+    private HashMap<Long, Boolean> constraintModes;
+
+    /**
+     * True if all deferrable constraints are deferred in this transaction.
+     */
+    private Boolean deferredAll;
+
+    public SQLSessionContextImpl (
+            SchemaDescriptor sd,
+            String currentUser) {
         currentRole = null;
         currentDefaultSchema = sd;
         this.currentUser = currentUser;
+
+        if (constraintModes != null) {
+            this.constraintModes = new HashMap<Long,Boolean>(constraintModes);
+        }
     }
 
     public void setRole(String role) {
@@ -60,4 +80,82 @@ public class SQLSessionContextImpl imple
     public SchemaDescriptor getDefaultSchema() {
         return currentDefaultSchema;
     }
+
+    /**
+     * {@inheritDoc}
+     */
+    public HashMap<Long, Boolean> getConstraintModes() {
+        return constraintModes != null ?
+            new HashMap<Long, Boolean>(constraintModes) :
+            null;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    public void setConstraintModes(HashMap<Long, Boolean> hm) {
+        this.constraintModes = hm != null ?
+                new HashMap<Long, Boolean>(hm) : null;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    public void setDeferred(long conglomId, boolean deferred) {
+        if (constraintModes == null) {
+            constraintModes = new HashMap<Long, Boolean>();
+        }
+
+        constraintModes.put(Long.valueOf(conglomId),
+                                Boolean.valueOf(deferred));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    public Boolean isDeferred(long conglomId) {
+        Boolean v = null;
+
+        if (constraintModes != null) {
+            v = constraintModes.get(Long.valueOf(conglomId));
+        }
+
+        if (v != null) {
+            return v; // Trumps ALL setting since it must have been
+                      // set later otherwise it would have been
+                      // deleted
+        } else {
+            return deferredAll;
+        }
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    public void resetConstraintModes() {
+        if (constraintModes != null) {
+            constraintModes.clear();
+        }
+
+        deferredAll = null;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    public void setDeferredAll(Boolean deferred) {
+        deferredAll = deferred;
+        // This now overrides any individual constraint setting, so
+        // clear those.
+        if (constraintModes != null) {
+            constraintModes.clear();
+        }
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    public Boolean getDeferredAll() {
+        return deferredAll;
+    }
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterConstraintConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterConstraintConstantAction.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterConstraintConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterConstraintConstantAction.java Mon Nov 25 20:30:45 2013
@@ -156,9 +156,12 @@ public class AlterConstraintConstantActi
         }
 
         if (characteristics[2] != ConstraintDefinitionNode.ENFORCED_DEFAULT) {
+            dd.checkVersion(DataDictionary.DD_VERSION_DERBY_10_11,
+                            "DEFERRED CONSTRAINTS");
+
             // Remove when feature DERBY-532 is completed
-            if (!PropertyUtil.getSystemProperty("derby.constraintsTesting",
-                    tableName).equals("true")) {
+            if (!PropertyUtil.getSystemProperty(
+                        "derby.constraintsTesting", "false").equals("true")) {
                 throw StandardException.newException(SQLState.NOT_IMPLEMENTED,
                         "non-default enforcement");
             }

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=1545394&r1=1545393&r2=1545394&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 Mon Nov 25 20:30:45 2013
@@ -2579,9 +2579,15 @@ class AlterTableConstantAction extends D
 		if(cd.getIndexDescriptor().isUniqueWithDuplicateNulls())
 		{
 			properties.put(
-                    "uniqueWithDuplicateNulls", Boolean.toString(true));
+                "uniqueWithDuplicateNulls", Boolean.toString(true));
 		}
 
+        if (cd.getIndexDescriptor().hasDeferrableChecking())
+        {
+            properties.put(
+                "hasDeferrableChecking", Boolean.toString(true));
+        }
+
 		properties.put(
             "rowLocationColumn", Integer.toString(indexRowLength - 1));
 		properties.put(

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java?rev=1545394&r1=1545393&r2=1545394&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java Mon Nov 25 20:30:45 2013
@@ -30,14 +30,9 @@ import java.util.HashSet;
 import java.util.Hashtable;
 import java.util.Stack;
 import java.util.Vector;
-
-import	org.apache.derby.catalog.Dependable;
-import	org.apache.derby.catalog.DependableFinder;
+import org.apache.derby.catalog.Dependable;
+import org.apache.derby.catalog.DependableFinder;
 import org.apache.derby.catalog.UUID;
-import org.apache.derby.iapi.services.uuid.UUIDFactory;
-import org.apache.derby.iapi.services.monitor.Monitor;
-import org.apache.derby.iapi.sql.depend.Provider;
-import org.apache.derby.iapi.sql.Row;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.jdbc.ConnectionContext;
 import org.apache.derby.iapi.reference.Property;
@@ -49,16 +44,20 @@ import org.apache.derby.iapi.services.io
 import org.apache.derby.iapi.services.loader.GeneratedByteCode;
 import org.apache.derby.iapi.services.loader.GeneratedClass;
 import org.apache.derby.iapi.services.loader.GeneratedMethod;
+import org.apache.derby.iapi.services.monitor.Monitor;
 import org.apache.derby.iapi.services.property.PropertyUtil;
-import org.apache.derby.shared.common.sanity.SanityManager;
+import org.apache.derby.iapi.services.uuid.UUIDFactory;
 import org.apache.derby.iapi.sql.Activation;
 import org.apache.derby.iapi.sql.ParameterValueSet;
+import org.apache.derby.iapi.sql.PreparedStatement;
 import org.apache.derby.iapi.sql.ResultDescription;
 import org.apache.derby.iapi.sql.ResultSet;
+import org.apache.derby.iapi.sql.Row;
 import org.apache.derby.iapi.sql.compile.Optimizer;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 import org.apache.derby.iapi.sql.conn.SQLSessionContext;
 import org.apache.derby.iapi.sql.depend.DependencyManager;
+import org.apache.derby.iapi.sql.depend.Provider;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.execute.ConstantAction;
 import org.apache.derby.iapi.sql.execute.CursorActivation;
@@ -80,6 +79,7 @@ import org.apache.derby.iapi.types.Numbe
 import org.apache.derby.iapi.types.RowLocation;
 import org.apache.derby.iapi.types.StringDataValue;
 import org.apache.derby.iapi.util.ReuseFactory;
+import org.apache.derby.shared.common.sanity.SanityManager;
 
 /**
  * BaseActivation
@@ -182,16 +182,16 @@ public abstract class BaseActivation imp
 	 *
 	 * A non-null 'parentActivation' represents the activation of the calling
 	 * statement (if we are in a nested connection of a stored routine), or the
-	 * activation of the parent statement (if we are executing a substatement)
+     * activation of the parent statement (if we are executing a sub-statement)
 	 *
 	 * 'parentActivation' is set when this activation is created (@see
-	 * GenericPreparedStatement#getActivation) based on the top of the
+     * PreparedStatement#getActivation) based on the top of the
 	 * dynamic call stack of execution, which is tracked by
 	 * StatementContext. The nested SQL session context is initialized
 	 * by code generated for the call, after parameters are evaluated
-	 * or just substatement execution starts.
-	 * @see org.apache.derby.impl.sql.compile.StaticMethodCallNode#generateSetupNestedSessionContext
-	 * @see org.apache.derby.impl.sql.GenericPreparedStatement#executeSubStatement
+     * or just sub-statement execution starts.
+     * @see org.apache.derby.impl.sql.compile.StaticMethodCallNode#generatePushNestedSessionContext
+     * @see PreparedStatement#executeSubStatement
 	 *
 	 */
 	private Activation parentActivation;
@@ -301,6 +301,7 @@ public abstract class BaseActivation imp
     /**
      * Create the ResultSet tree for this statement.
      * @return the root of the ResultSet tree for this statement
+     * @throws StandardException standard error policy
      */
     protected abstract ResultSet createResultSet() throws StandardException;
 
@@ -309,6 +310,7 @@ public abstract class BaseActivation imp
      * execution of the statement. The default implementation does nothing.
      * Sub-classes should override this method if they need to perform
      * operations before each execution.
+     * @throws org.apache.derby.iapi.error.StandardException
      */
     protected void reinit() throws StandardException {
         // Do nothing by default. Overridden by sub-classes that need it.
@@ -349,11 +351,13 @@ public abstract class BaseActivation imp
 	}
 
 	/**
-		Link this activation with its PreparedStatement.
-		It can be called with null to break the link with the
-		PreparedStatement.
-
-	*/
+     * Link this activation with its PreparedStatement.
+     * It can be called with null to break the link with the
+     * PreparedStatement.
+     * @param ps prepared statement
+     * @param scrollable activation for a scrollable result set
+     * @throws StandardException standard error policy
+     */
 	public final void setupActivation(ExecPreparedStatement ps, boolean scrollable) 
 	throws StandardException {
 		preStmt = ps;
@@ -509,9 +513,10 @@ public abstract class BaseActivation imp
 	}
 
 	/**
-		A generated class can create its own closeActivationAction
-		method to invoke special logic when the activation is closed.
-	*/
+     * A generated class can create its own closeActivationAction
+     * method to invoke special logic when the activation is closed.
+     * @throws java.lang.Exception error
+     */
 	protected void closeActivationAction() throws Exception {
 		// no code to be added here as generated code
 		// will not call super.closeActivationAction()
@@ -1412,13 +1417,6 @@ public abstract class BaseActivation imp
 	 * @see org.apache.derby.iapi.sql.Activation#getSQLSessionContextForChildren
 	 */
 	public SQLSessionContext getSQLSessionContextForChildren() {
-
-		if (SanityManager.DEBUG) {
-			SanityManager.ASSERT
-				(sqlSessionContextForChildren != null,
-				 "Expected sqlSessionContextForChildren to be non-null");
-		}
-
 		return sqlSessionContextForChildren;
 	}