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/01/09 21:22:17 UTC

svn commit: r1556938 - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/sql/dictionary/ engine/org/apache/derby/impl/sql/compile/ engine/org/apache/derby/impl/sql/execute/ testing/org/apache/derbyTesting/functionTests/tests/lang/

Author: dag
Date: Thu Jan  9 20:22:16 2014
New Revision: 1556938

URL: http://svn.apache.org/r1556938
Log:
DERBY-6453 Remove dead code in InsertResultSet and flag skipCheckConstraints

Patch _*cleanup-misc-3*_ which removes dead code but inserts asserts
in sane mode if we should somehow end up with triggers in bulk insert
mode and also removes a boolean variable in some interfaces which was
always called with false: {{skipCheckConstraints}}.
It adds a new test case, {{CheckConstraintTest#testbulkInsert}}.


Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TableDescriptor.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/InsertNode.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/execute/InsertResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoRowsResultSetImpl.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/CheckConstraintTest.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TableDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TableDescriptor.java?rev=1556938&r1=1556937&r2=1556938&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TableDescriptor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TableDescriptor.java Thu Jan  9 20:22:16 2014
@@ -731,7 +731,6 @@ public class TableDescriptor extends Uni
 	  *	statement type and its list of updated columns.
 	  *
 	  *	@param	statementType			As defined in StatementType.
-	  * @param	skipCheckConstraints	Skip check constraints
 	  * @param	changedColumnIds		If null, all columns being changed, otherwise array
 	  *									of 1-based column ids for columns being changed
 	  *	@param	needsDeferredProcessing	IN/OUT. true if the statement already needs
@@ -745,7 +744,6 @@ public class TableDescriptor extends Uni
 	public	void	getAllRelevantConstraints
 	(
 		int							statementType,
-		boolean						skipCheckConstraints,
 		int[]						changedColumnIds,
 		boolean[]					needsDeferredProcessing,
 		ConstraintDescriptorList	relevantConstraints
@@ -769,12 +767,6 @@ public class TableDescriptor extends Uni
 		{
 			ConstraintDescriptor cd = cdl.elementAt(index);
 
-			if (skipCheckConstraints &&
-					(cd.getConstraintType() == DataDictionary.CHECK_CONSTRAINT))
-			{
-				continue;
-			}
-
 			/*
 			** For each constraint, figure out if it requires deferred processing.
 			** Note that we need to do this on constraints that don't

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=1556938&r1=1556937&r2=1556938&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 Thu Jan  9 20:22:16 2014
@@ -627,7 +627,6 @@ abstract class DMLModStatementNode exten
 		ResultColumnList	sourceRCL,
 		int[]				changedColumnIds,
 		FormatableBitSet				readColsBitSet,
-		boolean				skipCheckConstraints,
 		boolean 			includeTriggers
     )
 		throws StandardException
@@ -647,7 +646,6 @@ abstract class DMLModStatementNode exten
 		try {
 			getAllRelevantConstraints(dataDictionary, 	
 											targetTableDescriptor, 
-											skipCheckConstraints,
 											changedColumnIds);
 			createConstraintDependencies(dataDictionary, relevantCdl, dependent);
 			generateFKInfo(relevantCdl, dataDictionary, targetTableDescriptor, readColsBitSet);
@@ -657,11 +655,6 @@ abstract class DMLModStatementNode exten
 			createTriggerDependencies(relevantTriggers, dependent);
             generateTriggerInfo(relevantTriggers);
 
-			if (skipCheckConstraints)
-			{
-				return null;
-			}
-
 			checkConstraints = generateCheckTree(relevantCdl,
 														targetTableDescriptor);
 
@@ -804,7 +797,7 @@ abstract class DMLModStatementNode exten
 	 *
 	 * Makes the calling object (usually a Statement) dependent on all the constraints.
 	 *
-	 * @param cdl				The constriant descriptor list
+     * @param cdl               The constraint descriptor list
 	 * @param td				The TableDescriptor
 	 *
 	 * @return	The ANDing of all appropriate check constraints as a query tree.
@@ -1286,7 +1279,6 @@ abstract class DMLModStatementNode exten
 	 *
 	 * @param dd				The DataDictionary
 	 * @param td				The TableDescriptor
-	 * @param skipCheckConstraints Skip check constraints
 	 * @param changedColumnIds	If null, all columns being changed, otherwise array
 	 *							of 1-based column ids for columns being changed
 	 *
@@ -1298,7 +1290,6 @@ abstract class DMLModStatementNode exten
 	(
 		DataDictionary		dd, 
 		TableDescriptor		td,
-		boolean				skipCheckConstraints,
 		int[]				changedColumnIds
     )
 		throws StandardException
@@ -1310,7 +1301,7 @@ abstract class DMLModStatementNode exten
 
 		needsDeferredProcessing[0] = requiresDeferredProcessing;
 		td.getAllRelevantConstraints
-			( statementType, skipCheckConstraints, changedColumnIds,
+            ( statementType, changedColumnIds,
 			  needsDeferredProcessing, relevantCdl );
 
 		adjustDeferredFlag( needsDeferredProcessing[0] );

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=1556938&r1=1556937&r2=1556938&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 Thu Jan  9 20:22:16 2014
@@ -296,7 +296,6 @@ class DeleteNode extends DMLModStatement
 							resultColumnList,
 							(int[]) null,
 							readColsBitSet,
-							false,
 							true);  /* we alway include triggers in core language */
 
 				/* If the target table is also a source table, then

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=1556938&r1=1556937&r2=1556938&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 Thu Jan  9 20:22:16 2014
@@ -500,7 +500,6 @@ public final class InsertNode extends DM
 												sourceRCL,
 												(int[]) null,
 												(FormatableBitSet) null,
-												false,
 											    true);  /* we always include
 														 * triggers in core language */
 	

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=1556938&r1=1556937&r2=1556938&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 Thu Jan  9 20:22:16 2014
@@ -598,7 +598,6 @@ public final class UpdateNode extends DM
                                                 sourceRCL,
                                                 changedColumnIds,
                                                 readColsBitSet,
-                                                false,
                                                 true); /* we always include triggers in core language */
 
             /* If the target table is also a source table, then
@@ -1204,8 +1203,11 @@ public final class UpdateNode extends DM
 		** because they are added as a side effect of adding
 		** their indexes above.
 		*/
-		baseTable.getAllRelevantConstraints
-			( StatementType.UPDATE, false, changedColumnIds, needsDeferredProcessing, relevantConstraints );
+		baseTable.getAllRelevantConstraints(
+            StatementType.UPDATE,
+            changedColumnIds,
+            needsDeferredProcessing,
+            relevantConstraints);
 
 		int rclSize = relevantConstraints.size();
 		for (int index = 0; index < rclSize; index++)

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java?rev=1556938&r1=1556937&r2=1556938&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java Thu Jan  9 20:22:16 2014
@@ -144,7 +144,6 @@ class InsertResultSet extends DMLWriteRe
 	private	TemporaryRowHolderImpl	rowHolder;
 	private RowLocation				rl;
 
-	private	boolean					hasBeforeStatementTrigger;
 	private	boolean					hasBeforeRowTrigger;
 	private	BulkTableScanResultSet	tableScan;
 
@@ -245,22 +244,11 @@ class InsertResultSet extends DMLWriteRe
 	public ExecRow preprocessSourceRow(ExecRow execRow)
 		throws StandardException
 	{
-		//System.out.println("preprocessrow is called ");
-		/*
-		** We can process before row triggers now.  All other
-		** triggers can only be fired after we have inserted
-		** all our rows.
-		*/
-		if (hasBeforeRowTrigger)
-		{
-			// RESOLVE
-			// Possibly dead code-- if there are triggers we don't do bulk insert.
-			rowHolder.truncate();
-			rowHolder.insert(execRow);
-			triggerActivator.notifyEvent(TriggerEvents.BEFORE_INSERT,
-											(CursorResultSet)null,
-											rowHolder.getResultSet(), 
-											(int[])null);
+        if (triggerInfo != null) {
+            // We do not use bulk insert if we have triggers
+            if (SanityManager.DEBUG) {
+                SanityManager.NOTREACHED();
+            }
 		}
 
         if ( generationClauses != null )
@@ -268,8 +256,7 @@ class InsertResultSet extends DMLWriteRe
             evaluateGenerationClauses( generationClauses, activation, sourceResultSet, execRow, false );
         }
 
-		if (checkGM != null && !hasBeforeStatementTrigger)
-		{
+        if (checkGM != null) {
 			evaluateCheckConstraints();
 		}
 		// RESOLVE - optimize the cloning
@@ -330,14 +317,6 @@ class InsertResultSet extends DMLWriteRe
 		fkInfoArray = constants.getFKInfo();
 		triggerInfo = constants.getTriggerInfo();
 		
-		/*
-		** If we have a before statement trigger, then
-		** we cannot check constraints inline.
-		*/
-		hasBeforeStatementTrigger = (triggerInfo != null) ?
-				triggerInfo.hasTrigger(true, false) :
-				false;
-
 		hasBeforeRowTrigger = (triggerInfo != null) ?
 				triggerInfo.hasTrigger(true, true) :
 				false;
@@ -456,55 +435,17 @@ class InsertResultSet extends DMLWriteRe
                     getSavedObject(fullTemplateId)).build(
                         activation.getExecutionFactory());
 
-            long baseTableConglom =
-                    bulkInsertCore(lcc, fullTemplate, heapConglom);
-
-			if (hasBeforeStatementTrigger)
-			{	
-				tableScan = getTableScanResultSet(baseTableConglom); 
-
-				// fire BEFORE trigger, do this before checking constraints
-				triggerActivator.notifyEvent(TriggerEvents.BEFORE_INSERT, 
-												(CursorResultSet)null,
-												tableScan, 
-												(int[])null);
-			
-				// if we have a check constraint or generation clauses, we have
-				// to do it the hard way now before we get
-				// to our AFTER triggers.
-				if ((checkGM != null) || (generationClauses != null) )
-				{
-					tableScan = getTableScanResultSet(baseTableConglom); 
+            bulkInsertCore(lcc, fullTemplate, heapConglom);
 
-					try
-					{
-						ExecRow currRow = null;
-						while ((currRow = tableScan.getNextRowCore()) != null)
-						{
-							// we have to set the source row so the check constraint
-							// sees the correct row.
-							sourceResultSet.setCurrentRow(currRow);
- 							evaluateCheckConstraints();
-						}
-					} finally
-					{
-						sourceResultSet.clearCurrentRow();
-					}
-				}
+            if (triggerInfo != null) {
+                if (SanityManager.DEBUG) {
+                    // If we have triggers, we do not use bulkInsert
+                    SanityManager.NOTREACHED();
+                }
 			}
 			
             bulkValidateForeignKeys(tc, lcc.getContextManager(), fullTemplate);
 	
-			// if we have an AFTER trigger, let 'er rip
-			if ((triggerInfo != null) && 
-				(triggerInfo.hasTrigger(false, true) ||
-				 triggerInfo.hasTrigger(false, false))) 
-			{
-				triggerActivator.notifyEvent(TriggerEvents.AFTER_INSERT,
-										(CursorResultSet)null,
-										getTableScanResultSet(baseTableConglom), 
-										(int[])null); 
-			}
 			bulkInsertPerformed = true;
 		}
 		else
@@ -1319,7 +1260,7 @@ class InsertResultSet extends DMLWriteRe
 	}
 
 	// Do the work for a bulk insert
-	private long bulkInsertCore(LanguageConnectionContext lcc,
+    private void bulkInsertCore(LanguageConnectionContext lcc,
                                 ExecRow fullTemplate,
 								long oldHeapConglom)
 		throws StandardException
@@ -1339,13 +1280,11 @@ class InsertResultSet extends DMLWriteRe
 		// Get the properties on the old heap
 		bulkHeapCC.getInternalTablePropertySet(properties);
 
-		if (triggerInfo != null)
-		{
-			triggerActivator = new TriggerEventActivator(lcc, 
-										constants.targetUUID,
-										triggerInfo,
-										TriggerExecutionContext.INSERT_EVENT,
-										activation, null);
+        if (triggerInfo != null) {
+            // no triggers in bulk insert mode
+            if (SanityManager.DEBUG) {
+                SanityManager.NOTREACHED();
+            }
 		}
 
 		/*
@@ -1417,7 +1356,7 @@ class InsertResultSet extends DMLWriteRe
 		 */
 		if (newHeapConglom == oldHeapConglom)
 		{
-			return oldHeapConglom;
+            return;
 		}
 
 		// Find out how many rows were inserted
@@ -1470,8 +1409,6 @@ class InsertResultSet extends DMLWriteRe
 		dd.updateConglomerateDescriptor(cd, newHeapConglom, tc);
 		tc.dropConglomerate(oldHeapConglom);
 		// END RESOLVE
-
-		return newHeapConglom;
 	}
 
 	/**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoRowsResultSetImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoRowsResultSetImpl.java?rev=1556938&r1=1556937&r2=1556938&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoRowsResultSetImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/NoRowsResultSetImpl.java Thu Jan  9 20:22:16 2014
@@ -555,55 +555,6 @@ abstract class NoRowsResultSetImpl imple
 		}
 	}
 
-	/**
-	  *	Run a check constraint against the current row. Raise an error if
-	  * the check constraint is violated.
-	  *
-	  *	@param	checkGM			Generated code to run the check constraint.
-	  * @param	checkName		Name of the constraint to check.
-	  *	@param	heapConglom		Number of heap conglomerate.
-	  *	@param	activation		Class in which checkGM lives.
-	  *
-	  * @exception StandardException thrown on error
-	  */
-	public	static	void	evaluateACheckConstraint
-	(
-	  GeneratedMethod checkGM,
-	  String checkName,
-	  long heapConglom,
-	  Activation activation
-	)
-		throws StandardException
-	{
-		if (checkGM != null)
-		{
-			DataValueDescriptor checkBoolean;
-
-			checkBoolean = (DataValueDescriptor) checkGM.invoke(activation);
-
-			/* Throw exception if check constraint is violated.
-			 * (Only if check constraint evaluates to false.)
-			 */ 
-			if ((checkBoolean != null) &&
-				(! checkBoolean.isNull()) &&
-				(! checkBoolean.getBoolean()))
-			{
-				/* Now we have a lot of painful work to get the
-				 * table name for the error message.  All we have 
-				 * is the conglomerate number to work with.
-				 */
-				DataDictionary dd = activation.getLanguageConnectionContext().getDataDictionary();
-				ConglomerateDescriptor cd = dd.getConglomerateDescriptor( heapConglom );
-				TableDescriptor td = dd.getTableDescriptor(cd.getTableID());
-
-				StandardException se = StandardException.newException(SQLState.LANG_CHECK_CONSTRAINT_VIOLATED, 
-					td.getQualifiedName(), checkName);
-
-				throw se;
-			}
-		}
-
-	}
 
 	/**
 	  * Compute the generation clauses on the current row in order to fill in

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/CheckConstraintTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/CheckConstraintTest.java?rev=1556938&r1=1556937&r2=1556938&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/CheckConstraintTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/CheckConstraintTest.java Thu Jan  9 20:22:16 2014
@@ -31,10 +31,14 @@ import junit.framework.Test;
 import junit.framework.TestSuite;
 import org.apache.derbyTesting.junit.JDBC;
 import org.apache.derbyTesting.junit.BaseJDBCTestCase;
+import org.apache.derbyTesting.junit.SupportFilesSetup;
 import org.apache.derbyTesting.junit.TestConfiguration;
 
 public final class CheckConstraintTest extends BaseJDBCTestCase {
 
+    // poached from GeneratedColumnsTest
+    private static  final   String  IMPORT_FILE_NAME = "t_bi_1.dat";
+
     /**
      * Public constructor required for running test as standalone JUnit.
      */
@@ -47,7 +51,10 @@ public final class CheckConstraintTest e
     {
         TestSuite suite = new TestSuite("checkConstraint Test");
         suite.addTest(TestConfiguration.defaultSuite(CheckConstraintTest.class));
-        return suite;
+
+        return new SupportFilesSetup(
+             suite,
+             new String [] { "functionTests/tests/lang/" + IMPORT_FILE_NAME });
     }
     
     public void testNotAllowedInCheckConstraints() throws Exception
@@ -1016,4 +1023,69 @@ public final class CheckConstraintTest e
         assertStatementError("23513", s, "insert into t values -10");
         assertStatementError("23513", s, "insert into t values 10");
     }
+
+    private void setupForBulkInsert(Statement s, int limit)
+            throws SQLException {
+        s.executeUpdate(
+            "create table t_bi_1( a int, b int check (b < " + limit + "))");
+
+        s.executeUpdate(
+            "create function func () returns int " + 
+            "    language java parameter style java deterministic no sql " +
+            "    external name '" + this.getClass().getName() + ".func'");
+    }
+
+    /**
+     * DERBY-6453. Exercise hitherto untested code path in
+     * InsertResultSet (call to evaluateCheckConstraints from
+     * preprocessSourceRow used by bulkInsert) and verify IMPORT with
+     * trigger (which would a priori mandate bulk insert, but is
+     * changed due to normal inserts due to the presence of a
+     * trigger).
+     */
+    public void testbulkInsert() throws SQLException {
+
+        setAutoCommit(false);
+        Statement s = createStatement();
+
+        setupForBulkInsert(s, 0);
+
+        assertStatementError(
+            "23513", s,
+            "call syscs_util.syscs_import_data( " + 
+            "    null, " + 
+            "    'T_BI_1', " + 
+            "    'A, B', " + 
+            "    '1, 2', " + 
+            "    'extin/" + IMPORT_FILE_NAME + "', " + 
+            "    null, null, null, 0 )");
+
+        setupForBulkInsert(s, 10);
+
+        s.executeUpdate(
+            "create trigger dagstrigger no cascade before insert on t_bi_1 " +
+            "    values func() ");
+
+        funcWasCalled = false;
+
+        s.executeUpdate(
+            "call syscs_util.syscs_import_data( " + 
+            "    null, " + 
+            "    'T_BI_1', " + 
+            "    'A, B', " + 
+            "    '1, 2', " + 
+            "    'extin/" + IMPORT_FILE_NAME + "', " + 
+            "    null, null, null, 0 )");
+
+        assertTrue(funcWasCalled);
+    }
+
+    static boolean funcWasCalled;
+    
+    public static int func()
+    {
+        funcWasCalled = true;
+        return 0;
+    }
+
 }