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 rh...@apache.org on 2022/08/18 15:01:58 UTC

svn commit: r1903526 - in /db/derby/code/trunk/java: org.apache.derby.engine/org/apache/derby/impl/sql/compile/ org.apache.derby.engine/org/apache/derby/impl/sql/execute/ org.apache.derby.tests/org/apache/derbyTesting/functionTests/tests/lang/

Author: rhillegas
Date: Thu Aug 18 15:01:58 2022
New Revision: 1903526

URL: http://svn.apache.org/viewvc?rev=1903526&view=rev
Log:
DERBY-7144: Fix bug in DEFAULT handling for identity columns in MERGE statements; commit derby-7144-03-aa-computeRowTemplateAndTrackIdentityColumnsBetter.diff.

Modified:
    db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/InsertNode.java
    db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/MatchingClauseNode.java
    db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/ResultColumn.java
    db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java
    db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/InsertConstantAction.java
    db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/InsertResultSet.java
    db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderImpl.java
    db/derby/code/trunk/java/org.apache.derby.tests/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.java

Modified: db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/InsertNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/InsertNode.java?rev=1903526&r1=1903525&r2=1903526&view=diff
==============================================================================
--- db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/InsertNode.java (original)
+++ db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/InsertNode.java Thu Aug 18 15:01:58 2022
@@ -87,6 +87,10 @@ public final class InsertNode extends DM
     private     ValueNode           fetchFirst;
     private     boolean           hasJDBClimitClause; // true if using JDBC limit/offset escape syntax
     
+
+    // true if the queryExpression sets the target table's identity column to DEFAULT
+    public     boolean           autoincrementColumnSetToDEFAULT;
+
     /**
      * Constructor for an InsertNode.
      *
@@ -802,7 +806,8 @@ public final class InsertNode extends DM
                   resultSet.isOneRowResultSet(), 
                   autoincRowLocation,
                   inMatchingClause(),
-                  identitySequenceUUIDString
+                  identitySequenceUUIDString,
+                  autoincrementColumnSetToDEFAULT
                     );
         }
         else

Modified: db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/MatchingClauseNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/MatchingClauseNode.java?rev=1903526&r1=1903525&r2=1903526&view=diff
==============================================================================
--- db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/MatchingClauseNode.java (original)
+++ db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/MatchingClauseNode.java Thu Aug 18 15:01:58 2022
@@ -839,6 +839,50 @@ public class MatchingClauseNode extends
         _dml.bindStatement();
 
         buildThenColumnsForInsert( fullFromList, targetTable, _dml.resultSet.getResultColumns(), _insertColumns, _insertValues );
+
+        boolean autoincrementColumnSetToDEFAULT = autoincrementColumnSetToDEFAULT(targetTable.getTableDescriptor(), selectList);
+        ((InsertNode) _dml).autoincrementColumnSetToDEFAULT = autoincrementColumnSetToDEFAULT;
+    }
+
+    /**
+     * Return true if the VALUES clause of the INSERT branch contains an autoincrement column
+     * which is set to DEFAULT or if the VALUES clause doesn't explicitly mention the autoincrement column.
+     *
+     * @param targetTableDescriptor Target table catalog info
+     * @param valuesList Columns explicitly mentioned in the VALUES clause
+     */
+    private boolean autoincrementColumnSetToDEFAULT(TableDescriptor targetTableDescriptor, ResultColumnList valuesList) {
+
+        for (int idx = 0; idx < valuesList.size(); idx++)
+        {
+            ResultColumn rc = valuesList.getResultColumn(idx+1);
+
+            if (rc.isAutoincrementGenerated() && rc.wasDefaultColumn()) { return true; }
+        }
+
+        boolean tableHasAutoincrementColumn = targetTableDescriptor.tableHasAutoincrement();
+        boolean autoincrementInValuesList = false;        
+        for (int idx = 0; idx < _insertColumns.size(); idx++)
+        {
+            ResultColumn insertColumn = _insertColumns.getResultColumn(idx+1);
+
+            if (insertColumn != null)
+            {
+                ColumnDescriptor colDesc = targetTableDescriptor.getColumnDescriptor(insertColumn.getName());
+
+                if ((colDesc != null) && colDesc.isAutoincrement())
+                {
+                    autoincrementInValuesList = true;
+                    break;
+                }
+            }
+        }
+
+        // If the table has an autoincrement column but it wasn't mentioned in the VALUES clause,
+        // then it will be implicitly set to DEFAULT.
+        if (tableHasAutoincrementColumn && !autoincrementInValuesList) { return true; }
+
+        return false;
     }
 
     /**  Bind the values in the INSERT list */

Modified: db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/ResultColumn.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/ResultColumn.java?rev=1903526&r1=1903525&r2=1903526&view=diff
==============================================================================
--- db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/ResultColumn.java (original)
+++ db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/compile/ResultColumn.java Thu Aug 18 15:01:58 2022
@@ -633,6 +633,7 @@ class ResultColumn extends ValueNode
                 "type: " + getTypeServices() + "\n" +
                 "columnDescriptor: " + _columnDescriptor + "\n" +
                 "isGenerated: " + _isGenerated + "\n" +
+                "autoincrementGenerated: " + _autoincrementGenerated + "\n" +
                 "isGeneratedForUnmatchedColumnInInsert: " + _isGeneratedForUnmatchedColumnInInsert + "\n" +
                 "isGroupingColumn: " + _isGroupingColumn + "\n" +
                 "isReferenced: " + _isReferenced + "\n" +

Modified: db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java?rev=1903526&r1=1903525&r2=1903526&view=diff
==============================================================================
--- db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java (original)
+++ db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java Thu Aug 18 15:01:58 2022
@@ -764,6 +764,7 @@ public class GenericConstantActionFactor
      autoincrement columns
      *  @param underMerge   True if this is an INSERT action of a MERGE statement.
      *  @param identitySequenceUUIDString   For 10.11 and higher, the handle on the sequence for the identity column
+     *  @param autoincrementColumnSetToDEFAULT True if the query expression sets the identity column to DEFAULT
      *
      * @exception StandardException		Thrown on failure
      */
@@ -793,7 +794,8 @@ public class GenericConstantActionFactor
         boolean				singleRowSource,
         RowLocation[]		autoincRowLocation,
         boolean		underMerge,
-        String		identitySequenceUUIDString
+        String		identitySequenceUUIDString,
+        boolean         autoincrementColumnSetToDEFAULT
         )
         throws StandardException {
         return new InsertConstantAction(tableDescriptor,
@@ -815,7 +817,8 @@ public class GenericConstantActionFactor
                                         singleRowSource,
                                         autoincRowLocation,
                                         underMerge,
-                                        identitySequenceUUIDString
+                                        identitySequenceUUIDString,
+                                        autoincrementColumnSetToDEFAULT
             );
     }
 

Modified: db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/InsertConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/InsertConstantAction.java?rev=1903526&r1=1903525&r2=1903526&view=diff
==============================================================================
--- db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/InsertConstantAction.java (original)
+++ db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/InsertConstantAction.java Thu Aug 18 15:01:58 2022
@@ -89,6 +89,9 @@ public class InsertConstantAction extend
     RowLocation[] autoincRowLocation;
     private long[] autoincIncrement;
 
+    // true if the queryExpression sets the target table's identity column to DEFAULT
+    private     boolean           autoincrementColumnSetToDEFAULT;
+
     /** Position of autogenerated column */
     private transient   int firstAutoGenColumn = -1;
     final public boolean hasDeferrableChecks;
@@ -133,6 +136,7 @@ public class InsertConstantAction extend
      * 							  in SYSCOLUMNS for each ai column.
      *  @param underMerge   True if this is an action of a MERGE statement.
      *  @param identitySequenceUUIDString   For 10.11 and higher, the handle on the sequence for the identity column
+     *  @param autoincrementColumnSetToDEFAULT True if the query assigns a DEFAULT value to the identity column
      */
     InsertConstantAction(TableDescriptor tableDescriptor,
                          long				conglomId,
@@ -153,7 +157,9 @@ public class InsertConstantAction extend
                          boolean				singleRowSource,
                          RowLocation[]		autoincRowLocation,
                          boolean		underMerge,
-                         String		identitySequenceUUIDString) {
+                         String		identitySequenceUUIDString,
+                         boolean         autoincrementColumnSetToDEFAULT
+        ) {
         super(conglomId, 
               heapSCOCI,
               irgs, 
@@ -181,6 +187,7 @@ public class InsertConstantAction extend
         this.indexNames = indexNames;
         this.hasDeferrableChecks = hasDeferrableChecks;
         this.identitySequenceUUIDString = identitySequenceUUIDString;
+        this.autoincrementColumnSetToDEFAULT = autoincrementColumnSetToDEFAULT;
     }
 
     // INTERFACE METHODS
@@ -216,10 +223,10 @@ public class InsertConstantAction extend
 		
         autoincIncrement = ArrayUtil.readLongArray(in);
         identitySequenceUUIDString = (String) in.readObject();
+        autoincrementColumnSetToDEFAULT = in.readBoolean();
     }
 
 
-
     /**
      * Write this object to a stream of stored objects.
      *
@@ -237,6 +244,7 @@ public class InsertConstantAction extend
         ArrayUtil.writeArray(out, columnNames);
         ArrayUtil.writeLongArray(out, autoincIncrement);
         out.writeObject( identitySequenceUUIDString );
+        out.writeBoolean(autoincrementColumnSetToDEFAULT);
     }
 
     /** Get the 0-based position of the autogenerated column */
@@ -258,6 +266,11 @@ public class InsertConstantAction extend
     }
 
     /**
+     * Return true if the queryExpression sets the target table's identity column to DEFAULT.
+     */
+    public boolean autoincrementColumnSetToDEFAULT() { return autoincrementColumnSetToDEFAULT; }
+
+    /**
      *	Gets the name of the schema that the table is in
      *
      *	@return	schema name

Modified: db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/InsertResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/InsertResultSet.java?rev=1903526&r1=1903525&r2=1903526&view=diff
==============================================================================
--- db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/InsertResultSet.java (original)
+++ db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/InsertResultSet.java Thu Aug 18 15:01:58 2022
@@ -1104,9 +1104,12 @@ class InsertResultSet extends DMLWriteGe
             // If the identity column was declared BY DEFAULT, then it could be
             // overridden by the WHEN NOT MATCHED clause. In that case, the
             // row will contain a non-null value which we do not want to clobber.
+            // Commented code preserves previous attempts to solve this situation--
+            // in case this issue needs to be revisited.
             //
             //boolean needToGenerateValue = ( dvd == null ) || ( dvd.isNullOp().getBoolean() );
-            boolean needToGenerateValue = ( dvd == null );
+            //boolean needToGenerateValue = ( dvd == null );
+            boolean needToGenerateValue = ( dvd == null ) || constants.autoincrementColumnSetToDEFAULT();
 
             if ( needToGenerateValue )
             {

Modified: db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderImpl.java?rev=1903526&r1=1903525&r2=1903526&view=diff
==============================================================================
--- db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderImpl.java (original)
+++ db/derby/code/trunk/java/org.apache.derby.engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderImpl.java Thu Aug 18 15:01:58 2022
@@ -302,7 +302,7 @@ class TemporaryRowHolderImpl implements
             CID = 
                 tc.createConglomerate(
                     "heap",
-                    inputRow.getRowArray(),
+                    makeTemplateRow(inputRow.getRowArray()),
                     null, //column sort order - not required for heap
                     collation_ids,
                     properties,
@@ -347,6 +347,46 @@ class TemporaryRowHolderImpl implements
         }
     }
 
+    /**
+     * Make a template row for creating the spillover conglomerate. We only do this
+     * for MERGE statments. MERGE processing can temporarily stuff literal nulls into
+     * the DataValueDescriptor slot for an autoincrement column. The null DataValueDescriptor
+     * can not be used to create the layout of the spillover conglomerate. See DERBY-7144.
+     *
+     * @param originalRow The original row
+     */
+    private DataValueDescriptor[] makeTemplateRow(DataValueDescriptor[] originalRow)
+        throws StandardException {
+
+        boolean useOriginalRow = true;
+
+        for (DataValueDescriptor dvd : originalRow)
+        {
+            if (dvd == null)
+            {
+                useOriginalRow = false;
+                break;
+            }
+        }
+
+        if (useOriginalRow) { return originalRow; }
+        else
+        {
+            int columnCount = resultDescription.getColumnCount();
+            ExecRow emptyRow = activation.getExecutionFactory().getValueRow(columnCount);
+            
+            for (int idx = 1; idx <= columnCount; idx++)
+            {
+                emptyRow.setColumn
+                    (
+                        idx,
+                        resultDescription.getColumnDescriptor(idx).getType().getNullabilityType(true).getNull()
+                    );
+            }
+
+            return emptyRow.getRowArray();
+        }
+    }
 
     /**
      * Maintain an unique index based on the input row's row location in the

Modified: db/derby/code/trunk/java/org.apache.derby.tests/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/org.apache.derby.tests/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.java?rev=1903526&r1=1903525&r2=1903526&view=diff
==============================================================================
--- db/derby/code/trunk/java/org.apache.derby.tests/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.java (original)
+++ db/derby/code/trunk/java/org.apache.derby.tests/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.java Thu Aug 18 15:01:58 2022
@@ -9561,6 +9561,128 @@ public class MergeStatementTest extends
 
     }
     
+    public void test_063_Derby7144() throws SQLException {
+        Connection  conn = openUserConnection( TEST_DBO );
+        
+        goodStatement(conn, "CREATE TABLE targetData_7144 (ID BIGINT GENERATED ALWAYS AS IDENTITY)");
+        goodStatement(conn, "CREATE TABLE sourceData_7144(CATEGORY INTEGER)");
+        goodStatement(conn, "INSERT INTO sourceData_7144 VALUES (10), (20)");
+
+        // the following statement used to kill the connection and raise the following error:
+        //
+        // ERROR XJ001: Java exception: 'ASSERT FAILED row template is null for column[0].: org.apache.derby.shared.common.sanity.AsserFailure'.
+        goodStatement
+            (
+                conn,
+                "MERGE INTO targetData_7144 target USING sourceData_7144 source ON 1 < 2\n" +
+                "WHEN NOT MATCHED THEN INSERT (id) VALUES (DEFAULT)\n"
+            );
+        assertResults
+            (
+                conn,
+                "SELECT * FROM targetData_7144",
+                new String[][]
+                {
+                    { "1" },   
+                    { "2" },    
+                },
+                false
+            );
+
+        // next test
+        goodStatement(conn, "CREATE TABLE targetData3_7144 (ID BIGINT GENERATED BY DEFAULT AS IDENTITY)");
+        goodStatement
+            (
+                conn,
+                "MERGE INTO targetData3_7144 target USING sourceData_7144 source ON 1 < 2\n" +
+                "WHEN NOT MATCHED THEN INSERT (id) VALUES (100)\n"
+            );
+        assertResults
+            (
+                conn,
+                "SELECT * FROM targetData3_7144",
+                new String[][]
+                {
+                    { "100" },   
+                    { "100" },    
+                },
+                false
+            );
+
+        // next test
+        goodStatement
+            (
+                conn,
+                "CREATE TABLE targetData_2_7144 (\n" +
+                "	  ID        BIGINT    PRIMARY KEY GENERATED BY DEFAULT AS IDENTITY NOT NULL,\n" +
+                "	  CATEGORY  INTEGER  NOT NULL,\n" +
+                "	  VALUE     DOUBLE    NOT NULL,\n" +
+                "	  ATTIME    TIMESTAMP NOT NULL,\n" +
+                "	  AGGDATE   DATE      NOT NULL,\n" +
+                "	  AGGCOUNT  INTEGER   NOT NULL,\n" +
+                "\n" +
+                "	  UNIQUE    (AGGDATE, CATEGORY)\n" +
+                "	)\n"
+            );
+         goodStatement
+            (
+                conn,
+                "CREATE TABLE sourceData2_7144\n" +
+                "(\n" +
+                "	  CATEGORY  INTEGER,\n" +
+                "	  VALUE     DOUBLE,\n" +
+                "	  ATTIME    TIMESTAMP,\n" +
+                "	  AGGDATE   DATE,\n" +
+                "	  AGGCOUNT  INTEGER\n" +
+                ")\n"
+            );
+         goodStatement
+            (
+                conn,
+                "INSERT INTO sourceData2_7144 VALUES\n" +
+                "(1, 45.67, TIMESTAMP('2022-07-29 01:24:21.336'), DATE('2022-07-29'), 3),\n" +
+                "(4, 37.15, TIMESTAMP('2022-07-29 01:24:21.336'), DATE('2022-07-29'), 3),\n" +
+                "(2, 123.34, TIMESTAMP('2022-07-31 01:38:24.66'), DATE('2022-07-31'), 3),\n" +
+                "(1, 78.9, TIMESTAMP('2022-07-31 01:38:24.66'), DATE('2022-07-31'), 2),\n" +
+                "(3, 1.2, TIMESTAMP('2022-07-31 01:38:24.66'), DATE('2022-07-31'), 1),\n" +
+                "(4, 5.6, TIMESTAMP('2022-07-31 01:38:24.66'), DATE('2022-07-31'), 1)\n"
+            );
+
+         // this statement used to kill the connection and raise the following error:
+         //
+         // ERROR XJ001: Java exception: 'ASSERT FAILED row template is null for column[0].: org.apache.derby.shared.common.sanity.AssetFailure'.
+         goodStatement
+            (
+                conn,
+                "MERGE INTO targetData_2_7144 target\n" +
+                "	USING sourceData2_7144 source\n" +
+                "	   ON target.CATEGORY = source.CATEGORY\n" +
+                "	  AND target.AGGDATE = source.AGGDATE\n" +
+                "	 WHEN MATCHED THEN\n" +
+                "	      UPDATE SET VALUE = target.VALUE + source.VALUE,\n" +
+                "	      ATTIME = CASE WHEN source.ATTIME < target.ATTIME THEN target.ATTIME ELSE source.ATTIME END,\n" +
+                "	      AGGCOUNT = target.AGGCOUNT + source.AGGCOUNT\n" +
+                "	 WHEN NOT MATCHED THEN\n" +
+                "	      INSERT (CATEGORY, VALUE, ATTIME, AGGDATE, AGGCOUNT)\n" +
+                "	      VALUES (source.CATEGORY, source.VALUE, source.ATTIME, source.AGGDATE, source.AGGCOUNT)\n"
+            );
+        assertResults
+            (
+                conn,
+                "SELECT * FROM targetData_2_7144",
+                new String[][]
+                {
+                    { "1", "1", "45.67", "2022-07-29 01:24:21.336", "2022-07-29", "3" },        
+                    { "2", "4", "37.15", "2022-07-29 01:24:21.336", "2022-07-29", "3" },         
+                    { "3", "2", "123.34", "2022-07-31 01:38:24.66", "2022-07-31", "3" },         
+                    { "4", "1", "78.9", "2022-07-31 01:38:24.66", "2022-07-31", "2" },          
+                    { "5", "3", "1.2", "2022-07-31 01:38:24.66",  "2022-07-31", "1" },         
+                    { "6", "4",  "5.6", "2022-07-31 01:38:24.66", "2022-07-31", "1" }          
+                },
+                false
+            );
+    }
+    
     ///////////////////////////////////////////////////////////////////////////////////
     //
     // ROUTINES