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 2009/09/10 10:10:07 UTC

svn commit: r813293 - in /db/derby/code/trunk/java: engine/org/apache/derby/impl/sql/compile/ testing/org/apache/derbyTesting/functionTests/master/ testing/org/apache/derbyTesting/functionTests/tests/lang/

Author: kahatlen
Date: Thu Sep 10 08:10:07 2009
New Revision: 813293

URL: http://svn.apache.org/viewvc?rev=813293&view=rev
Log:
DERBY-4355: Implement CROSS JOIN

This patch adds CROSS JOIN to the grammar and wires it into the
existing join machinery. It also adds test cases to verify that the
syntax is correctly understood, and it removes a negative test case
for CROSS JOIN from the db2Compatibility test.

Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/db2Compatibility.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/JoinTest.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/db2Compatibility.sql

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj?rev=813293&r1=813292&r2=813293&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj Thu Sep 10 08:10:07 2009
@@ -1369,6 +1369,7 @@
 	 *
 	 *	JOIN
 	 *	INNER JOIN
+	 *	CROSS JOIN
 	 *	LEFT OUTER JOIN
 	 *	RIGHT OUTER JOIN
 	 *
@@ -1390,6 +1391,10 @@
 		{
 			retval = true;
 		}
+		else if (tokKind1 == CROSS && tokKind2 == JOIN)
+		{
+			retval = true;
+		}
 		else if ((tokKind1 == LEFT || tokKind1 == RIGHT) && tokKind2 == OUTER)
 		{
 			if (getToken(3).kind == JOIN)
@@ -1988,6 +1993,7 @@
 |	<CORRESPONDING: "corresponding">
 |	<COUNT: "count">
 |	<CREATE: "create">
+|	<CROSS: "cross">
 |	<CURRENT: "current">
 |	<CURRENT_DATE: "current_date">
 |	<CURRENT_TIME: "current_time">
@@ -8904,12 +8910,27 @@
 FromTable
 tableReference(boolean nestedInParens) throws StandardException :
 {
+    FromTable fromTable;
+    TableOperatorNode joinTable = null;
+}
+{
+    fromTable = tableFactor()
+        (LOOKAHEAD({joinedTableExpressionFollows()})
+            joinTable = joinedTableExpression(
+                (joinTable == null) ? fromTable : joinTable,
+                nestedInParens)) *
+    {
+        return joinTable == null ? fromTable : joinTable;
+    }
+}
+
+FromTable tableFactor() throws StandardException :
+{
 	JavaToSQLValueNode	javaToSQLNode = null;
 	TableName			tableName;
 	String				correlationName = null;
 	ResultColumnList	derivedRCL = null;
 	FromTable			fromTable;
-	TableOperatorNode	joinTable = null;
 	FromTable			tableReference;
 	Object[]			optionalTableClauses = new Object[OPTIONAL_TABLE_CLAUSES_SIZE];
 	Properties			tableProperties = null;
@@ -8926,25 +8947,8 @@
 		[ <AS> ] correlationName = identifier(Limits.MAX_IDENTIFIER_LENGTH, true)
 		[ <LEFT_PAREN> derivedRCL = derivedColumnList() <RIGHT_PAREN> ]
 		[ optionalTableClauses = optionalTableProperties() ]
-		(LOOKAHEAD( { joinedTableExpressionFollows() } )
-		 joinTable = joinedTableExpression(
-									(joinTable != null) ?
-										joinTable :
-										(FromTable) nodeFactory.getNode(
-											C_NodeTypes.FROM_VTI,
-											javaToSQLNode.getJavaValueNode(), 
-											correlationName,
-											derivedRCL, 
-											((optionalTableClauses != null) ? 
-												(Properties) optionalTableClauses[OPTIONAL_TABLE_CLAUSES_TABLE_PROPERTIES] :
-												(Properties) null),
-											getContextManager()),
-									nestedInParens)) *
 	{
-		/* Build a derived table if not a join expression */
-		if (joinTable == null)
-		{
-			fromTable = (FromTable) nodeFactory.getNode(
+		fromTable = (FromTable) nodeFactory.getNode(
 								C_NodeTypes.FROM_VTI,
 								javaToSQLNode.getJavaValueNode(), 
 								correlationName,
@@ -8953,46 +8957,20 @@
 									(Properties) optionalTableClauses[OPTIONAL_TABLE_CLAUSES_TABLE_PROPERTIES] :
 									(Properties) null),
 								getContextManager()); 
-		}
-		else
-		{
-			fromTable = joinTable;
-		}
-
 		return fromTable;
 	}
 |
 	/* identifier() used to be correlationName() */
 	tableName = qualifiedName(Limits.MAX_IDENTIFIER_LENGTH) 
 		optionalTableClauses = optionalTableClauses()
-		(LOOKAHEAD( { joinedTableExpressionFollows() } )
-		 joinTable = joinedTableExpression(
-									(joinTable != null) ?
-										joinTable :
-										(FromTable) nodeFactory.getNode(
-												C_NodeTypes.FROM_BASE_TABLE,
-												tableName,
-												(String) optionalTableClauses[OPTIONAL_TABLE_CLAUSES_CORRELATION_NAME],
-												(ResultColumnList) optionalTableClauses[OPTIONAL_TABLE_CLAUSES_DERIVED_RCL],
-												(Properties) optionalTableClauses[OPTIONAL_TABLE_CLAUSES_TABLE_PROPERTIES],
-												getContextManager()), 
-									nestedInParens)) *
 	{
-		/* Build a from table if not a join expression */
-		if (joinTable == null)
-		{
-			fromTable = (FromTable) nodeFactory.getNode(
+		fromTable = (FromTable) nodeFactory.getNode(
 											C_NodeTypes.FROM_BASE_TABLE,
 											tableName,
 												(String) optionalTableClauses[OPTIONAL_TABLE_CLAUSES_CORRELATION_NAME],
 												(ResultColumnList) optionalTableClauses[OPTIONAL_TABLE_CLAUSES_DERIVED_RCL],
 												(Properties) optionalTableClauses[OPTIONAL_TABLE_CLAUSES_TABLE_PROPERTIES],
 											getContextManager());
-		}
-		else
-		{		   
-			fromTable = joinTable;
-		}
 		return fromTable;
 	}
 |
@@ -9022,25 +9000,8 @@
 	derivedTable = derivedTable() [ <AS> ] correlationName = identifier(Limits.MAX_IDENTIFIER_LENGTH, true)
 		[ <LEFT_PAREN> derivedRCL = derivedColumnList() <RIGHT_PAREN> ]
 		[ optionalTableClauses = optionalTableProperties() ]
-		( LOOKAHEAD( { joinedTableExpressionFollows() } )
-		  joinTable = joinedTableExpression(
-									(joinTable != null) ?
-										joinTable :
-										(FromTable) nodeFactory.getNode(
-													C_NodeTypes.FROM_SUBQUERY,
-													derivedTable, 
-													correlationName,
-													derivedRCL,
-													((optionalTableClauses != null) ?
-														(Properties) optionalTableClauses[OPTIONAL_TABLE_CLAUSES_TABLE_PROPERTIES] :
-														(Properties) null),
-													getContextManager()),
-									nestedInParens)) *
 	{
-		/* Build a derived table if not a join expression */
-		if (joinTable == null)
-		{
-			fromTable = (FromTable) nodeFactory.getNode(
+		fromTable = (FromTable) nodeFactory.getNode(
 											C_NodeTypes.FROM_SUBQUERY,
 											derivedTable, 
 											correlationName,
@@ -9049,31 +9010,13 @@
 												(Properties) optionalTableClauses[OPTIONAL_TABLE_CLAUSES_TABLE_PROPERTIES] :
 												(Properties) null),
 											getContextManager()); 
-		}
-		else
-		{
-			fromTable = joinTable;
-		}
 
 		return fromTable;
 	}
 |
 	<LEFT_PAREN> tableReference = tableReferenceTypes(true) <RIGHT_PAREN>
-		( LOOKAHEAD( { joinedTableExpressionFollows() } )
-		  joinTable = joinedTableExpression(
-									(joinTable != null) ?
-										joinTable :
-										tableReference,
-									nestedInParens)) *
 	{
-		if (joinTable == null)
-		{
-			fromTable = tableReference; 
-		}
-		else
-		{
-			fromTable = joinTable;
-		}
+		fromTable = tableReference;
 
 		return fromTable;
 	}
@@ -9188,6 +9131,11 @@
 	TableOperatorNode joinNode;
 }
 {
+	joinNode = crossJoin(leftRSN, nestedInParens)
+	{
+		return joinNode;
+	}
+|
 	joinNode = qualifiedJoin(leftRSN, nestedInParens) 
 	{
 		return joinNode;
@@ -9195,6 +9143,30 @@
 }
 
 TableOperatorNode
+crossJoin(ResultSetNode leftRSN, boolean nestedInParens)
+throws StandardException :
+{
+    ResultSetNode rightRSN;
+}
+{
+    <CROSS> <JOIN> rightRSN = tableFactor()
+    {
+        TableOperatorNode ton = (TableOperatorNode) nodeFactory.getNode(
+                C_NodeTypes.JOIN_NODE,
+                leftRSN,
+                rightRSN,
+                null, // no ON clause in CROSS JOIN
+                null, // no USING clause in CROSS JOIN
+                null,
+                null,
+                null,
+                getContextManager());
+        ton.setNestedInParens(nestedInParens);
+        return ton;
+    }
+}
+
+TableOperatorNode
 qualifiedJoin(ResultSetNode leftRSN, boolean nestedInParens) throws StandardException :
 {
 	int					joinType = JoinNode.INNERJOIN;
@@ -9205,24 +9177,14 @@
 	ValueNode			onClause;
 }
 {
-	/* RESOLVE - If we ever support NATURAL JOIN then we will need to break up
-	 * this rule.  Right now the joinSpecification() is non-optional.  This
-	 * allows us to build the Join tree from left to right. With NATURAL JOINS
-	 * there is no joinSpecification() and we would want to build the tree from
-	 * right to left.
-	 */
-	//[ <NATURAL> ] 
 		[ joinType = joinType() ] <JOIN>
 		rightRSN = tableReferenceTypes(nestedInParens) 
 		onOrUsingClause = joinSpecification(leftRSN, rightRSN)
 	{
-		/* If NATURAL OR UNION is specified, then no joinSpecification()
-		 * is required, otherwise it is required.
-		 */
-
-		/* RESOLVE - Since we don't support NATURAL or UNION joins yet,
-		 * onOrUsingClause must be non-null.  (Change error message if and
-		 * when grammar changes.)
+		/* If NATURAL or CROSS is specified, then no joinSpecification()
+		 * is required, otherwise it is required. NATURAL and CROSS should
+		 * be handled by other rules, so this rule should always see a
+		 * joinSpecification().
 		 */
 
 		/* Figure out whether an ON or USING clause was used */
@@ -13655,6 +13617,7 @@
 |	tok = <CONVERT>
 |	tok = <CORRESPONDING>
 |	tok = <CREATE>
+|	tok = <CROSS>
 |	tok = <CURRENT>
 |	tok = <CURRENT_DATE>
 |	tok = <CURRENT_TIME>

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/db2Compatibility.out
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/db2Compatibility.out?rev=813293&r1=813292&r2=813293&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/db2Compatibility.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/db2Compatibility.out Thu Sep 10 08:10:07 2009
@@ -1189,13 +1189,6 @@
 1 row inserted/updated/deleted
 ij> INSERT INTO t2 VALUES(3,4);
 1 row inserted/updated/deleted
-ij> -- CROSS JOIN not supported in both Cloudscape and DB2 mode and that is why rather than getting feature not implemented, we will get syntax error
--- (1) CROSS JOIN should be disabled in FROM clause of SELECT statement
-SELECT * FROM t1 CROSS JOIN t2;
-ERROR 42X01: Syntax error: Encountered "<EOF>" at line 3, column 30.
-Issue the 'help' command for general information on IJ command syntax.
-Any unrecognized commands are treated as potential SQL commands and executed directly.
-Consult your DBMS server reference documentation for details of the SQL syntax supported by your server.
 ij> -- (2) USING should be disabled in INNER JOIN of SELECT statement
 SELECT * FROM t1 INNER JOIN t2 USING (col1);
 ERROR 42X01: Syntax error: Encountered "USING" at line 2, column 32.

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/JoinTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/JoinTest.java?rev=813293&r1=813292&r2=813293&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/JoinTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/JoinTest.java Thu Sep 10 08:10:07 2009
@@ -21,6 +21,7 @@
 
 package org.apache.derbyTesting.functionTests.tests.lang;
 
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -33,6 +34,9 @@
  * Test cases for JOINs.
  */
 public class JoinTest extends BaseJDBCTestCase {
+    private static final String SYNTAX_ERROR = "42X01";
+    private static final String AMBIGUOUS_COLNAME = "42X03";
+
     public JoinTest(String name) {
         super(name);
     }
@@ -121,4 +125,243 @@
         JDBC.assertNullability(rs, new boolean[]{true, false, true, true});
         JDBC.assertEmpty(rs);
     }
+
+    /**
+     * Test the CROSS JOIN syntax that was added in DERBY-4355.
+     */
+    public void testCrossJoins() throws SQLException {
+        // No auto-commit to make it easier to clean up the test tables.
+        setAutoCommit(false);
+
+        final String[][] T1 = {
+            {"1", "one"}, {"2", "two"}, {"3", null},
+            {"5", "five"}, {"6", "six"}
+        };
+
+        final String[][] T2 = {
+            {"1", null}, {"2", "II"}, {"4", "IV"}
+        };
+
+        Statement s = createStatement();
+        s.execute("create table t1(c1 int, c2 varchar(10))");
+        fillTable("insert into t1 values (?,?)", T1);
+        s.execute("create table t2(c1 int, c2 varchar(10))");
+        fillTable("insert into t2 values (?,?)", T2);
+
+        // Simple join
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select * from t1 cross join t2"),
+            cross(T1, T2));
+
+        // Self join
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select * from t1 a cross join t1 b"),
+            cross(T1, T1));
+
+        // Change order in select list
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select t2.*, t1.* from t1 cross join t2"),
+            cross(T2, T1));
+
+        // Multiple joins
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select * from t1 cross join t2 cross join t1 t3"),
+            cross(T1, cross(T2, T1)));
+
+        // Project one column
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select t1.c2 from t1 cross join t2"),
+            project(new int[]{1}, cross(T1, T2)));
+
+        // Project more columns
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select t1.c1, t2.c2, t2.c2 from t1 cross join t2"),
+            project(new int[]{0, 3, 3}, cross(T1, T2)));
+
+        // Aggregate function
+        JDBC.assertSingleValueResultSet(
+            s.executeQuery("select count(*) from t1 cross join t2"),
+            Integer.toString(T1.length * T2.length));
+
+        // INNER JOIN using CROSS JOIN + WHERE
+        String[][] expectedInnerJoin = new String[][] {
+            {"1", "one", "1", null}, {"2", "two", "2", "II"}
+        };
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select * from t1 cross join t2 where t1.c1=t2.c1"),
+            expectedInnerJoin);
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select * from t1 inner join t2 on t1.c1=t2.c1"),
+            expectedInnerJoin);
+
+        // ORDER BY
+        JDBC.assertFullResultSet(
+            s.executeQuery("select * from t1 cross join t2 " +
+                           "order by t1.c1 desc"),
+            reverse(cross(T1, T2)));
+
+        // GROUP BY
+        JDBC.assertFullResultSet(
+            s.executeQuery("select t1.c1, count(t1.c2) from t1 cross join t2 " +
+                           "group by t1.c1 order by t1.c1"),
+            new String[][]{
+                {"1", "3"}, {"2", "3"}, {"3", "0"}, {"5", "3"}, {"6", "3"}
+            });
+
+        // Join VALUES expressions
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery(
+                "select * from (values 1,2) v1 cross join (values 'a','b') v2"),
+            new String[][]{{"1", "a"}, {"1", "b"}, {"2", "a"}, {"2", "b"}});
+
+        // Mix INNER and CROSS
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery(
+                "select * from t1 a cross join t2 b inner join t2 c on 1=1"),
+            cross(T1, cross(T2, T2)));
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery(
+                "select * from t1 a inner join t2 b on 1=1 cross join t2 c"),
+            cross(T1, cross(T2, T2)));
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery(
+                "select * from t1 a inner join (t2 b cross join t2 c) on 1=1"),
+            cross(T1, cross(T2, T2)));
+        // RESOLVE: The syntax below should be allowed.
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery(
+                "select * from t1 a inner join t2 b cross join t2 c on 1=1"),
+            cross(T1, cross(T2, T2)));
+
+        // Check that the implicit nesting is correct.
+        // A CROSS B RIGHT C should nest as (A CROSS B) RIGHT C and
+        // not as A CROSS (B RIGHT C).
+        //
+        // 1) Would have failed if nesting was incorrect because A.C1 would be
+        //    out of scope for the join specification
+        JDBC.assertSingleValueResultSet(
+            s.executeQuery("select count(*) from t2 a cross join "+
+                           "t1 b right join t2 c on a.c1=c.c1"),
+            Integer.toString(T1.length * T2.length));
+        // 2) Would have returned returned wrong result if nesting was
+        //    incorrect
+        String[][] expectedCorrectlyNested =
+                new String[][]{{null, null, null, null, "4", "IV"}};
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select * from t2 a cross join t1 b " +
+                           "right join t2 c on b.c1=c.c1 where c.c1=4"),
+            expectedCorrectlyNested);
+        // 3) An explicitly nested query, equivalent to (2), so expect the
+        //    same result
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select * from (t2 a cross join t1 b) " +
+                           "right join t2 c on b.c1=c.c1 where c.c1=4"),
+            expectedCorrectlyNested);
+        // 4) An explicitly nested query, not equivalent to (2) or (3), so
+        //    expect different results
+        JDBC.assertUnorderedResultSet(
+            s.executeQuery("select * from t2 a cross join (t1 b " +
+                           "right join t2 c on b.c1=c.c1) where c.c1=4"),
+            new String[][] {
+                {"1", null, null, null, "4", "IV"},
+                {"2", "II", null, null, "4", "IV"},
+                {"4", "IV", null, null, "4", "IV"}});
+
+        // ***** Negative tests *****
+
+        // Self join must have alias to disambiguate column names
+        assertStatementError(
+                AMBIGUOUS_COLNAME, s, "select * from t1 cross join t1");
+
+        // Column name must be qualified if ambiguous
+        assertStatementError(
+                AMBIGUOUS_COLNAME, s, "select c1 from t1 cross join t2");
+
+        // CROSS JOIN cannot have ON clause, expect syntax error
+        assertStatementError(
+                SYNTAX_ERROR, s,
+                "select * from t1 cross join t2 on t1.c1 = t2.c2");
+
+        // Mixed CROSS with INNER/LEFT/RIGHT still needs ON
+        assertStatementError(
+                SYNTAX_ERROR, s,
+                "select * from t1 inner join t2 cross join t2 t3");
+        assertStatementError(
+                SYNTAX_ERROR, s,
+                "select * from t1 left join t2 cross join t2 t3");
+        assertStatementError(
+                SYNTAX_ERROR, s,
+                "select * from t1 right join t2 cross join t2 t3");
+        assertStatementError(
+                SYNTAX_ERROR, s,
+                "select * from t1 cross join t2 inner join t2 t3");
+    }
+
+    /**
+     * Fill a table with rows.
+     *
+     * @param sql the insert statement used to populate the table
+     * @param data the rows to insert into the table
+     */
+    private void fillTable(String sql, String[][] data) throws SQLException {
+        PreparedStatement ins = prepareStatement(sql);
+        for (int i = 0; i < data.length; i++) {
+            for (int j = 0; j < data[i].length; j++) {
+                ins.setString(j + 1, data[i][j]);
+            }
+            ins.executeUpdate();
+        }
+        ins.close();
+    }
+
+    /**
+     * Calculate the Cartesian product of two tables.
+     *
+     * @param t1 the rows in the table on the left side
+     * @param t2 the rows in the table on the right side
+     * @return a two-dimensional array containing the Cartesian product of the
+     * two tables (primary ordering same as t1, secondary ordering same as t2)
+     */
+    private static String[][] cross(String[][] t1, String[][] t2) {
+        String[][] result = new String[t1.length * t2.length][];
+        for (int i = 0; i < result.length; i++) {
+            String[] r1 = t1[i / t2.length];
+            String[] r2 = t2[i % t2.length];
+            result[i] = new String[r1.length + r2.length];
+            System.arraycopy(r1, 0, result[i], 0, r1.length);
+            System.arraycopy(r2, 0, result[i], r1.length, r2.length);
+        }
+        return result;
+    }
+
+    /**
+     * Project columns from a table.
+     *
+     * @param cols the column indexes (0-based) to project
+     * @param rows the rows in the table
+     * @return the projected result
+     */
+    private static String[][] project(int[] cols, String[][] rows) {
+        String[][] result = new String[rows.length][cols.length];
+        for (int i = 0; i < rows.length; i++) {
+            for (int j = 0; j < cols.length; j++) {
+                result[i][j] = rows[i][cols[j]];
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Reverse the order of rows in a table.
+     *
+     * @param rows the rows in the table
+     * @return the rows in reverse order
+     */
+    private static String[][] reverse(String[][] rows) {
+        String[][] result = new String[rows.length][];
+        for (int i = 0; i < rows.length; i++) {
+            result[i] = rows[rows.length - 1 - i];
+        }
+        return result;
+    }
 }

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/db2Compatibility.sql
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/db2Compatibility.sql?rev=813293&r1=813292&r2=813293&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/db2Compatibility.sql (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/db2Compatibility.sql Thu Sep 10 08:10:07 2009
@@ -606,10 +606,6 @@
 INSERT INTO t1 VALUES(3,4);
 INSERT INTO t2 VALUES(3,4);
 
--- CROSS JOIN not supported in both Cloudscape and DB2 mode and that is why rather than getting feature not implemented, we will get syntax error
--- (1) CROSS JOIN should be disabled in FROM clause of SELECT statement
-SELECT * FROM t1 CROSS JOIN t2;
-
 -- (2) USING should be disabled in INNER JOIN of SELECT statement
 SELECT * FROM t1 INNER JOIN t2 USING (col1);