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/03/11 15:46:17 UTC

svn commit: r1576367 [1/4] - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/error/ engine/org/apache/derby/iapi/sql/compile/ engine/org/apache/derby/iapi/sql/conn/ engine/org/apache/derby/iapi/sql/dictionary/ engine/org/apache/derby/iapi/sq...

Author: dag
Date: Tue Mar 11 14:46:15 2014
New Revision: 1576367

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

Patch derby-532-check-constraints-2 which implements deferred CHECK
constraints and supporting tests.

The high level approach is as follows.  When a violation occurs, we note the row
location in the base table of the offending row. At commit time (or when
switching a constraint to immediate), we revisit those rows using the row
locations if they are still valid, and validate those rows again. This is
achieved by positioning to the saved row locations in combination with a
specially crafted result set: ValidateCheckConstraintResultSet (see
ProjectRestrictResultSet#getNextRowCore) which positions to the offending base
row using ValidateCheckConstraintResultSet#positionScanAtRowLocation before
letting ValidateCheckConstraintResultSet read the row. If the row locations are
no longer valid, e.g. an intervening compress happened, we do a full table scan
to verify the constraints instead.

Adding a constraint in deferred constraint mode is currently sub-optimal, since
we currently do a full table scan via an internally generated "SELECT .. WHERE
NOT <constraints>", and we don't have a way the get at the row locations of the
offending rows in this case. I might add a specially tailored result set for
that purpose later.

Normally, when a row is inserted or updated, we execute a generated method which
combines evaluation of all check constraints on the table relevant for the
inserted or updated columns. This evaluation is performed using McCarthy boolean
evaluation (short-circuits as soon as result is known). This isn't optimal for
deferred constraints, as we'd need to assume all constraints were violated in
such a case. The implementation replaces the short-circuited evaluation with a
full evaluation, so we can remember exactly which constraints were violated,
cf. AndNoShortCircuitNode and SQLBoolean#throwExceptionIfImmediateAndFalse. A
violation in throwExceptionIfImmediateAndFalse when we have a deferred
constraint is noted (DMLWriteResultSet#rememberConstraint implemented by
UpdateResultSet and InsertResultSet) by adding the violation to a list for that
row. After the insert/update is completed, the set of violations is remembered
for posterity, cf. InsertResultSet#normalInsertCode and
UpdateResultSet#collectAffectedRows by inspecting the lists
(#violatingCheckConstraints).

Note that we currently do not note which constraints were violated *for each
individual row*, only per table in the transaction. This means that we visit
potentially more rows over again when a single constraint is changed to
immediate.  This could be improved further by storing the set of violated
constraints along with the row location.

For bulk insert and deferred (see panel 1 below) insert row processing there is
special code paths, cf.  InsertResultSet#offendingRowLocation which is invoked
via a callback from HeapController#load and another path in
InsertResultSet#normalInsertCode respectively.

For update, the code for deferred treatment is in in one of
UpdateResultSet#collectAffectedRows and UpdateResultSet#updateDeferredRows
depending on whether there are triggers.

The existing test ConstraintCharacteristcsTest has been built out by adding
check constraint to those fixture for which it is relevant, as well as adding
new ones which are only relevant for check constraints.

[1] This "deferred" refers to Derby special handling of rows in certain
situation, for example when doing an insert which uses the same table as a
source result set, we need to make sure we don't get confused and see the
incrementally inserted rows "again" as we process the original result set,
essentially we do a snapshot of the source result set, hence "deferred rows".

All regressions passed.

Detailed code comments:

M java/engine/org/apache/derby/iapi/sql/conn/SQLSessionContext.java
M java/engine/org/apache/derby/impl/sql/conn/SQLSessionContextImpl.java
M java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java
M java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java
D java/engine/org/apache/derby/impl/sql/execute/DeferredDuplicates.java
A java/engine/org/apache/derby/impl/sql/execute/DeferredConstraintsMemory.java

Extended and refactored slightly existing mechanism for deferred primary
key/unique constraints to also cater for check constraints. Since the hash key
we used for the memory of primary key and unique constraints was the
conglomerate id of the indexes, and those are guaranteed to be disjoint from the
conglomerate ids of the base tables having deferred constraints, we can use the
same hash table to find the "memory" in the form of the disk based hash table
(BackingStoreHashtable), cf.  LCC#getDeferredHashTables.--

M java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java-

Code to drop any deferred constraints memory in the transaction when a
constraint is dropped.-

M java/engine/org/apache/derby/impl/store/access/heap/HeapController.java

Call back added for bulk insert in the presence of deferrable check constraints.

M java/engine/org/apache/derby/iapi/sql/execute/NoPutResultSet.java
M java/engine/org/apache/derby/impl/sql/execute/NoPutResultSetImpl.java
M java/engine/org/apache/derby/iapi/store/access/RowLocationRetRowSource.java

Extra plumbing to be able to signal to HeapController that we need to do a
callback with the inserted row location (for bulk insert)

M java/engine/org/apache/derby/iapi/sql/execute/TargetResultSet.java

Extra interface method, offendingRowLocation. Only implemented with meaningful
semantics for NoPutResultSetImpl which calls it for its targetResultSet, an
InsertResultSet.

M java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
M java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java

More parameters to getProjectRestrictResult set to do the magic mention in the
overview for that result set, pass along schema and table name to
InsertResultSet so we can remember them for check violations. They are used to
produced checking SQL statements. This may be a bit fragile, since a rename
schema or table could make those invalid. However, there is presently no RENAME
SCHEMA in Derby and the RENAME TABLE is illegal in certain cases, notably if
there is a check constraint defined on it, so the solution should be OK for
now. Also adds an interface method, getValidateCheckConstraintResultSet, to
allow the execution run-time to build one of those, cf. code generation logic in
NestedLoopStrategy#resultSetMethodName.

M java/engine/org/apache/derby/iapi/sql/execute/RowChanger.java
M java/engine/org/apache/derby/impl/sql/execute/RowChangerImpl.java

Extra parameter to insertRow to get at the row location if needed.

M java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java
M java/engine/org/apache/derby/iapi/store/access/ScanController.java

Javadoc fixes.

M java/engine/org/apache/derby/iapi/types/BooleanDataValue.java
M java/engine/org/apache/derby/iapi/types/SQLBoolean.java

Extra method throwExceptionIfImmediateAndFalse used by deferred check
constraints to make a note of all violated constraints as evaluated by the
generated method. Picked up by InsertResultSet or UpdateResultSet.

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

AndNoShortCircuitNode is used to represent a non-McCarthy evaluation of the
combined check constraints. See usage in DMLModStatementNode#generateCheckTree.

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

Extra dummy parameter added for call to super#bindConstraints
(DMLModStatementNode). Only used by insert.

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

Pick up the DERBY_PROPERTIES value for property "validateCheckConstraint =
<conlomerateId>" we provide to the checking query (internal syntax only)
generated by DeferredConstraintsMemory#validateCheck.  The conglomerate id is
used to retrieve the violating rows information set up by
ProjectRestrictResultSet#openCore to drive ValidateCheckConstraintResultSet.

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

Boolean member variable to know if we have a deferrable check constraint; also
pass only schema and table name to the result set. Passed on to the
InsertConstantAction from which InsertResultSet can pick it up.

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

Logic to keep track of whether we are used by the special internal query to
check violated check constraints.  In this case we also do not push the check
predicates down to store for simpler handling.

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

Code to parse a long value from "--DERBY-PROPERTIES" property.

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

Extra code to comply with the sane mode parse tree printing conventions.

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

Handle different code generation for deferrable check contraints.

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

Pass on more info: schema and table name + small refactoring.

M  java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj

Handle the new internal query to validate violated check constraints. Cf. query
in DeferredConstraintsMemory#validateCheck.

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

Open up for check constraints.

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

ATCA: Special handling of adding a deferred check constraint: need different
code path to get the UUID of constraint soon enough to be able to note any
constraint violations. CCA: note any violation and remember it.  We'd like to
remember that row locations of the offending rows here, but not done for now, so
at checking time, we'll need a full table scan. This can be improved upon, see
code comment.

M java/engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java
M java/engine/org/apache/derby/impl/sql/execute/UpdateConstantAction.java
M java/engine/org/apache/derby/impl/sql/execute/InsertConstantAction.java

Pass on more info to InsertConstantAction and UpdateConstantAction needed by the
result sets.

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

Drives the checking for check constraints, and picks up the result. If we have
violations and deferred constraints, we remember that. Also some refactorings to
avoid local variables shadowing globals.

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

Drives the checking for check constraints, and picks up the result. If we have
violations and deferred constraints, we remember that.

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

Removed unused method.

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

Drive the special result set, ValidateCheckConstraintResultSet by positioning it
correctly for each row retrieved, using the remembered row locations from
violation time.

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

Added logic for check constraints. Also added a new check that the user don't
specify the same constraint twice, cf new test case for it.

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

Make some members protected rather than private, to let the new result set
ValidateCheckConstraintResultSet inherit from it.

M java/engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderResultSet.java
M java/engine/org/apache/derby/impl/store/access/sort/MergeScanRowSource.java
M java/engine/org/apache/derby/impl/store/access/sort/SortBufferRowSource.java
M java/engine/org/apache/derby/impl/sql/execute/CardinalityCounter.java
M java/engine/org/apache/derby/impl/sql/execute/DMLWriteResultSet.java

Boiler plate to comply with interface (not used).

M java/engine/org/apache/derby/impl/sql/execute/UniqueIndexSortObserver.java
M java/engine/org/apache/derby/impl/sql/execute/UniqueWithDuplicateNullsIndexSortObserver.java
M java/engine/org/apache/derby/impl/sql/execute/IndexChanger.java
M java/engine/org/apache/derby/impl/sql/execute/CreateIndexConstantAction.java

Refactoring only.

A java/engine/org/apache/derby/impl/sql/execute/ValidateCheckConstraintResultSet.java

The new result we use to check violating rows only based on row location

M java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java
M java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java
M java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java

New boolean to signal that we want ValidateCheckConstraintResultSet

M java/engine/org/apache/derby/jdbc/EmbedXAResource.java
M java/engine/org/apache/derby/jdbc/XATransactionState.java

Extra logic to handle check constraints (already had it for primary key and unique).

M java/engine/org/apache/derby/iapi/error/ExceptionUtil.java

Utility method to determine if an exception if a transaction deferred constraint
violation. Needed by the XA code.

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

New error messages

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

New test cases and extension of present ones to include check constraints

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

Extension of present test cases to include check constraints.



Added:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/AndNoShortCircuitNode.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/ValidateCheckConstraintResultSet.java
Removed:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeferredDuplicates.java
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/error/ExceptionUtil.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.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/dictionary/ConstraintDescriptor.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/NoPutResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/RowChanger.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/TargetResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/RowLocationRetRowSource.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/ScanController.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/BooleanDataValue.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/AndNode.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/DeleteNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ProjectRestrictNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/QueryTreeNode.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/TestConstraintNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.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/CardinalityCounter.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/CreateIndexConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DMLWriteResultSet.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/GenericResultSetFactory.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/InsertConstantAction.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/NoPutResultSetImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoRowsResultSetImpl.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/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/TableScanResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderResultSet.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/sql/execute/UpdateConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapController.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/sort/MergeScanRowSource.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/sort/SortBufferRowSource.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/upgradeTests/Changes10_11.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/error/ExceptionUtil.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/error/ExceptionUtil.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/error/ExceptionUtil.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/error/ExceptionUtil.java Tue Mar 11 14:46:15 2014
@@ -34,7 +34,7 @@ import java.security.AccessControlExcept
 import java.security.AccessController;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
-
+import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.shared.common.error.ExceptionSeverity;
 
 /**
@@ -60,7 +60,11 @@ public class ExceptionUtil
     }
     
     /**
-     * Get the severity given a message identifier from SQLState.
+     * Get the severity given a message identifier from {@code SQLState}.
+     *
+     * @param messageID the string carrying the SQL state
+     * @return the constant representing the severity, as defined in
+     *        {@link org.apache.derby.iapi.error.ExceptionSeverity}.
      */
     public static int getSeverityFromIdentifier(String messageID) {
 
@@ -138,6 +142,7 @@ public class ExceptionUtil
      *
      * @return stack traces for all live threads as a string or an error message.
      */
+    @SuppressWarnings({"BroadCatchBlock", "TooBroadCatch", "UseSpecificCatch"})
     public static String dumpThreads() {
 
         StringWriter out = new StringWriter();
@@ -147,7 +152,7 @@ public class ExceptionUtil
         try {
             //This checks that we are on a jvm >= 1.5 where we
             //can actually do threaddumps.
-            Thread.class.getMethod("getAllStackTraces", new Class[] {});
+            Thread.class.getMethod("getAllStackTraces", new Class<?>[] {});
 
             //Then get the thread dump.
             Class<?> c = Class.forName("org.apache.derby.iapi.error.ThreadDump");
@@ -161,7 +166,7 @@ public class ExceptionUtil
                 IllegalArgumentException,
                 IllegalAccessException,
                 InvocationTargetException{
-                    return m.invoke(null, null);
+                    return m.invoke(null, (Object[]) null);
                 }
             }
             );
@@ -192,4 +197,21 @@ public class ExceptionUtil
         return out.toString();
     }
 
+    /**
+     * Determine if the given {@code SQLState} string constant is a deferred
+     * constraint transactional error. If this is so, return {@code true}, else
+     * return {@code false}.
+     *
+     * @param e the string with the SQL state
+     * @return see method description
+     */
+    public static boolean isDeferredConstraintViolation(String e) {
+        return
+            e.equals(
+                ExceptionUtil.getSQLStateFromIdentifier(
+                    SQLState.LANG_DEFERRED_DUPLICATE_KEY_CONSTRAINT_T)) ||
+            e.equals(
+                ExceptionUtil.getSQLStateFromIdentifier(
+                    SQLState.LANG_DEFERRED_CHECK_CONSTRAINT_T));
+    }
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java Tue Mar 11 14:46:15 2014
@@ -185,8 +185,13 @@ public interface JoinStrategy {
 	 * @param bulkFetch True means bulk fetch is being done on the inner table
 	 * @param multiprobe True means we are probing the inner table for rows
 	 *  matching a specified list of values.
+     * @param validatingCheckConstraint True of this is a special scan to
+     *        validate a check constraint.
 	 */
-	String resultSetMethodName(boolean bulkFetch, boolean multiprobe);
+    String resultSetMethodName(
+        boolean bulkFetch,
+        boolean multiprobe,
+        boolean validatingCheckConstraint);
 
 	/**
 	 * Get the name of the join result set method for the join

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=1576367&r1=1576366&r2=1576367&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 Mar 11 14:46:15 2014
@@ -48,6 +48,7 @@ import org.apache.derby.iapi.sql.execute
 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;
 
 /**
  * LanguageConnectionContext keeps the result sets,
@@ -1277,7 +1278,8 @@ public interface LanguageConnectionConte
                                        FormatableBitSet map);
 
     /**
-     * Set the constraint mode for this constraint/index to {@code deferred}.
+     * Set the constraint mode for this primary key or unique constraint to
+     * {@code deferred}.
      * If {@code deferred} is {@code false}, to immediate checking,
      * if {@code true} to deferred checking.
      *
@@ -1287,8 +1289,25 @@ public interface LanguageConnectionConte
      * @param deferred  The new constraint mode
      * @throws StandardException
      */
-    public void setDeferred(Activation a, long conglomId, boolean deferred)
-            throws StandardException;
+    public void setConstraintDeferred(Activation a,
+                                    long conglomId,
+                                    boolean deferred) throws StandardException;
+
+    /**
+     * Set the constraint mode for this check constraint to {@code deferred}.
+     * If {@code deferred} is {@code false}, to immediate checking,
+     * if {@code true} to deferred checking.
+     *
+     * @param a         Activation
+     * @param baseTableCID conglomerate id of constraint's base table
+     * @param constraintId The constraint id
+     * @param deferred  The new constraint mode
+     * @throws StandardException
+     */
+    public void setConstraintDeferred(Activation a,
+                                 long baseTableCID,
+                                 UUID constraintId,
+                                 boolean deferred) throws StandardException;
 
     /**
      * Get the constraint mode set, if any.
@@ -1304,6 +1323,19 @@ public interface LanguageConnectionConte
             throws StandardException;
 
     /**
+     * Get the constraint mode set, if any.
+     *
+     * @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
+     */
+    public boolean isEffectivelyDeferred(Activation a, UUID constraintId)
+            throws StandardException;
+
+    /**
      * Set the constraint mode for all deferrable constraints to
      * {@code deferred}.
      * If {@code deferred} is {@code false}, to immediate checking,
@@ -1316,12 +1348,15 @@ public interface LanguageConnectionConte
             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.
+     * Get the set of disk backed hash tables containing any index
+     * rows saved for deferred unique/PK constraints in this
+     * transaction, keyed by the conglomerate id, or rows saved
+     * containing row locations violating rows for deferred check
+     * constraints.
      * @return the set
      */
-    HashMap<Long, BackingStoreHashtable> getDeferredHashTables();
+    HashMap<Long, DeferredConstraintsMemory.ValidationInfo>
+        getDeferredHashTables();
 
     /**
      * Check that deferred constraints are valid, if not roll back the
@@ -1332,13 +1367,13 @@ public interface LanguageConnectionConte
     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
+     * Forget any violating rows for the deferred constraint associated
+     * by conglomId,
+     * @param conglomId The conglomerate identifier of the backing
+     *        index, or a base table conglomerate id of it is a CHECK
+     *        constraint.
      * @throws StandardException
      */
-    public void invalidateDeferredConstraintsData(long indexCID)
+    public void forgetDeferredConstraintsData(long conglomId)
             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=1576367&r1=1576366&r2=1576367&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 Tue Mar 11 14:46:15 2014
@@ -23,6 +23,7 @@ package org.apache.derby.iapi.sql.conn;
 
 import java.lang.String;
 import java.util.HashMap;
+import org.apache.derby.catalog.UUID;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
 
 /**
@@ -104,7 +105,14 @@ public interface SQLSessionContext {
      * The caller is responsible for any cloning needed.
      * @return constraint modes map
      */
-    public HashMap<Long, Boolean> getConstraintModes();
+    public HashMap<Long, Boolean> getUniquePKConstraintModes();
+
+    /**
+     * Get a handle to the session's check constraint modes.
+     * The caller is responsible for any cloning needed.
+     * @return constraint modes map
+     */
+    public HashMap<UUID, Boolean> getCheckConstraintModes();
 
     /**
      * Initialize a inferior session context with the constraint mode map
@@ -114,6 +122,13 @@ public interface SQLSessionContext {
     public void setConstraintModes(HashMap<Long, Boolean> hm);
 
     /**
+     * Initialize a inferior session context with the check constraint mode map
+     * of the parent session context.
+     * @param hm constraint mode map
+     */
+    public void setCheckConstraintModes(HashMap<UUID, 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.
@@ -125,6 +140,16 @@ public interface SQLSessionContext {
     public void setDeferred(long conglomId, boolean deferred);
 
     /**
+     * Set the constraint mode for this constraint to {@code deferred}.
+     * If {@code deferred} is {@code false}, to immediate checking,
+     * if {@code true} to deferred checking.
+     *
+     * @param constraintId The constraint id
+     * @param deferred  The new constraint mode
+     */
+    public void setDeferred(UUID constraintId, 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.
@@ -135,6 +160,21 @@ public interface SQLSessionContext {
     public Boolean isDeferred(long conglomId);
 
     /**
+     * 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.
+     *
+     * @param constraintId the constraint id
+     * @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.
+     */
+    public Boolean isDeferred(UUID constraintId);
+
+    /**
      * Clear deferred information for this transaction.
      */
     public void resetConstraintModes();

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=1576367&r1=1576366&r2=1576367&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 Tue Mar 11 14:46:15 2014
@@ -733,12 +733,15 @@ public abstract class ConstraintDescript
             // information since they point to the same physical index.
             for (ConglomerateDescriptor cd : conglomDescs) {
                 if (cd.isConstraint()) {
-                    lcc.invalidateDeferredConstraintsData(
+                    lcc.forgetDeferredConstraintsData(
                             cd.getConglomerateNumber());
                     newBackingConglomCD = cd.drop(lcc, table);
                     break;
                 }
             }
+        } else {
+            lcc.forgetDeferredConstraintsData(
+                    getTableDescriptor().getHeapConglomerateId());
         }
 
         table.removeConstraintDescriptor(this);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/NoPutResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/NoPutResultSet.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/NoPutResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/NoPutResultSet.java Tue Mar 11 14:46:15 2014
@@ -137,6 +137,12 @@ public interface NoPutResultSet extends 
 	 */
 	public void setNeedsRowLocation(boolean needsRowLocation);
 
+    /**
+     * Set that we are acting on behalf of an insert result set that has
+     * deferrable check constraints
+     */
+    public void setHasDeferrableChecks();
+
 	/**
 	 * Get the estimated row count from this result set.
 	 *

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java Tue Mar 11 14:46:15 2014
@@ -115,11 +115,15 @@ public interface ResultSetFactory {
 		@param checkGM	The code to enforce the check constraints, if any
         @param fullTemplate Saved item for a row template used by bulk insert,
                             or -1 if this is not a bulk insert
+        @param schemaNameName schema name of table
+        @param tableName table name
 		@return the insert operation as a result set.
 		@exception StandardException thrown when unable to perform the insert
 	 */
 	ResultSet getInsertResultSet(NoPutResultSet source, GeneratedMethod generationClauses,
-                                 GeneratedMethod checkGM, int fullTemplate)
+                                 GeneratedMethod checkGM, int fullTemplate,
+                                 String schemaNameName,
+                                 String tableName)
         throws StandardException;
 
 	/**
@@ -313,6 +317,10 @@ public interface ResultSetFactory {
         @param cloneMapItem Item # for columns that need cloning
         @param reuseResult  Whether or not to reuse the result row.
 		@param doesProjection	Whether or not this PRN does a projection
+        @param validatingCheckConstraint {@code true if this PRN is used to
+            for validating a deferred check constraint}.
+        @param validatingBaseTableCID The conglomerate id for the table being
+            validated.
 		@param optimizerEstimatedRowCount	Estimated total # of rows by
 											optimizer
 		@param optimizerEstimatedCost		Estimated total cost by optimizer
@@ -328,6 +336,8 @@ public interface ResultSetFactory {
         int cloneMapItem,
 		boolean reuseResult,
 		boolean doesProjection,
+        boolean validatingCheckConstraint,
+        long validatingBaseTableCID,
 		double optimizerEstimatedRowCount,
 		double optimizerEstimatedCost) throws StandardException;
 
@@ -891,6 +901,33 @@ public interface ResultSetFactory {
 		@exception StandardException thrown when unable to create the
 			result set
 	 */
+    NoPutResultSet getValidateCheckConstraintResultSet(
+                                Activation activation,
+                                long conglomId,
+                                int scociItem,
+                                int resultRowTemplate,
+                                int resultSetNumber,
+                                GeneratedMethod startKeyGetter,
+                                int startSearchOperator,
+                                GeneratedMethod stopKeyGetter,
+                                int stopSearchOperator,
+                                boolean sameStartStopPosition,
+                                Qualifier[][] qualifiers,
+                                String tableName,
+                                String userSuppliedOptimizerOverrides,
+                                String indexName,
+                                boolean isConstraint,
+                                boolean forUpdate,
+                                int colRefItem,
+                                int indexColItem,
+                                int lockMode,
+                                boolean tableLocked,
+                                int isolationLevel,
+                                boolean oneRowScan,
+                                double optimizerEstimatedRowCount,
+                                double optimizerEstimatedCost)
+            throws StandardException;
+
 	NoPutResultSet getTableScanResultSet(
 			                    Activation activation,
 								long conglomId,
@@ -918,7 +955,7 @@ public interface ResultSetFactory {
 								double optimizerEstimatedCost)
 			throws StandardException;
 
-	/**
+    /**
 		A table scan result set forms a result set on a scan
 		of a table.
 		The rows can be constructed as they are requested from the

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/RowChanger.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/RowChanger.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/RowChanger.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/RowChanger.java Tue Mar 11 14:46:15 2014
@@ -91,9 +91,11 @@ public interface RowChanger
 	  Insert a row into the table and perform associated index maintenance.
 
 	  @param baseRow the row.
+     * @param getRL return a row location of the inserted row
+     * @return row location if requested, else {@code null}.
 	  @exception StandardException		Thrown on error
 	  */
-	public void insertRow(ExecRow baseRow)
+    public RowLocation insertRow(ExecRow baseRow, boolean getRL)
 		 throws StandardException;
 		
 	/**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/TargetResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/TargetResultSet.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/TargetResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/TargetResultSet.java Tue Mar 11 14:46:15 2014
@@ -49,6 +49,8 @@ public interface TargetResultSet extends
 	 */
 	public void changedRow(ExecRow execRow, RowLocation rowLocation) throws StandardException;
 
+    public void offendingRowLocation(
+            RowLocation rl, long constainerId) throws StandardException;
 	/**
 	 * Preprocess the source row prior to getting it back from the source.
 	 * This is useful for bulk insert where the store stands between the target and 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java Tue Mar 11 14:46:15 2014
@@ -1000,12 +1000,13 @@ public class BackingStoreHashtable
      * "needsToClone" is false then the hash table will keep a reference to
      * the row passed in and no copy will be made.
      * <p>
-     * If rouine returns false, then no reference is kept to the duplicate
+     * If routine returns false, then no reference is kept to the duplicate
      * row which was rejected (thus allowing caller to reuse the object).
      *
      * @param needsToClone does this routine have to make a copy of the row,
      *                     in order to keep a reference to it after return?
      * @param row          The row to insert into the table.
+     * @param rowLocation  Location of row in conglomerate; could be null.
      *
 	 * @return true if row was inserted into the hash table.  Returns
      *              false if the BackingStoreHashtable is eliminating 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/RowLocationRetRowSource.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/RowLocationRetRowSource.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/RowLocationRetRowSource.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/RowLocationRetRowSource.java Tue Mar 11 14:46:15 2014
@@ -50,6 +50,8 @@ public interface RowLocationRetRowSource
 	 */
 	boolean needsRowLocation();
 
+    boolean needsRowLocationForDeferredCheckConstraints();
+
 	/**
 		rowLocation is a callback for the drainer of the row source to return
 		the rowLocation of the current row, i.e, the row that is being returned
@@ -72,7 +74,7 @@ public interface RowLocationRetRowSource
 
 		NeedsRowLocation and rowLocation will ONLY be called by a drainer of
 		the row source which CAN return a row location.  Drainer of row source
-		which cannot return rowLocation will guarentee to not call either
+        which cannot return rowLocation will guarantee to not call either
 		callbacks. Conversely, if NeedsRowLocation is called and it returns
 		true, then for every row return by getNextRowFromRowSource, a
 		rowLocation callback must also be issued with the row location of the
@@ -87,4 +89,7 @@ public interface RowLocationRetRowSource
 		@exception StandardException on error
 	 */
 	void rowLocation(RowLocation rl) throws StandardException;
+
+    void offendingRowLocation(
+            RowLocation rl, long containdId) throws StandardException;
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/ScanController.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/ScanController.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/ScanController.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/ScanController.java Tue Mar 11 14:46:15 2014
@@ -189,7 +189,7 @@ public interface ScanController extends 
 	fetch must be compatible with the number of scan columns
 	requested at the openScan call time.
 	<BR>
-	A fetch can return a sub-set of the scan columns reqested
+    A fetch can return a sub-set of the scan columns requested
 	at scan open time by supplying a destRow will less elements
 	than the number of requested columns. In this case the N leftmost
 	of the requested columns are fetched, where N = destRow.length.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/BooleanDataValue.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/BooleanDataValue.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/BooleanDataValue.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/BooleanDataValue.java Tue Mar 11 14:46:15 2014
@@ -22,6 +22,7 @@
 package org.apache.derby.iapi.types;
 
 import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.sql.Activation;
 
 public interface BooleanDataValue extends DataValueDescriptor
 {
@@ -103,7 +104,34 @@ public interface BooleanDataValue extend
 									String constraintName)
 							throws StandardException;
 
-	/*
+    /**
+     * If this value is false and we have a deferred constraint, remember the
+     * violation and proceed, else throw.  See also
+     * {@link org.apache.derby.impl.sql.compile.AndNoShortCircuitNode}.
+     *
+     * @param SQLState      The SQLState of the exception to throw if
+     *                      this SQLBoolean is false.
+     * @param tableName     The name of the table to include in the exception
+     *                      message.
+     * @param constraintName    The name of the failed constraint to include
+     *                          in the exception message.
+     * @param a             The activation
+     * @param savedUUIDIdx  The saved object number of the constraint's UUID.
+     *
+     * @return  this
+     *
+     * @exception   StandardException   Thrown if this BooleanDataValue
+     *                                  is false.
+     */
+    public BooleanDataValue throwExceptionIfImmediateAndFalse(
+                                    String SQLState,
+                                    String tableName,
+                                    String constraintName,
+                                    Activation a,
+                                    int savedUUIDIdx)
+                            throws StandardException;
+
+    /*
 	** NOTE: The NOT operator is translated to "= FALSE", which does the same
 	** thing.
 	*/

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=1576367&r1=1576366&r2=1576367&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 Mar 11 14:46:15 2014
@@ -40,6 +40,12 @@ import java.math.BigDecimal;
 import java.sql.ResultSet;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
+import org.apache.derby.catalog.UUID;
+import org.apache.derby.iapi.sql.Activation;
+import org.apache.derby.iapi.sql.Row;
+import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
+import org.apache.derby.iapi.sql.execute.ExecPreparedStatement;
+import org.apache.derby.impl.sql.execute.DMLWriteResultSet;
 
 /**
  * SQLBoolean satisfies the DataValueDescriptor
@@ -866,6 +872,35 @@ public final class SQLBoolean
 		return this;
 	}
 
+    public BooleanDataValue throwExceptionIfImmediateAndFalse(
+                                    String sqlState,
+                                    String tableName,
+                                    String constraintName,
+                                    Activation a,
+                                    int savedUUIDIdx)
+                            throws StandardException
+    {
+        if ( !isNull() && (value == false) ) {
+            final ExecPreparedStatement ps = a.getPreparedStatement();
+            final UUID constrId = (UUID)ps.getSavedObject(savedUUIDIdx);
+            final LanguageConnectionContext lcc =
+                a.getLanguageConnectionContext();
+            final boolean isDeferred = lcc.isEffectivelyDeferred(a, constrId);
+
+            if (!isDeferred) {
+                throw StandardException.newException(
+                        sqlState, tableName, constraintName);
+            } else {
+                // Just return the false value and validate later,
+                // cf NoRowsResultSetImpl#evaluateCheckConstraints.
+                // and InsertResultSet#evaluateCheckConstraints
+                DMLWriteResultSet rs =  (DMLWriteResultSet)a.getResultSet();
+                rs.rememberConstraint(constrId);
+            }
+        }
+
+        return this;
+    }
 	/*
 	 * DataValueDescriptor interface
 	 */

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/AndNoShortCircuitNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/AndNoShortCircuitNode.java?rev=1576367&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/AndNoShortCircuitNode.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/AndNoShortCircuitNode.java Tue Mar 11 14:46:15 2014
@@ -0,0 +1,87 @@
+/*
+
+   Derby - Class org.apache.derby.impl.sql.compile.AndNoShortCircuitNode
+
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to you under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+ */
+
+package org.apache.derby.impl.sql.compile;
+
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.reference.ClassName;
+import org.apache.derby.iapi.services.classfile.VMOpcode;
+import org.apache.derby.iapi.services.compiler.MethodBuilder;
+import org.apache.derby.iapi.services.context.ContextManager;
+
+/**
+ * Used for deferrable CHECK constraint. When we evaluate check constraints for
+ * a row where at least one constraint is deferrable, we need to know exactly
+ * which set of constraints violated the checks.  The normal evaluation of
+ * check constraints is generated as one big (NOT c1) AND (NOT c2) AND ...  AND
+ * (NOT cn) using short-circuited (McCarthy) boolean evaluation.
+ * <p>
+ * This kind of evaluation of the expression can only tell us the first failing
+ * constraint, so we use full evaluation instead, as embodied in this node.
+ * See also {@link
+ * org.apache.derby.iapi.types.BooleanDataValue#throwExceptionIfImmediateAndFalse}.
+ */
+class AndNoShortCircuitNode extends AndNode
+{
+    /**
+     * @param leftOperand The left operand of the AND
+     * @param rightOperand The right operand of the AND
+     * @param cm context manager
+     * @throws StandardException standard error policy
+     */
+    AndNoShortCircuitNode(
+            ValueNode leftOperand,
+            ValueNode rightOperand,
+            ContextManager cm) throws StandardException {
+        super(leftOperand, rightOperand, "andnoshortcircuitnode", cm);
+    }
+
+    /**
+     * Generate code for no short-circuiting AND operator. Used to evaluate
+     * check constraints where at least one is deferrable, since we need to
+     * know exactly which constraint(s) violated the checks.
+     * @throws StandardException standard error policy
+     */
+    @Override
+    void generateExpression(ExpressionClassBuilder acb, MethodBuilder mb)
+        throws StandardException
+    {
+        /*
+        ** This generates the following code:
+        **
+        **   <leftOperand>.<and/or>(<rightOperand>)
+        **/
+        leftOperand.generateExpression(acb, mb);
+        // stack - left
+        mb.upCast(ClassName.BooleanDataValue);
+
+        rightOperand.generateExpression(acb, mb);
+        mb.upCast(ClassName.BooleanDataValue);
+
+        // stack - left, right
+        mb.callMethod(VMOpcode.INVOKEINTERFACE,
+                      (String) null,
+                      "and",
+                      ClassName.BooleanDataValue,
+                      1);
+        // stack - result
+    }
+}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/AndNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/AndNode.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/AndNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/AndNode.java Tue Mar 11 14:46:15 2014
@@ -43,6 +43,21 @@ class AndNode extends BinaryLogicalOpera
     }
 
     /**
+     * @param leftOperand The left operand of the AND
+     * @param rightOperand The right operand of the AND
+     * @param methodName The methods name
+     * @param cm context manager
+     * @throws StandardException
+     */
+    AndNode(ValueNode leftOperand,
+            ValueNode rightOperand,
+            String  methodName,
+            ContextManager cm) throws StandardException {
+        super(leftOperand, rightOperand, methodName, cm);
+        this.shortCircuitValue = false;
+    }
+
+    /**
 	 * Bind this logical operator.  All that has to be done for binding
 	 * a logical operator is to bind the operands, check that both operands
 	 * are BooleanDataValue, and set the result type to BooleanDataValue.

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=1576367&r1=1576366&r2=1576367&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 Mar 11 14:46:15 2014
@@ -229,7 +229,9 @@ abstract class DMLModStatementNode exten
 					throw StandardException.newException(SQLState.LANG_TABLE_NOT_FOUND, targetTableName);
 			}
 			
-			switch (targetTableDescriptor.getTableType())
+            targetTableName.setSchemaName(sdtc.getSchemaName());
+
+            switch (targetTableDescriptor.getTableType())
 			{
 			case TableDescriptor.VIEW_TYPE:
 				// Views are currently not updatable
@@ -616,6 +618,9 @@ abstract class DMLModStatementNode exten
 	 *								of 1-based column ids for columns being changed
 	 * @param readColsBitSet		bit set for the read scan
 	 * @param includeTriggers		whether triggers are included in the processing
+     * @param hasDeferrableCheckConstraints
+     *                        OUT semantics: set element 0 to true if the
+     *                        target table has any deferrable CHECK constraints
 	 *
 	 * @return	The bound, ANDed check constraints as a query tree.
 	 *
@@ -630,7 +635,8 @@ abstract class DMLModStatementNode exten
 		ResultColumnList	sourceRCL,
 		int[]				changedColumnIds,
 		FormatableBitSet				readColsBitSet,
-		boolean 			includeTriggers
+        boolean             includeTriggers,
+        boolean[]           hasDeferrableCheckConstraints
     )
 		throws StandardException
 	{
@@ -658,8 +664,10 @@ abstract class DMLModStatementNode exten
 			createTriggerDependencies(relevantTriggers, dependent);
             generateTriggerInfo(relevantTriggers);
 
-			checkConstraints = generateCheckTree(relevantCdl,
-														targetTableDescriptor);
+            checkConstraints = generateCheckTree(
+                    relevantCdl,
+                    targetTableDescriptor,
+                    hasDeferrableCheckConstraints);
 
             if (checkConstraints != null)
 			{
@@ -810,7 +818,8 @@ abstract class DMLModStatementNode exten
 	private	ValueNode generateCheckTree
 	(
 		ConstraintDescriptorList	cdl,
-		TableDescriptor				td
+        TableDescriptor             td,
+        boolean[]                   hasDeferrable
     )
 		throws StandardException
 	{
@@ -818,6 +827,14 @@ abstract class DMLModStatementNode exten
 		int							ccCDLSize = ccCDL.size();
 		ValueNode					checkTree = null;
 
+        for (ConstraintDescriptor cd : ccCDL) {
+            if (cd.deferrable()) {
+                hasDeferrable[0] = true;
+                break;
+            }
+        }
+
+
 		// Get the text of all the check constraints
 		for (int index = 0; index < ccCDLSize; index++)
 		{
@@ -834,7 +851,7 @@ abstract class DMLModStatementNode exten
 					oneConstraint,
 					SQLState.LANG_CHECK_CONSTRAINT_VIOLATED,
 					td.getQualifiedName(),
-					cd.getConstraintName(),
+                    cd,
 					getContextManager());
 					
 			// Link consecutive TestConstraintNodes with AND nodes
@@ -844,7 +861,12 @@ abstract class DMLModStatementNode exten
 			}
 			else
 			{
-               checkTree = new AndNode(tcn, checkTree, getContextManager());
+               if (hasDeferrable[0]) {
+                   checkTree = new AndNoShortCircuitNode(
+                       tcn, checkTree, getContextManager());
+               } else {
+                   checkTree = new AndNode(tcn, checkTree, getContextManager());
+               }
 			}
 		}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java Tue Mar 11 14:46:15 2014
@@ -307,15 +307,15 @@ class DeleteNode extends DMLModStatement
 			{
 				/* Bind the new ResultColumn */
 				rowLocationColumn.bindResultColumnToExpression();
-
 				bindConstraints(dataDictionary,
-                            getOptimizerFactory(),
-							targetTableDescriptor,
-							null,
-							resultColumnList,
-							(int[]) null,
-							readColsBitSet,
-							true);  /* we alway include triggers in core language */
+                        getOptimizerFactory(),
+                        targetTableDescriptor,
+                        null,
+                        resultColumnList,
+                        (int[]) null,
+                        readColsBitSet,
+                        true, // we alway include triggers in core language
+                        new boolean[1]); // dummy
 
 				/* If the target table is also a source table, then
 			 	* the delete will have to be in deferred mode

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java Tue Mar 11 14:46:15 2014
@@ -133,6 +133,14 @@ class FromBaseTable extends FromTable
 	*/
 	int 			bulkFetch = UNSET;
 
+    /*
+    ** Used to validate deferred check constraints.
+    ** It is the conglomerate number of the target inserted into or updated
+    ** when a violation was detected but deferred.
+    */
+    private long            targetTableCID;
+    private boolean         validatingCheckConstraint = false;
+
 	/* We may turn off bulk fetch for a variety of reasons,
 	 * including because of the min optimization.  
 	 * bulkFetchTurnedOff is set to true in those cases.
@@ -824,6 +832,10 @@ class FromBaseTable extends FromTable
 					throw StandardException.newException(SQLState.LANG_INVALID_BULK_FETCH_UPDATEABLE);
 				}
 			}
+            else if (key.equals("validateCheckConstraint")) {
+                // the property "validateCheckConstraint" is read earlier
+                // cf. isValidatingCheckConstraint
+            }
 			else
 			{
 				// No other "legal" values at this time
@@ -854,6 +866,23 @@ class FromBaseTable extends FromTable
 		}
 	}
 
+    private boolean isValidatingCheckConstraint() throws StandardException {
+        if (tableProperties == null) {
+            return false;
+        }
+
+        for (Enumeration<?> e = tableProperties.keys(); e.hasMoreElements();) {
+            String key = (String)e.nextElement();
+            String value = (String) tableProperties.get(key);
+            if (key.equals("validateCheckConstraint")) {
+                targetTableCID = getLongProperty(value, key);
+                validatingCheckConstraint = true;
+                return true;
+            }
+        }
+        return false;
+    }
+
 	/** @see org.apache.derby.iapi.sql.compile.Optimizable#getBaseTableName */
     @Override
 	public String getBaseTableName()
@@ -2944,7 +2973,8 @@ class FromBaseTable extends FromTable
 		prRCList.doProjection();
 
 		/* Finally, we create the new ProjectRestrictNode */
-        return new ProjectRestrictNode(
+        ProjectRestrictNode result =
+                new ProjectRestrictNode(
 								this,
 								prRCList,
 								null,	/* Restriction */
@@ -2953,6 +2983,21 @@ class FromBaseTable extends FromTable
 								null,	/* Restrict subquery list */
 								null,
                                 getContextManager());
+
+        if (isValidatingCheckConstraint()) {
+            CompilerContext cc = getCompilerContext();
+
+            if ((cc.getReliability() &
+                 // Internal feature: throw if used on app level
+                 CompilerContext.INTERNAL_SQL_ILLEGAL) != 0) {
+
+                throw StandardException.newException(
+                        SQLState.LANG_SYNTAX_ERROR, "validateCheckConstraint");
+            }
+
+            result.setValidatingCheckConstraints(targetTableCID);
+        }
+        return result;
 	}
 
 	/**
@@ -3073,7 +3118,8 @@ class FromBaseTable extends FromTable
 			(bulkFetch == UNSET) && 
 			!forUpdate() && 
 			!isOneRowResultSet() &&
-			getLevel() == 0)
+            getLevel() == 0 &&
+            !validatingCheckConstraint)
 		{
 			bulkFetch = getDefaultBulkFetch();	
 		}
@@ -3448,7 +3494,7 @@ class FromBaseTable extends FromTable
 
 		mb.callMethod(VMOpcode.INVOKEINTERFACE, (String) null,
 			trulyTheBestJoinStrategy.resultSetMethodName(
-				(bulkFetch != UNSET), multiProbing),
+                (bulkFetch != UNSET), multiProbing, validatingCheckConstraint),
 			ClassName.NoPutResultSet, nargs);
 
 		/* If this table is the target of an update or a delete, then we must 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java Tue Mar 11 14:46:15 2014
@@ -292,7 +292,10 @@ class HashJoinStrategy extends BaseJoinS
 
 
 	/** @see JoinStrategy#resultSetMethodName */
-	public String resultSetMethodName(boolean bulkFetch, boolean multiprobe) {
+    public String resultSetMethodName(
+            boolean bulkFetch,
+            boolean multiprobe,
+            boolean validatingCheckConstraint) {
 		return "getHashScanResultSet";
 	}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java Tue Mar 11 14:46:15 2014
@@ -78,6 +78,7 @@ public final class InsertNode extends DM
     private     ResultColumnList    targetColumnList;
     private     boolean             deferred;
 	public		ValueNode			checkConstraints;
+    public      boolean             hasDeferrableCheckConstraints;
 	public		Properties			targetProperties;
 	public		FKInfo				fkInfo;
 	protected	boolean				bulkInsert;
@@ -494,16 +495,21 @@ public final class InsertNode extends DM
                 ( dataDictionary, targetTableDescriptor, sourceRCL, resultColumnList, false, null );
             
 			/* Get and bind all constraints on the table */
-			checkConstraints = bindConstraints(dataDictionary,
-                                                getOptimizerFactory(),
-												targetTableDescriptor,
-												null,
-												sourceRCL,
-												(int[]) null,
-												(FormatableBitSet) null,
-											    true);  /* we always include
-														 * triggers in core language */
-	
+            boolean[] hasDCC = new boolean[]{false /* a priori*/ };
+
+            checkConstraints = bindConstraints(
+                    dataDictionary,
+                    getOptimizerFactory(),
+                    targetTableDescriptor,
+                    null,
+                    sourceRCL,
+                    (int[]) null,
+                    (FormatableBitSet) null,
+                    true, // we always include triggers in core language
+                    hasDCC);
+
+            hasDeferrableCheckConstraints = hasDCC[0];
+
 			/* Do we need to do a deferred mode insert */
 			/* 
 		 	** Deferred if:
@@ -791,6 +797,7 @@ public final class InsertNode extends DM
 				  indexNames,
 				  deferred,
 				  false,
+                  hasDeferrableCheckConstraints,
 				  targetTableDescriptor.getUUID(),
 				  lockMode,
 				  null, null, 
@@ -971,8 +978,17 @@ public final class InsertNode extends DM
                 mb.push(-1);
             }
 
+            // arg 5, 6 table name
+            if (targetTableName.getSchemaName() == null) {
+                mb.pushNull("java.lang.String");
+            } else {
+                mb.push(targetTableName.getSchemaName());
+            }
+
+            mb.push(targetTableName.getTableName());
+
             mb.callMethod(VMOpcode.INVOKEINTERFACE, (String) null,
-                    "getInsertResultSet", ClassName.ResultSet, 4);
+                    "getInsertResultSet", ClassName.ResultSet, 6);
 		}
 		else
 		{

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java Tue Mar 11 14:46:15 2014
@@ -38,6 +38,7 @@ import org.apache.derby.iapi.store.acces
 
 class NestedLoopJoinStrategy extends BaseJoinStrategy {
     NestedLoopJoinStrategy() {
+        int i = 3;
 	}
 
 
@@ -167,13 +168,20 @@ class NestedLoopJoinStrategy extends Bas
     public  String  getOperatorSymbol() { return "*"; }
 
 	/** @see JoinStrategy#resultSetMethodName */
-	public String resultSetMethodName(boolean bulkFetch, boolean multiprobe) {
-		if (bulkFetch)
+    public String resultSetMethodName(
+            boolean bulkFetch,
+            boolean multiprobe,
+            boolean validatingCheckConstraint) {
+
+        if (validatingCheckConstraint) {
+            return "getValidateCheckConstraintResultSet";
+        } else if (bulkFetch) {
 			return "getBulkTableScanResultSet";
-		else if (multiprobe)
+        } else if (multiprobe) {
 			return "getMultiProbeTableScanResultSet";
-		else
+        } else {
 			return "getTableScanResultSet";
+        }
 	}
 
 	/** @see JoinStrategy#joinResultSetMethodName */

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ProjectRestrictNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ProjectRestrictNode.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ProjectRestrictNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ProjectRestrictNode.java Tue Mar 11 14:46:15 2014
@@ -89,6 +89,12 @@ class ProjectRestrictNode extends Single
 	 */
 	private boolean getTableNumberHere;
 
+    /**
+     * Used with {@code validatingBaseTableCID} to validating deferred check
+     * constraints.
+     */
+    private boolean validatingCheckConstraints = false;
+    private long validatingBaseTableCID;
 	/**
      * Constructor for a ProjectRestrictNode.
 	 *
@@ -698,7 +704,10 @@ class ProjectRestrictNode extends Single
 			(trulyTheBestAccessPath.getJoinStrategy() != null) &&
 			trulyTheBestAccessPath.getJoinStrategy().isHashJoin();
 
-		if ((restrictionList != null) && !alreadyPushed && !hashJoinWithThisPRN)
+        if ((restrictionList != null) &&
+            !alreadyPushed &&
+            !hashJoinWithThisPRN &&
+            !validatingCheckConstraints)
 		{
 			restrictionList.pushUsefulPredicates((Optimizable) childResult);
 		}
@@ -1577,11 +1586,13 @@ class ProjectRestrictNode extends Single
         mb.push(cloneMapItem);
 		mb.push(getResultColumns().reusableResult());
 		mb.push(doesProjection);
+        mb.push(validatingCheckConstraints);
+        mb.push(validatingBaseTableCID);
 		mb.push(getCostEstimate().rowCount());
 		mb.push(getCostEstimate().getEstimatedCost());
 
 		mb.callMethod(VMOpcode.INVOKEINTERFACE, (String) null, "getProjectRestrictResultSet",
-                    ClassName.NoPutResultSet, 11);
+                    ClassName.NoPutResultSet, 13);
 	}
 
 	/**
@@ -1885,4 +1896,9 @@ class ProjectRestrictNode extends Single
     {
         childResult.pushOffsetFetchFirst( offset, fetchFirst, hasJDBClimitClause );
     }
+
+    void setValidatingCheckConstraints(long baseTableCID) {
+        validatingCheckConstraints = true;
+        validatingBaseTableCID = baseTableCID;
+    }
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/QueryTreeNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/QueryTreeNode.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/QueryTreeNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/QueryTreeNode.java Tue Mar 11 14:46:15 2014
@@ -760,6 +760,34 @@ public abstract class QueryTreeNode impl
 	}
 	
 	/**
+     * Get the long value of a Property
+     *
+     * @param value     Property value as a String
+     * @param key       Key value of property
+     *
+     * @return  The long value of the property
+     *
+     * @exception StandardException     Thrown on failure
+     */
+    protected long getLongProperty(String value, String key)
+        throws StandardException
+    {
+        long longVal = -1;
+        try
+        {
+            longVal = Long.parseLong(value);
+        }
+        catch (NumberFormatException nfe)
+        {
+            throw StandardException.newException(
+                SQLState.LANG_INVALID_NUMBER_FORMAT_FOR_OVERRIDE,
+                value,
+                key);
+        }
+        return longVal;
+    }
+
+    /**
 	** Parse the a SQL statement from the body
 	* of another SQL statement. Pushes and pops a
 	* separate CompilerContext to perform the compilation.

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=1576367&r1=1576366&r2=1576367&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 Tue Mar 11 14:46:15 2014
@@ -24,10 +24,10 @@ package     org.apache.derby.impl.sql.co
 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.Visitor;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
 import org.apache.derby.iapi.sql.execute.ConstantAction;
+import org.apache.derby.shared.common.sanity.SanityManager;
 
 /**
  * A SetConstraintsNode is the root of a QueryTree that represents a
@@ -54,7 +54,6 @@ class SetConstraintsNode extends Miscell
      * @param deferred    Encodes IMMEDIATE ({@code false}) or DEFERRED
      *                    ({@code true})
      * @param cm          The context manager
-     * @throws StandardException
      */
     SetConstraintsNode(
             List<TableName> constraints,
@@ -83,14 +82,22 @@ class SetConstraintsNode extends Miscell
     }
 
     String formatList(List<TableName> constraints) {
-        StringBuilder sb = new StringBuilder();
+        if (SanityManager.DEBUG) {
+            StringBuilder sb = new StringBuilder();
 
-        for (TableName tn : constraints) {
-            sb.append(tn);
-            sb.append(", ");
-        }
+            if (constraints == null) {
+                return "ALL";
+            } else {
+                for (TableName tn : constraints) {
+                    sb.append(tn);
+                    sb.append(", ");
+                }
 
-        return sb.substring(0, Math.max(0, sb.length() - 2));
+                return sb.substring(0, Math.max(0, sb.length() - 2));
+            }
+        } else {
+            return "";
+        }
     }
 
     public String statementToString()

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TestConstraintNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TestConstraintNode.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TestConstraintNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TestConstraintNode.java Tue Mar 11 14:46:15 2014
@@ -22,11 +22,13 @@
 package	org.apache.derby.impl.sql.compile;
 
 import java.util.List;
+import org.apache.derby.catalog.UUID;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.reference.ClassName;
 import org.apache.derby.iapi.services.classfile.VMOpcode;
 import org.apache.derby.iapi.services.compiler.MethodBuilder;
 import org.apache.derby.iapi.services.context.ContextManager;
+import org.apache.derby.iapi.sql.dictionary.ConstraintDescriptor;
 import org.apache.derby.iapi.types.DataTypeDescriptor;
 import org.apache.derby.iapi.types.TypeId;
 
@@ -38,16 +40,18 @@ import org.apache.derby.iapi.types.TypeI
 
 class TestConstraintNode extends UnaryLogicalOperatorNode
 {
-	private String sqlState;
-	private String tableName;
-	private String constraintName;
+    private final String sqlState;
+    private final String tableName;
+    private final UUID cid;
+    private final boolean deferrable;
+    private final String constraintName;
 
     /**
      * @param booleanValue The operand of the constraint test
      * @param sqlState The SQLState of the exception to throw if the
     *              constraint has failed
      * @param tableName The name of the table that the constraint is on
-     * @param constraintName The name of the constraint being checked
+     * @param cd The descriptor of the constraint being checked
      * @param cm context manager
      * @throws StandardException
      */
@@ -55,12 +59,18 @@ class TestConstraintNode extends UnaryLo
             ValueNode booleanValue,
             String sqlState,
             String tableName,
-            String constraintName,
+            ConstraintDescriptor cd,
             ContextManager cm) throws StandardException {
-        super(booleanValue, "throwExceptionIfFalse", cm);
+        super(booleanValue,
+                cd.deferrable() ?
+                        "throwExceptionIfImmediateAndFalse" :
+                        "throwExceptionIfFalse",
+                cm);
         this.sqlState = sqlState;
         this.tableName = tableName;
-        this.constraintName = constraintName;
+        this.cid = cd.getUUID();
+        this.deferrable = cd.deferrable();
+        this.constraintName = cd.getConstraintName();
     }
 
     /**
@@ -125,10 +135,21 @@ class TestConstraintNode extends UnaryLo
 
 		mb.push(sqlState);
 		mb.push(tableName);
-		mb.push(constraintName);
-
-		mb.callMethod(VMOpcode.INVOKEINTERFACE, ClassName.BooleanDataValue,
-				"throwExceptionIfFalse", ClassName.BooleanDataValue, 3);
+        mb.push(constraintName);
 
+        if (deferrable) {
+            acb.pushThisAsActivation(mb); // arg 4
+            mb.push(acb.addItem(cid)); // arg 5
+
+            mb.callMethod(
+                VMOpcode.INVOKEINTERFACE,
+                ClassName.BooleanDataValue,
+                "throwExceptionIfImmediateAndFalse",
+                ClassName.BooleanDataValue,
+                5);
+        } else {
+            mb.callMethod(VMOpcode.INVOKEINTERFACE, ClassName.BooleanDataValue,
+                    "throwExceptionIfFalse", ClassName.BooleanDataValue, 3);
+        }
 	}
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java?rev=1576367&r1=1576366&r2=1576367&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java Tue Mar 11 14:46:15 2014
@@ -611,14 +611,16 @@ public final class UpdateNode extends DM
                 ( dataDictionary, targetTableDescriptor, afterColumns, resultColumnList, true, resultSet );
 
             /* Get and bind all constraints on the columns being updated */
-            checkConstraints = bindConstraints( dataDictionary,
-                                                getOptimizerFactory(),
-                                                targetTableDescriptor,
-                                                null,
-                                                sourceRCL,
-                                                changedColumnIds,
-                                                readColsBitSet,
-                                                true); /* we always include triggers in core language */
+            checkConstraints = bindConstraints(
+                dataDictionary,
+                getOptimizerFactory(),
+                targetTableDescriptor,
+                null,
+                sourceRCL,
+                changedColumnIds,
+                readColsBitSet,
+                true, /* we always include triggers in core language */
+                new boolean[1]); // dummy
 
             /* If the target table is also a source table, then
              * the update will have to be in deferred mode
@@ -910,8 +912,7 @@ public final class UpdateNode extends DM
 
 
 		return	getGenericConstantActionFactory().getUpdateConstantAction
-			( heapConglomId,
-			  targetTableDescriptor.getTableType(),
+            ( targetTableDescriptor,
 			  tc.getStaticCompiledConglomInfo(heapConglomId),
 			  indicesToMaintain,
 			  indexConglomerateNumbers,