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 ka...@apache.org on 2013/10/11 10:29:14 UTC

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

Author: kahatlen
Date: Fri Oct 11 08:29:14 2013
New Revision: 1531226

URL: http://svn.apache.org/r1531226
Log:
DERBY-534: Support use of the WHEN clause in CREATE TRIGGER statements

Make the code in TriggerDescriptor.getActionSPS() reusable for
TriggerDescriptor.getWhenClauseSPS() so that the fixes for DERBY-4874
and Cloudscape bug 4821 also get applied to the WHEN clause.

Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TriggerDescriptor.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/GenericTriggerExecutor.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/TriggerWhenClauseTest.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TriggerDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TriggerDescriptor.java?rev=1531226&r1=1531225&r2=1531226&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TriggerDescriptor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TriggerDescriptor.java Fri Oct 11 08:29:14 2013
@@ -333,18 +333,38 @@ public class TriggerDescriptor extends U
 	public SPSDescriptor getActionSPS(LanguageConnectionContext lcc)
 		throws StandardException
 	{
-		if (actionSPS == null)
-		{
+        return getSPS(lcc, false /* isWhenClause */);
+    }
+
+    /**
+     * Get the SPS for the triggered SQL statement or the WHEN clause.
+     *
+     * @param lcc the LanguageConnectionContext to use
+     * @param isWhenClause {@code true} if the SPS for the WHEN clause is
+     *   requested, {@code false} if it is the triggered SQL statement
+     * @return the requested SPS
+     * @throws StandardException if an error occurs
+     */
+    private SPSDescriptor getSPS(LanguageConnectionContext lcc,
+                                 boolean isWhenClause)
+            throws StandardException
+    {
+        DataDictionary dd = getDataDictionary();
+        SPSDescriptor sps = isWhenClause ? whenSPS : actionSPS;
+        UUID spsId = isWhenClause ? whenSPSId : actionSPSId;
+        String originalSQL = isWhenClause ? whenClauseText : triggerDefinition;
+
+        if (sps == null) {
 			//bug 4821 - do the sysstatement look up in a nested readonly
 			//transaction rather than in the user transaction. Because of
 			//this, the nested compile transaction which is attempting to
 			//compile the trigger will not run into any locking issues with
 			//the user transaction for sysstatements.
 			lcc.beginNestedTransaction(true);
-			actionSPS = getDataDictionary().getSPSDescriptor(actionSPSId);
+            sps = dd.getSPSDescriptor(spsId);
 			lcc.commitNestedTransaction();
 		}
-		
+
 		//We need to regenerate the trigger action sql if 
 		//1)the trigger is found to be invalid, 
 		//2)the trigger is defined at row level (that is the only kind of 
@@ -363,39 +383,49 @@ public class TriggerDescriptor extends U
 		//To fix varchar(30) in trigger action sql to varchar(64), we need
 		//to regenerate the trigger action sql. This new trigger action sql
 		//will then get updated into SYSSTATEMENTS table.
-		DataDictionary dd = getDataDictionary();
 		boolean in10_9_orHigherVersion = dd.checkVersion(DataDictionary.DD_VERSION_DERBY_10_9,null);
 		boolean usesReferencingClause = (in10_9_orHigherVersion) ? 
 				referencedColsInTriggerAction != null :
 					(referencingOld || referencingNew);
 
-		if((!actionSPS.isValid() ||
-				 (actionSPS.getPreparedStatement() == null)) && 
+        if ((!sps.isValid() ||
+                (sps.getPreparedStatement() == null)) &&
 					isRow &&
 					usesReferencingClause)
 		{
             CompilerContext newCC = lcc.pushCompilerContext(
-                    dd.getSchemaDescriptor(actionSPS.getCompSchemaId(), null));
+                    dd.getSchemaDescriptor(sps.getCompSchemaId(), null));
 			Parser	pa = newCC.getParser();
-			Visitable stmtnode = pa.parseStatement(triggerDefinition);
+            Visitable stmtnode =
+                    isWhenClause ? pa.parseSearchCondition(originalSQL)
+                                 : pa.parseStatement(originalSQL);
 			lcc.popCompilerContext(newCC);
-					
-            actionSPS.setText(dd.getTriggerActionString(stmtnode,
+
+            String newText = dd.getTriggerActionString(stmtnode,
 					oldReferencingName,
 					newReferencingName,
-					triggerDefinition,
+                    originalSQL,
 					referencedCols,
 					referencedColsInTriggerAction,
 					0,
 					td,
 					-1,
-					false
-					));
+                    false);
+
+            if (isWhenClause) {
+                // The WHEN clause is not a full SQL statement, just a search
+                // condition, so we need to turn it into a statement in order
+                // to create an SPS.
+                newText = "VALUES " + newText;
+            }
+
+            sps.setText(newText);
+
 			//By this point, we are finished transforming the trigger action if
 			//it has any references to old/new transition variables.
 		}
-		
-		return actionSPS;
+
+        return sps;
 	}
 
 	/**
@@ -420,18 +450,20 @@ public class TriggerDescriptor extends U
 	/**
 	 * Get the trigger when clause sps 
 	 *
+     * @param lcc the LanguageConnectionContext to use
 	 * @return the sps of the when clause
 	 *
 	 * @exception StandardException on error
 	 */
-	public SPSDescriptor getWhenClauseSPS()
+    public SPSDescriptor getWhenClauseSPS(LanguageConnectionContext lcc)
 		throws StandardException
 	{
-        if (whenSPSId != null && whenSPS == null)
-		{
-			whenSPS = getDataDictionary().getSPSDescriptor(whenSPSId);
-		}
-		return whenSPS;
+        if (whenSPSId == null) {
+            // This trigger doesn't have a WHEN clause.
+            return null;
+        }
+
+        return getSPS(lcc, true /* isWhenClause */);
 	}
 
 	/**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java?rev=1531226&r1=1531225&r2=1531226&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java Fri Oct 11 08:29:14 2013
@@ -1864,7 +1864,7 @@ class AlterTableConstantAction extends D
 			//    FOR EACH ROW 
 			//    SELECT oldt.c11 from DERBY4998_SOFT_UPGRADE_RESTRICT
 
-            SPSDescriptor sps = isWhenClause ? trd.getWhenClauseSPS()
+            SPSDescriptor sps = isWhenClause ? trd.getWhenClauseSPS(lcc)
                                              : trd.getActionSPS(lcc);
 			int[] referencedColsInTriggerAction = new int[td.getNumberOfColumns()];
 			java.util.Arrays.fill(referencedColsInTriggerAction, -1);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericTriggerExecutor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericTriggerExecutor.java?rev=1531226&r1=1531225&r2=1531226&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericTriggerExecutor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericTriggerExecutor.java Fri Oct 11 08:29:14 2013
@@ -107,7 +107,7 @@ abstract class GenericTriggerExecutor
 		if (!whenClauseRetrieved)
 		{
 			whenClauseRetrieved = true;
-			whenClause = triggerd.getWhenClauseSPS();
+            whenClause = triggerd.getWhenClauseSPS(lcc);
 		}
 		return whenClause;
 	}

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/TriggerWhenClauseTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/TriggerWhenClauseTest.java?rev=1531226&r1=1531225&r2=1531226&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/TriggerWhenClauseTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/TriggerWhenClauseTest.java Fri Oct 11 08:29:14 2013
@@ -52,6 +52,7 @@ public class TriggerWhenClauseTest exten
     private static final String HAS_PARAMETER = "42Y27";
     private static final String HAS_DEPENDENTS = "X0Y25";
     private static final String TABLE_DOES_NOT_EXIST = "42X05";
+    private static final String TRUNCATION = "22001";
 
     public TriggerWhenClauseTest(String name) {
         super(name);
@@ -534,4 +535,64 @@ public class TriggerWhenClauseTest exten
             s.executeQuery("select triggername from sys.systriggers"), "TR");
         getConnection().rollback(sp);
     }
+
+    /**
+     * Verify that DERBY-4874, which was fixed before support for the WHEN
+     * clause was implemented, does not affect the WHEN clause.
+     * The first stab at the WHEN clause implementation did suffer from it.
+     */
+    public void testDerby4874() throws SQLException {
+        Statement s = createStatement();
+        s.execute("create table t(x varchar(3))");
+        s.execute("create trigger tr after update of x on t "
+                + "referencing new as new for each row "
+                + "when (new.x < 'abc') values 1");
+        s.execute("insert into t values 'aaa'");
+
+        // Updating X to something longer than 3 characters should fail,
+        // since it's a VARCHAR(3).
+        assertStatementError(TRUNCATION, s, "update t set x = 'aaaa'");
+
+        // Change the type of X to VARCHAR(4) and try again. This time it
+        // should succeed, but it used to fail because the trigger hadn't
+        // been recompiled and still thought the max length was 3.
+        s.execute("alter table t alter x set data type varchar(4)");
+        assertUpdateCount(s, 1, "update t set x = 'aaaa'");
+
+        // Updating it to a longer value should still fail.
+        assertStatementError(TRUNCATION, s, "update t set x = 'aaaaa'");
+    }
+
+    /**
+     * Verify that Cloudscape bug 4821, which was fixed long before support
+     * for the WHEN clause was implemented, does not affect the WHEN clause.
+     * The first stab at the WHEN clause implementation did suffer from it.
+     */
+    public void testCloudscapeBug4821() throws SQLException {
+        // First create a trigger, and immediately perform an ALTER TABLE
+        // statement on the trigger table to make sure the trigger's SPS is
+        // invalid and must be recompiled the first time it's fired.
+        Statement s = createStatement();
+        s.execute("create table cs4821.t(x int)");
+        s.execute("create trigger cs4821.tr after insert on cs4821.t "
+                + "when (true) values 1");
+        s.execute("alter table cs4821.t add column y int");
+        commit();
+
+        // Fire the trigger and leave the transaction open afterwards.
+        s.execute("insert into cs4821.t(x) values 1");
+
+        // Now try to read all rows from the SYS.SYSSTATEMENTS table from
+        // another transaction. Used to time out because the transaction
+        // that recompiled the trigger kept the lock on the system table.
+        Connection c2 = openDefaultConnection();
+        Statement s2 = c2.createStatement();
+        JDBC.assertDrainResults(
+                s2.executeQuery("select * from sys.sysstatements"));
+        s2.close();
+        JDBC.cleanup(c2);
+
+        // Remove all tables and triggers created by this test case.
+        JDBC.dropSchema(getConnection().getMetaData(), "CS4821");
+    }
 }