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 2010/06/19 17:10:38 UTC

svn commit: r956238 - in /db/derby/code/branches/10.6: ./ java/engine/org/apache/derby/impl/sql/compile/ java/testing/org/apache/derbyTesting/functionTests/tests/lang/

Author: dag
Date: Sat Jun 19 15:10:37 2010
New Revision: 956238

URL: http://svn.apache.org/viewvc?rev=956238&view=rev
Log:
DERBY-4698 Simple query with HAVING clause crashes with NullPointerException

Patch derby-4698-2. The case of column references in HAVING clauses
being wrong after JOIN flattening was initially solved by
DERBY-3880. That solution was partial in that it can sometimes happen
too late. 

This patch changes the fix-up of column references in a having clause
after join flattening to the same point in time as that of other
column references that need fix-up after the flattening (rcl, column
references in join predicates and group by clauses). Thus the fixup is
moved from the modifyaccesspath phase to the preprocess phase.

Merged from trunk with no adjustments needed as

svn merge -c 956234 https://svn.apache.org/repos/asf/db/derby/code/trunk


Modified:
    db/derby/code/branches/10.6/   (props changed)
    db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/AggregateNode.java
    db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromList.java
    db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromSubquery.java
    db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromTable.java
    db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java
    db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java
    db/derby/code/branches/10.6/java/testing/org/apache/derbyTesting/functionTests/tests/lang/GroupByTest.java

Propchange: db/derby/code/branches/10.6/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Sat Jun 19 15:10:37 2010
@@ -1,2 +1,2 @@
-/db/derby/code/trunk:938547,938796,938959,939231,940462,940469,941627,942031,944152,946794,948045,948069,951346,952138,954344,954544,955001
+/db/derby/code/trunk:938547,938796,938959,939231,940462,940469,941627,942031,944152,946794,948045,948069,951346,952138,954344,954544,955001,956234
 /db/derby/docs/trunk:954344

Modified: db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/AggregateNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/AggregateNode.java?rev=956238&r1=956237&r2=956238&view=diff
==============================================================================
--- db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/AggregateNode.java (original)
+++ db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/AggregateNode.java Sat Jun 19 15:10:37 2010
@@ -548,26 +548,6 @@ public class AggregateNode extends Unary
 			operand;
 
 
-		/* The operand for this aggregate node was initialized at bind
-		 * time. Between then and now it's possible that certain changes
-		 * have been made to the query tree which affect this operand. In
-		 * particular, if the operand was pointing to a result column in
-		 * a JoinNode and then that JoinNode was flattened during pre-
-		 * processing, all of the references to that JoinNode--including
-		 * this aggregate's operand--need to be updated to reflect the
-		 * fact that the Join Node no longer exists. So check to see if
-		 * the operand is a column reference, and if so, make a call to
-		 * remap it to its underlying expression. If nothing has happened
-		 * then this will be a no-op; but if something has changed to void
-		 * out the result column to which the operand points, the result
-		 * column will be marked "redundant" and the following call should
-		 * remap as appropriate. DERBY-3880.
-		 */
-		if (operand instanceof ColumnReference)
-		{
-			((ColumnReference)operand).remapColumnReferencesToExpressions();
-		}
-
 		return (ResultColumn) getNodeFactory().getNode(
 								C_NodeTypes.RESULT_COLUMN,
 								"##aggregate expression",

Modified: db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromList.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromList.java?rev=956238&r1=956237&r2=956238&view=diff
==============================================================================
--- db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromList.java (original)
+++ db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromList.java Sat Jun 19 15:10:37 2010
@@ -708,13 +708,15 @@ public class FromList extends QueryTreeN
 	 * @param predicateList		The PredicateList from the outer query
 	 * @param sql				The SubqueryList from the outer query
 	 * @param gbl				The group by list, if any
+     * @param havingClause      The HAVING clause, if any
 	 *
 	 * @exception StandardException		Thrown on error
 	 */
 	public void flattenFromTables(ResultColumnList rcl,
 								  PredicateList predicateList,
 								  SubqueryList sql,
-								  GroupByList gbl)
+                                  GroupByList gbl,
+                                  ValueNode havingClause)
 									throws StandardException
 	{
 		boolean			flattened = true;
@@ -757,7 +759,8 @@ public class FromList extends QueryTreeN
 														rcl,
 														predicateList,
 														sql,
-														gbl);
+                                                        gbl,
+                                                        havingClause);
 					if (SanityManager.DEBUG)
 					{
 						SanityManager.ASSERT(flatteningFL == null ||

Modified: db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromSubquery.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromSubquery.java?rev=956238&r1=956237&r2=956238&view=diff
==============================================================================
--- db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromSubquery.java (original)
+++ db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromSubquery.java Sat Jun 19 15:10:37 2010
@@ -483,6 +483,7 @@ public class FromSubquery extends FromTa
 	 * @param outerPList	PredicateList to append wherePredicates to.
 	 * @param sql				The SubqueryList from the outer query
 	 * @param gbl				The group by list, if any
+     * @param havingClause      The HAVING clause, if any
 	 *
 	 * @return FromList		The fromList from the underlying SelectNode.
 	 *
@@ -491,7 +492,8 @@ public class FromSubquery extends FromTa
 	public FromList flatten(ResultColumnList rcl,
 							PredicateList outerPList,
 							SubqueryList sql,
-							GroupByList gbl)
+                            GroupByList gbl,
+                            ValueNode havingClause)
 
 			throws StandardException
 	{
@@ -541,6 +543,10 @@ public class FromSubquery extends FromTa
 			gbl.remapColumnReferencesToExpressions();
 		}
 
+        if (havingClause != null) {
+            havingClause.remapColumnReferencesToExpressions();
+        }
+
 		return fromList;
 	}
 

Modified: db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromTable.java?rev=956238&r1=956237&r2=956238&view=diff
==============================================================================
--- db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromTable.java (original)
+++ db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/FromTable.java Sat Jun 19 15:10:37 2010
@@ -1420,6 +1420,7 @@ abstract class FromTable extends ResultS
 	 * @param outerPList	PredicateList to append wherePredicates to.
 	 * @param sql				The SubqueryList from the outer query
 	 * @param gbl				The group by list, if any
+     * @param havingClause      The HAVING clause, if any
 	 *
 	 * @return FromList		The fromList from the underlying SelectNode.
 	 *
@@ -1428,7 +1429,8 @@ abstract class FromTable extends ResultS
 	public FromList flatten(ResultColumnList rcl,
 							PredicateList outerPList,
 							SubqueryList sql,
-							GroupByList gbl)
+                            GroupByList gbl,
+                            ValueNode havingClause)
 
 			throws StandardException
 	{

Modified: db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java?rev=956238&r1=956237&r2=956238&view=diff
==============================================================================
--- db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java (original)
+++ db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java Sat Jun 19 15:10:37 2010
@@ -1414,6 +1414,7 @@ public class JoinNode extends TableOpera
 	 * @param outerPList		PredicateList to append wherePredicates to.
 	 * @param sql				The SubqueryList from the outer query
 	 * @param gbl				The group by list, if any
+     * @param havingClause      The HAVING clause, if any
 	 *
 	 * @return FromList		The fromList from the underlying SelectNode.
 	 *
@@ -1422,7 +1423,8 @@ public class JoinNode extends TableOpera
 	public FromList flatten(ResultColumnList rcl,
 							PredicateList outerPList,
 							SubqueryList sql,
-							GroupByList gbl)
+                            GroupByList gbl,
+                            ValueNode havingClause)
 
 			throws StandardException
 	{
@@ -1466,6 +1468,11 @@ public class JoinNode extends TableOpera
 			gbl.remapColumnReferencesToExpressions();
 		}
 
+        if (havingClause != null) {
+            havingClause.remapColumnReferencesToExpressions();
+        }
+
+
 		if (joinPredicates.size() > 0)
 		{
 			outerPList.destructiveAppend(joinPredicates);

Modified: db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java?rev=956238&r1=956237&r2=956238&view=diff
==============================================================================
--- db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java (original)
+++ db/derby/code/branches/10.6/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java Sat Jun 19 15:10:37 2010
@@ -1087,7 +1087,8 @@ public class SelectNode extends ResultSe
 		fromList.flattenFromTables(resultColumns, 
 								   wherePredicates, 
 								   whereSubquerys,
-								   groupByList);
+                                   groupByList,
+                                   havingClause);
 
 		if (wherePredicates != null && wherePredicates.size() > 0 && fromList.size() > 0)
 		{

Modified: db/derby/code/branches/10.6/java/testing/org/apache/derbyTesting/functionTests/tests/lang/GroupByTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.6/java/testing/org/apache/derbyTesting/functionTests/tests/lang/GroupByTest.java?rev=956238&r1=956237&r2=956238&view=diff
==============================================================================
--- db/derby/code/branches/10.6/java/testing/org/apache/derbyTesting/functionTests/tests/lang/GroupByTest.java (original)
+++ db/derby/code/branches/10.6/java/testing/org/apache/derbyTesting/functionTests/tests/lang/GroupByTest.java Sat Jun 19 15:10:37 2010
@@ -2270,4 +2270,85 @@ public class GroupByTest extends BaseJDB
                 {"50"},{"45"},{"46"},{"90"}});
         rollback();
     }
+
+
+    /**
+     * HAVING with an aggregate function in presence of join flattening
+     * DERBY-4698. See also DERBY-3880.
+     * @throws SQLException
+     */
+    public void testDerby4698() throws SQLException {
+        setAutoCommit(false);
+        Statement s = createStatement();
+        ResultSet rs;
+
+        s.executeUpdate(
+            "create table usr ( user_id  bigint primary key," +
+            "                  deleted  char(1) not null)");
+        s.executeUpdate(
+            "create table  user_account ( user_account_id bigint primary key," +
+            "                             user_id  bigint not null," +
+            "                             account_id  bigint not null)");
+
+        s.executeUpdate(
+            "CREATE TABLE  ACCOUNT  (ACCOUNT_ID  BIGINT PRIMARY KEY," +
+            "                        ACCOUNT_TYPE  VARCHAR(10) NOT NULL," +
+            "                        BALANCE  DOUBLE NOT NULL)");
+
+        s.executeUpdate(
+            "insert into usr values " +
+            "   (3003, 'Y'), (3004, 'N'), (1001, 'N'), (1002, 'Y')," +
+            "   (1003, 'N'), (1004, 'N'), (1005, 'N'), (1006, 'N')," +
+            "   (1007, 'N'), (1008, 'N'), (2002, 'N')");
+
+        s.executeUpdate(
+            "insert into user_account values " +
+            "  (4004, 3003, 9009), (4005, 3004, 9010), (5005, 1001, 10010)," +
+            "  (5006, 3004, 10011), (5007, 2002, 10012), (5008, 1003, 10013)," +
+            "  (5009, 1004, 10014), (5010, 1005, 10015), (5011, 1006, 10016)," +
+            "  (5012, 1007, 10017), (5013, 1008, 10018), (6006, 1001, 11011)," +
+            "  (6007, 3004, 11012), (6008, 2002, 11013), (6009, 1003, 11014)," +
+            "  (6010, 1004, 11015), (6011, 1005, 11016), (6012, 1006, 11017)," +
+            "  (6013, 1007, 11018), (6014, 1008, 11019), (1001, 1001, 1002)," +
+            "  (1002, 1002, 1003), (1003, 1003, 1004), (1004, 1004, 1005)," +
+            "  (1005, 1005, 1006), (1006, 1006, 1007), (1007, 1007, 1008)," +
+            "  (1008, 1008, 1009), (1009, 1004, 1010), (2002, 1004, 6006)," +
+            "  (3003, 2002, 7007)");
+
+        s.executeUpdate(
+            "insert into account values " +
+            " (9009, 'USER', 12.5), (9010, 'USER', 12.5)," +
+            " (10010, 'USER-01', 0.0), (10011, 'USER-01', 0.0)," +
+            " (10012, 'USER-01', 0.0), (10013, 'USER-01', 0.0)," +
+            " (10014, 'USER-01', 99.0), (10015, 'USER-01', 0.0)," +
+            " (10016, 'USER-01', 0.0), (10017, 'USER-01', 0.0)," +
+            " (10018, 'USER-01', 0.0), (11011, 'USER-02', 0.0)," +
+            " (11012, 'USER-02', 0.0), (11013, 'USER-02', 0.0)," +
+            " (11014, 'USER-02', 0.0), (11015, 'USER-02', 0.0)," +
+            " (11016, 'USER-02', 0.0), (11017, 'USER-02', 0.0)," +
+            " (11018, 'USER-02', 0.0), (11019, 'USER-02', 0.0)," +
+            " (1002, 'USER', 10.0), (1003, 'USER', 80.31)," +
+            " (1004, 'USER', 10.0), (1005, 'USER', 161.7)," +
+            " (1006, 'USER', 10.0), (1007, 'USER', 10.0)," +
+            " (1008, 'USER', 10.0), (1009, 'USER', 10.0)," +
+            " (7007, 'USER', 11.0)");
+
+        rs = s.executeQuery(
+            "SELECT user0_.user_id AS col_0_0_," +
+            "   SUM(account2_.balance) AS col_1_0_ " +
+            "   FROM usr user0_ " +
+            "   INNER JOIN user_account accountlin1_ " +
+            "   ON user0_.user_id = accountlin1_.user_id " +
+            "   INNER JOIN account account2_ " +
+            "   ON accountlin1_.account_id = account2_.account_id " +
+            "WHERE user0_.deleted = 'N' " +
+            "      AND ( account2_.account_type IN ( 'USER-01', 'USER' ) )" +
+            "GROUP BY user0_.user_id " +
+            "HAVING SUM(account2_.balance) >= 100.0 ");
+
+        JDBC.assertFullResultSet(rs, new String[][] {
+                {"1004", "260.7"} });
+
+            rollback();
+    }
 }