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 ba...@apache.org on 2005/05/12 02:07:27 UTC

svn commit: r169735 - in /incubator/derby/code/trunk/java: engine/org/apache/derby/impl/sql/compile/FromList.java testing/org/apache/derbyTesting/functionTests/master/distinctElimination.out testing/org/apache/derbyTesting/functionTests/tests/lang/distinctElimination.sql

Author: bandaram
Date: Wed May 11 17:07:26 2005
New Revision: 169735

URL: http://svn.apache.org/viewcvs?rev=169735&view=rev
Log:
Derby-251: DISTINCT query returning duplicate rows because of incorrectly applying distinct elimination logic.

Submitted by Mamta Satoor (msatoor@gmail.com)


Modified:
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromList.java
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/distinctElimination.out
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/distinctElimination.sql

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromList.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromList.java?rev=169735&r1=169734&r2=169735&view=diff
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromList.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromList.java Wed May 11 17:07:26 2005
@@ -1040,7 +1040,7 @@
 	}
 
 	/**
-	 * Decrement (query block) level (0-based) for 
+	 * Decrement (query block) level (0-based) for
 	 * all of the tables in this from list.
 	 * This is useful when flattening a subquery.
 	 *
@@ -1069,7 +1069,7 @@
 		}
 	}
 
-	
+
 	/**
 	 * This method is used for both subquery flattening and distinct
 	 * elimination based on a uniqueness condition.  For subquery
@@ -1079,7 +1079,7 @@
 	 * any duplicates.
 	 * This is true if every table in the from list is
 	 * (a base table and the set of columns from the table that
-	 * are in equality comparisons with expressions that do not include columns 
+	 * are in equality comparisons with expressions that do not include columns
 	 * from the same table is a superset of any unique index
 	 * on the table) or an EXISTS FBT.  In addition, at least 1 of the tables
 	 * in the list has a set of columns in equality comparisons with expressions
@@ -1105,28 +1105,28 @@
 	 *		create an array of columns from the table(eqOuterCol)
 	 *		(this is used to determine that only one row will be returned
 	 *		from a join)
-	 *			
+	 *
 	 *		if the current table is the table for the result columns
 	 *			set the result columns in the eqOuterCol and tableColMap
 	 *			(if these columns are a superset of a unique index and
 	 *			all joining tables result in only one row, the
 	 *			results will be distinct)
 	 *		go through all the predicates and update tableColMap  and
-	 *		eqOuterCol with join columns and correlation variables, 
+	 *		eqOuterCol with join columns and correlation variables,
 	 *		parameters and constants
 	 *		since setting constants, correlation variables and parameters,
-	 * 		reduces the number of columns required for uniqueness in a 
+	 * 		reduces the number of columns required for uniqueness in a
 	 *		multi-column index, they are set for all the tables (if the
 	 *		table is not the result table, in this case only the column of the
      *		result table is set)
 	 *		join columns are just updated for the column in the row of the
 	 *		joining table.
-	 *		
-	 *		check if the marked columns in tableColMap are a superset of a unique 
-	 *			index		
+	 *
+	 *		check if the marked columns in tableColMap are a superset of a unique
+	 *			index
 	 *			(This means that the join will only produce 1 row when joined
 	 *			with 1 row of another table)
-	 *		check that there is a least one table for which the columns in 
+	 *		check that there is a least one table for which the columns in
 	 *			eqOuterCol(i.e. constant values) are a superset of a unique index
 	 *			(This quarantees that there will be only one row selected
 	 *			from this table).
@@ -1134,7 +1134,7 @@
 	 *	Once all tables have been evaluated, check that all the tables can be
 	 * 	joined by unique index or will have only one row
 	 *
-	 *	
+	 *
 	 *
 	 * @param rcl				If non-null, the RCL from the query block.
 	 *							If non-null for subqueries, then entry can
@@ -1150,8 +1150,8 @@
 	 *
 	 * @exception StandardException		Thrown on error
 	 */
-	boolean returnsAtMostSingleRow(ResultColumnList rcl, 
-								   ValueNode whereClause, 
+	boolean returnsAtMostSingleRow(ResultColumnList rcl,
+								   ValueNode whereClause,
 								   PredicateList wherePredicates,
 								   DataDictionary dd)
 		throws StandardException
@@ -1160,6 +1160,13 @@
 		int[]			tableNumbers;
 		ColumnReference	additionalCR = null;
 
+		PredicateList predicatesTemp;
+		predicatesTemp = (PredicateList) getNodeFactory().getNode(
+			C_NodeTypes.PREDICATE_LIST,	getContextManager());
+		int wherePredicatesSize = wherePredicates.size();
+		for (int index = 0; index < wherePredicatesSize; index++)
+			predicatesTemp.addPredicate((Predicate)wherePredicates.elementAt(index));
+
 		/* When considering subquery flattening, we are interested
 		 * in the 1st (and only) entry in the RCL.  (The RCL will be
 		 * null if result column is not of interest for subquery flattening.)
@@ -1193,6 +1200,77 @@
 			{
 				return false;
 			}
+			FromBaseTable fbt = (FromBaseTable) prn.getChildResult();
+			//Following for loop code is to take care of Derby-251 (DISTINCT returns
+			//duplicate rows).
+			//Derby-251 returned duplicate rows because we were looking at predicates
+			//that belong to existsTable to determine DISTINCT elimination
+			//
+			//(Check method level comments to understand DISTINCT elimination rules.)
+			//
+			//For one specific example, consider the query below
+			//select  distinct  q1."NO1" from IDEPT q1, IDEPT q2
+			//where  ( q2."DISCRIM_DEPT" = 'HardwareDept')
+			//and  ( q1."DISCRIM_DEPT" = 'SoftwareDept')  and  ( q1."NO1" <> ALL
+			//(select  q3."NO1" from IDEPT q3 where  (q3."REPORTTO_NO" =  q2."NO1")))
+			//(select  q3."NO1" from IDEPT q3 where  ( ABS(q3."REPORTTO_NO") =  q2."NO1")))
+			//
+			//Table IDEPT in the query above has a primary key defined on column "NO1"
+			//This query gets converted to following during optimization
+			//
+			//select  distinct  q1."NO1" from IDEPT q1, IDEPT q2
+			//where  ( q2."DISCRIM_DEPT" = 'HardwareDept')
+			//and  ( q1."DISCRIM_DEPT" = 'SoftwareDept')  and  not exists (
+			//(select  q3."NO1" from IDEPT q3 where
+			//(  ( ABS(q3."REPORTTO_NO") =  q2."NO1")  and q3."NO1" = q1."NO1") ) )  ;
+			//
+			//For the optimized query above, Derby generates following predicates.
+			//ABS(q3.reportto_no) = q2.no1
+			//q2.discrim_dept = 'HardwareDept'
+			//q1.descrim_dept = 'SoftwareDept'
+			//q1.no1 = q3.no1
+			//The predicate ABS(q3."NO1") = q1."NO1" should not be considered when trying
+			//to determine if q1 in the outer query has equality comparisons. 
+			//Similarly, the predicate q3.reportto_no = q2.no1 should not be
+			//considered when trying to determine if q2 in the outer query has
+			//equality comparisons. To achieve this, predicates based on exists base
+			//table q3 (the first and the last predicate) should be removed while
+			//evaluating outer query for uniqueness.
+			//
+			if (fbt.getExistsBaseTable())
+			{
+				int existsTableNumber = fbt.getTableNumber();
+				int predicatesTempSize = predicatesTemp.size();
+				for (int predicatesTempIndex = predicatesTempSize-1;
+					predicatesTempIndex >= 0; predicatesTempIndex--)
+				{
+					AndNode topAndNode = (AndNode)
+						((Predicate) predicatesTemp.elementAt(predicatesTempIndex)).getAndNode();
+
+					for (ValueNode whereWalker = topAndNode; whereWalker instanceof AndNode;
+						whereWalker = ((AndNode) whereWalker).getRightOperand())
+					{
+						// See if this is a candidate =
+						AndNode and = (AndNode) whereWalker;
+
+						//we only need to worry about equality predicates because only those
+						//predicates are considered during DISTINCT elimination.
+						if (!and.getLeftOperand().isRelationalOperator() ||
+							!(((RelationalOperator)(and.getLeftOperand())).getOperator() ==
+							RelationalOperator.EQUALS_RELOP))
+						{
+							continue;
+						}
+
+						JBitSet referencedTables = and.getLeftOperand().getTablesReferenced();
+						if (referencedTables.get(existsTableNumber))
+						{
+							predicatesTemp.removeElementAt(predicatesTempIndex);
+							break;
+						}
+					}
+				}
+			}
 		}
 
 		/* Build an array of tableNumbers from this query block.
@@ -1245,7 +1323,7 @@
 			/* Now see if there are any equality conditions
 			 * of interest in the where predicates.
 			 */
-			wherePredicates.checkTopPredicatesForEqualsConditions(
+			predicatesTemp.checkTopPredicatesForEqualsConditions(
 								tableNumber, eqOuterCols, tableNumbers,
 								tableColMap[index], resultColTable);
 
@@ -1298,7 +1376,7 @@
 							/* unique key join - exists tables already marked as 
 							 * 1 row - so don't need to look at them
 							 */
-							if (!oneRow[i] && tableColMap[i][index].get(0))	
+							if (!oneRow[i] && tableColMap[i][index].get(0))
 							{
 								oneRow[i] = true;
 								foundOneRow = true;

Modified: incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/distinctElimination.out
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/distinctElimination.out?rev=169735&r1=169734&r2=169735&view=diff
==============================================================================
--- incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/distinctElimination.out (original)
+++ incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/distinctElimination.out Wed May 11 17:07:26 2005
@@ -19,6 +19,13 @@
 0 rows inserted/updated/deleted
 ij> create unique index four_c1c3 on four(c1, c3);
 0 rows inserted/updated/deleted
+ij> CREATE TABLE "APP"."IDEPT" ("DISCRIM_DEPT" VARCHAR(32), "NO1" INTEGER NOT NULL, 
+"NAME" VARCHAR(50), "AUDITOR_NO" INTEGER, "REPORTTO_NO" INTEGER, "HARDWAREASSET"
+ VARCHAR(15), "SOFTWAREASSET" VARCHAR(15));
+0 rows inserted/updated/deleted
+ij> -- primary/unique
+ALTER TABLE "APP"."IDEPT" ADD CONSTRAINT "PK_IDEPT" PRIMARY KEY ("NO1");
+0 rows inserted/updated/deleted
 ij> insert into one values (1, 1, 1, 1, 1);
 1 row inserted/updated/deleted
 ij> insert into one values (2, 1, 1, 1, 1);
@@ -87,6 +94,16 @@
 1 row inserted/updated/deleted
 ij> insert into four values (3, 1, 3, 1, 1);
 1 row inserted/updated/deleted
+ij> insert into idept values ('Dept', 1, 'Department1', null, null, null, null);
+1 row inserted/updated/deleted
+ij> insert into idept values ('HardwareDept', 2, 'Department2', 25, 1, 'hardwareaset2', null);
+1 row inserted/updated/deleted
+ij> insert into idept values ('HardwareDept', 3, 'Department3', 25, 2, 'hardwareaset3', null);
+1 row inserted/updated/deleted
+ij> insert into idept values ('SoftwareDept', 4, 'Department4', 25, 1, null, 'softwareasset4');
+1 row inserted/updated/deleted
+ij> insert into idept values ('SoftwareDept', 5, 'Department5', 30, 4, null, 'softwareasset5');
+1 row inserted/updated/deleted
 ij> call SYSCS_UTIL.SYSCS_SET_RUNTIMESTATISTICS(1);
 0 rows inserted/updated/deleted
 ij> maximumdisplaywidth 20000;
@@ -143,6 +160,31 @@
 None
 	next qualifiers:
 None
+ij> --Derby251 Distinct should not get eliminated for following query
+--because there is no equality condition on unique column of table
+--in the outside query
+select  distinct  q1."NO1",  q1."NAME",  q1."AUDITOR_NO",  
+q1."REPORTTO_NO",  q1."DISCRIM_DEPT",  q1."SOFTWAREASSET" from 
+IDEPT q1, IDEPT q2 where  ( q2."DISCRIM_DEPT" = 'HardwareDept') 
+ and  ( q1."DISCRIM_DEPT" = 'SoftwareDept')  and  ( q1."NO1" 
+<> ALL  ( select  q3."NO1" from IDEPT q3 where  ( ( 
+q3."DISCRIM_DEPT" = 'Dept')  or  ( q3."DISCRIM_DEPT" = 
+'HardwareDept')  or  ( q3."DISCRIM_DEPT" = 'SoftwareDept') )  
+and  ( q3."REPORTTO_NO" =  q2."NO1") ) )  ;
+NO1        |NAME                                              |AUDITOR_NO |REPORTTO_NO|DISCRIM_DEPT                    |SOFTWAREASSET  
+---------------------------------------------------------------------------------------------------------------------------------------
+4          |Department4                                       |25         |1          |SoftwareDept                    |softwareasset4 
+5          |Department5                                       |30         |4          |SoftwareDept                    |softwareasset5 
+ij> --
+--Another test case of Derby251 where the exists table column is embedded in an expression.
+select  distinct  q1."NO1" from IDEPT q1, IDEPT q2
+where  ( q2."DISCRIM_DEPT" = 'HardwareDept')
+and  ( q1."DISCRIM_DEPT" = 'SoftwareDept')  and  ( q1."NO1" <> ALL
+(select  q3."NO1" from IDEPT q3 where  ( ABS(q3."REPORTTO_NO") =  q2."NO1")));
+NO1        
+-----------
+4          
+5          
 ij> -- result ordering is not guaranteed, but order by clause will change how
 -- distinct is executed.  So test by retrieving data into a temp table and
 -- return results ordered after making sure the query was executed as expected.
@@ -2318,5 +2360,7 @@
 ij> drop table three;
 0 rows inserted/updated/deleted
 ij> drop table four;
+0 rows inserted/updated/deleted
+ij> drop table idept;
 0 rows inserted/updated/deleted
 ij> 

Modified: incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/distinctElimination.sql
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/distinctElimination.sql?rev=169735&r1=169734&r2=169735&view=diff
==============================================================================
--- incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/distinctElimination.sql (original)
+++ incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/distinctElimination.sql Wed May 11 17:07:26 2005
@@ -12,6 +12,11 @@
 create unique index three_c1 on three(c1);
 create table four(c1 int, c2 int, c3 int, c4 int, c5 int);
 create unique index four_c1c3 on four(c1, c3);
+CREATE TABLE "APP"."IDEPT" ("DISCRIM_DEPT" VARCHAR(32), "NO1" INTEGER NOT NULL, 
+"NAME" VARCHAR(50), "AUDITOR_NO" INTEGER, "REPORTTO_NO" INTEGER, "HARDWAREASSET"
+ VARCHAR(15), "SOFTWAREASSET" VARCHAR(15));
+-- primary/unique
+ALTER TABLE "APP"."IDEPT" ADD CONSTRAINT "PK_IDEPT" PRIMARY KEY ("NO1");
 
 insert into one values (1, 1, 1, 1, 1);
 insert into one values (2, 1, 1, 1, 1);
@@ -51,6 +56,12 @@
 insert into four values (3, 1, 2, 1, 1);
 insert into four values (3, 1, 3, 1, 1);
 
+insert into idept values ('Dept', 1, 'Department1', null, null, null, null);
+insert into idept values ('HardwareDept', 2, 'Department2', 25, 1, 'hardwareaset2', null);
+insert into idept values ('HardwareDept', 3, 'Department3', 25, 2, 'hardwareaset3', null);
+insert into idept values ('SoftwareDept', 4, 'Department4', 25, 1, null, 'softwareasset4');
+insert into idept values ('SoftwareDept', 5, 'Department5', 30, 4, null, 'softwareasset5');
+
 call SYSCS_UTIL.SYSCS_SET_RUNTIMESTATISTICS(1);
 maximumdisplaywidth 20000;
 
@@ -61,6 +72,24 @@
 -- Following runtime statistics output should have Distinct Scan in it
 values SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
 
+--Derby251 Distinct should not get eliminated for following query
+--because there is no equality condition on unique column of table
+--in the outside query
+select  distinct  q1."NO1",  q1."NAME",  q1."AUDITOR_NO",  
+q1."REPORTTO_NO",  q1."DISCRIM_DEPT",  q1."SOFTWAREASSET" from 
+IDEPT q1, IDEPT q2 where  ( q2."DISCRIM_DEPT" = 'HardwareDept') 
+ and  ( q1."DISCRIM_DEPT" = 'SoftwareDept')  and  ( q1."NO1" 
+<> ALL  ( select  q3."NO1" from IDEPT q3 where  ( ( 
+q3."DISCRIM_DEPT" = 'Dept')  or  ( q3."DISCRIM_DEPT" = 
+'HardwareDept')  or  ( q3."DISCRIM_DEPT" = 'SoftwareDept') )  
+and  ( q3."REPORTTO_NO" =  q2."NO1") ) )  ;
+--
+--Another test case of Derby251 where the exists table column is embedded in an expression.
+select  distinct  q1."NO1" from IDEPT q1, IDEPT q2
+where  ( q2."DISCRIM_DEPT" = 'HardwareDept')
+and  ( q1."DISCRIM_DEPT" = 'SoftwareDept')  and  ( q1."NO1" <> ALL
+(select  q3."NO1" from IDEPT q3 where  ( ABS(q3."REPORTTO_NO") =  q2."NO1")));
+
 -- result ordering is not guaranteed, but order by clause will change how
 -- distinct is executed.  So test by retrieving data into a temp table and
 -- return results ordered after making sure the query was executed as expected.
@@ -170,3 +199,4 @@
 drop table two;
 drop table three;
 drop table four;
+drop table idept;