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 mi...@apache.org on 2006/09/20 02:09:19 UTC

svn commit: r448016 - in /db/derby/code/branches/10.1/java: engine/org/apache/derby/impl/sql/compile/ testing/org/apache/derbyTesting/functionTests/master/ testing/org/apache/derbyTesting/functionTests/suites/ testing/org/apache/derbyTesting/functionTe...

Author: mikem
Date: Tue Sep 19 17:09:18 2006
New Revision: 448016

URL: http://svn.apache.org/viewvc?view=rev&rev=448016
Log:
DERBY-1784
contributed by Yip Ng
committing derby1784-10.1-diff01.txt patch as backport of issue to 10.1 branch.

After studying the
compiler code abit more, I found that DML statements such as INSERT, UPDATE and
DELETE also suffer from the same problem (they use different bind logic)

With that said, this patch attempts to address all the stated problems above
when column reference is qualified with a synonym table name.

The fundamental problem is that Derby does not keep the original unbound table
name around once the synonym is resolved. So the fix is to address this case
and apply the qualification properly.

In the VIEW resolution case, the system needs to preserve the synonym name as
VIEW gets expanded to a subquery, the name to be set should be the exposed
name of the table and not the resolved table name.

For * expansion in the SELECT list, if the FROM clause happens to be a synonym,
the system should prepend it with the unbound name and not the resolved table
name. This way the binding logic is normalized.

For DML cases, the synonym name needs to be normalized to its base table so tha
setColumnDescriptor can apply correctly. When the system binds the expression
for this result column, it will resolve this properly since the column binding
logic are in the respective FromTable subclasses implementation where they
will use the exposed name this time to check for qualification.

I wrote more testcases for synonym.sql but I found out that this SQL file is
actually not part of derbylang suite, so the patch added this back to the test
bucket.


Modified:
    db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java
    db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
    db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/FromTable.java
    db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java
    db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/ResultColumn.java
    db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java
    db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/master/synonym.out
    db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/suites/derbylang.runall
    db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/tests/lang/synonym.sql

Modified: db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java?view=diff&rev=448016&r1=448015&r2=448016
==============================================================================
--- db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java (original)
+++ db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/DMLModStatementNode.java Tue Sep 19 17:09:18 2006
@@ -125,7 +125,8 @@
 	protected int[][] fkColArrays; 
 	protected Hashtable graphHashTable; 
                           // Hash Table which maitains the querytreenode graph 
-
+	protected TableName synonymTableName;
+	
 	/**
 	 * Initializer for a DMLModStatementNode -- delegate to DMLStatementNode
 	 *
@@ -226,6 +227,7 @@
 				TableName synonymTab = resolveTableToSynonym(targetTableName);
 				if (synonymTab == null)
 					throw StandardException.newException(SQLState.LANG_TABLE_NOT_FOUND, targetTableName);
+				synonymTableName = targetTableName;
 				targetTableName = synonymTab;
 				sdtc = getSchemaDescriptor(targetTableName.getSchemaName());
 
@@ -1576,6 +1578,50 @@
 								   dependentScan);
 	}
 
+	/**
+	 * Normalize synonym column references to have the name of the base table. 
+	 *
+	 * @param rcl	           The result column list of the target table
+	 * @param targetTableName  The target tablename
+	 *
+	 * @exception StandardException		Thrown on error
+	 */
+	public void normalizeSynonymColumns( 
+    ResultColumnList    rcl, 
+    TableName           targetTableName)
+		throws StandardException
+	{
+		if (synonymTableName == null) 
+            return;
+		
+		String synTableName = synonymTableName.getTableName();
+		
+		int	count = rcl.size();
+		for (int i = 0; i < count; i++)
+		{
+			ResultColumn    column    = (ResultColumn) rcl.elementAt(i);
+			ColumnReference	reference = column.getReference();
+
+			if ( reference != null )
+			{
+				String crTableName = reference.getTableName();
+				if ( crTableName != null )
+				{
+					if ( synTableName.equals( crTableName ) )
+					{
+						reference.setTableNameNode( targetTableName );
+					}
+					else
+					{
+						throw StandardException.newException(
+                                SQLState.LANG_TABLE_NAME_MISMATCH, 
+                                synTableName, 
+                                crTableName);
+					}
+				}
+			}
+		}
+	}
 }
 
 

Modified: db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java?view=diff&rev=448016&r1=448015&r2=448016
==============================================================================
--- db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java (original)
+++ db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java Tue Sep 19 17:09:18 2006
@@ -239,6 +239,7 @@
 			resultColumns = (ResultColumnList) arg3;
 		}
 
+		setOrigTableName(this.tableName);
 		templateColumns = resultColumns;
 	}
 
@@ -2213,7 +2214,8 @@
 				fsq = (FromTable) getNodeFactory().getNode(
 					C_NodeTypes.FROM_SUBQUERY,
 					rsn, 
-					(correlationName != null) ? correlationName : tableName.getTableName(), 
+					(correlationName != null) ? 
+                        correlationName : getOrigTableName().getTableName(), 
 					resultColumns,
 					tableProperties,
 					getContextManager());
@@ -2459,14 +2461,7 @@
 		** If there is a correlation name, use that instead of the
 		** table name.
 		*/
-		if (correlationName != null)
-		{
-			exposedTableName = makeTableName(null, correlationName);
-		}
-		else
-		{
-			exposedTableName = tableName;
-		}
+        exposedTableName = getExposedTableName();
 
         if(exposedTableName.getSchemaName() == null && correlationName == null)
             exposedTableName.bind(this.getDataDictionary());
@@ -3397,9 +3392,25 @@
 		if (correlationName != null)
 			return correlationName;
 		else
-			return tableName.getFullTableName();
+			return getOrigTableName().getFullTableName();
 	}
-
+	
+	/**
+	 * Get the exposed table name for this table, which is the name that can
+	 * be used to refer to it in the rest of the query.
+	 *
+	 * @return	TableName The exposed name of this table.
+	 *
+	 * @exception StandardException  Thrown on error
+	 */
+	private TableName getExposedTableName() throws StandardException  
+	{
+		if (correlationName != null)
+			return makeTableName(null, correlationName);
+		else
+			return getOrigTableName();
+	}
+	
 	/**
 	 * Return the table name for this table.
 	 *
@@ -3426,7 +3437,8 @@
 	public ResultColumnList getAllResultColumns(TableName allTableName)
 			throws StandardException
 	{
-		return getResultColumnsForList(allTableName, resultColumns, tableName);
+		return getResultColumnsForList(allTableName, resultColumns, 
+				getOrigTableName());
 	}
 
 	/**
@@ -3453,14 +3465,7 @@
 		 * The exposed name becomes the qualifier for each column
 		 * in the expanded list.
 		 */
-		if (correlationName == null)
-		{
-			exposedName = tableName;
-		}
-		else
-		{
-			exposedName = makeTableName(null, correlationName);
-		}
+		exposedName = getExposedTableName();
 
 		/* Add all of the columns in the table */
 		rcList = (ResultColumnList) getNodeFactory().getNode(
@@ -3529,14 +3534,7 @@
 		 * The exposed name becomes the qualifier for each column
 		 * in the expanded list.
 		 */
-		if (correlationName == null)
-		{
-			exposedName = tableName;
-		}
-		else
-		{
-			exposedName = makeTableName(null, correlationName);
-		}
+		exposedName = getExposedTableName();
 
 		/* Add all of the columns in the table */
 		ResultColumnList newRcl = (ResultColumnList) getNodeFactory().getNode(

Modified: db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/FromTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/FromTable.java?view=diff&rev=448016&r1=448015&r2=448016
==============================================================================
--- db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/FromTable.java (original)
+++ db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/FromTable.java Tue Sep 19 17:09:18 2006
@@ -112,6 +112,9 @@
   //Used by replication enabled databases where the target-only view failure is detected
   //using this boolean
   private boolean allColumnsProjected;
+  
+	/** the original unbound table name */
+	protected TableName origTableName;
 
   public boolean areAllColumnsProjected() {
     return allColumnsProjected;
@@ -135,6 +138,11 @@
 		optimizerToBestPlanMap = null;
 	}
 
+	/**
+	 * Get this table's correlation name, if any.
+	 */
+	public	String	getCorrelationName() { return correlationName; }
+
 	/*
 	 *  Optimizable interface
 	 */
@@ -1485,5 +1493,28 @@
 	public boolean needsSpecialRCLBinding()
 	{
 		return false;
+	}
+	/**
+	 * Sets the original or unbound table name for this FromTable.  
+	 * 
+	 * @param tableName the unbound table name
+	 *
+	 */
+	public void setOrigTableName(TableName tableName) 
+	{
+		this.origTableName = tableName;
+	}
+	
+	/**
+	 * Gets the original or unbound table name for this FromTable.  
+	 * The tableName field can be changed due to synonym resolution.
+	 * Use this method to retrieve the actual unbound tablename.
+	 * 
+	 * @return TableName the original or unbound tablename
+	 *
+	 */
+	public TableName getOrigTableName() 
+	{
+		return this.origTableName;
 	}
 }

Modified: db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java?view=diff&rev=448016&r1=448015&r2=448016
==============================================================================
--- db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java (original)
+++ db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java Tue Sep 19 17:09:18 2006
@@ -260,6 +260,14 @@
 		 */
 		if (targetColumnList != null)
 		{
+			/*
+			 * Normalize synonym qualifers for column references.
+			 */
+			if (synonymTableName != null)
+			{
+				normalizeSynonymColumns ( targetColumnList, targetTableName );
+			}
+			
 			/* Bind the target column list */
 			if (targetTableDescriptor != null)
 			{

Modified: db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/ResultColumn.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/ResultColumn.java?view=diff&rev=448016&r1=448015&r2=448016
==============================================================================
--- db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/ResultColumn.java (original)
+++ db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/ResultColumn.java Tue Sep 19 17:09:18 2006
@@ -1745,6 +1745,10 @@
     public TableName getTableNameObject() {
         return null;
     }
+
+	/* Get the wrapped reference if any */
+	public	ColumnReference	getReference() { return reference; }
+	
 	/**
 	 * Get the source BaseColumnNode for this result column. The
 	 * BaseColumnNode cannot be found unless the ResultColumn is bound

Modified: db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java?view=diff&rev=448016&r1=448015&r2=448016
==============================================================================
--- db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java (original)
+++ db/derby/code/branches/10.1/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java Tue Sep 19 17:09:18 2006
@@ -205,7 +205,10 @@
 		{
 			TableName synonymTab = resolveTableToSynonym(this.targetTableName);
 			if (synonymTab != null)
-				this.targetTableName = synonymTab;
+			{
+				this.synonymTableName = targetTableName;
+				this.targetTableName  = synonymTab;
+			}
 		}
 
 		bindTables(dataDictionary);
@@ -339,7 +342,13 @@
 		SanityManager.ASSERT(resultFromList.size() == 1,
 			"More than one table in result from list in an update.");
 
+		/* Normalize the SET clause's result column list for synonym */
+		if (synonymTableName != null)
+			normalizeSynonymColumns( resultSet.resultColumns, targetTable );
+		
 		/* Bind the original result columns by column name */
+		normalizeCorrelatedColumns( resultSet.resultColumns, targetTable );
+
  		resultSet.bindResultColumns(targetTableDescriptor,
 									targetVTI,
  									resultSet.resultColumns, this,
@@ -495,17 +504,12 @@
 			/* Append to the ResultColumnList */
         resultColumnList.addResultColumn(rowLocationColumn);
 
-		/* The last thing that we do to the generated RCL is to clear
-		 * the table name out from each RC.  The table name is
-		 * unnecessary for an update.  More importantly, though, it
-		 * creates a problem in the degenerate case with a positioned
-		 * update.  The user must specify the base table name for a
-		 * positioned update.  If a correlation name was specified for
-		 * the cursor, then a match for the ColumnReference would not
-		 * be found if we didn't null out the name.  (Aren't you
-		 * glad you asked?)
+		/*
+		 * The last thing that we do to the generated RCL is to clear
+		 * the table name out from each RC. See comment on 
+		 * checkTableNameAndScrubResultColumns().
 		 */
-		resultColumnList.clearTableNames();
+		checkTableNameAndScrubResultColumns(resultColumnList);
 
 		/* Set the new result column list in the result set */
 		resultSet.setResultColumns(resultColumnList);
@@ -999,4 +1003,124 @@
 
 		return	columnMap;
 	}
+
+	/*
+	 * Force correlated column references in the SET clause to have the
+	 * name of the base table. This dances around the problem alluded to
+	 * in scrubResultColumn().
+	 */
+	private	void	normalizeCorrelatedColumns( ResultColumnList rcl, FromTable fromTable )
+		throws StandardException
+	{
+		String		correlationName = fromTable.getCorrelationName();
+
+		if ( correlationName == null ) { return; }
+
+		TableName	tableNameNode;
+
+		if ( fromTable instanceof CurrentOfNode )
+		{ tableNameNode = ((CurrentOfNode) fromTable).getBaseCursorTargetTableName(); }
+		else { tableNameNode = makeTableName( null, fromTable.getBaseTableName() ); }
+		
+		int			count = rcl.size();
+
+		for ( int i = 0; i < count; i++ )
+		{
+			ResultColumn	column = (ResultColumn) rcl.elementAt( i );
+			ColumnReference	reference = column.getReference();
+
+			if ( (reference != null) && correlationName.equals( reference.getTableName() ) )
+			{
+				reference.setTableNameNode( tableNameNode );
+			}
+		}
+		
+	}
+
+	/**
+	 * Check table name and then clear it from the result set columns.
+	 * 
+	 * @exception StandardExcepion if invalid column/table is specified.
+	 */
+	private void checkTableNameAndScrubResultColumns(ResultColumnList rcl) 
+			throws StandardException
+	{
+		int columnCount = rcl.size();
+		int tableCount = ((SelectNode)resultSet).fromList.size();
+
+		for ( int i = 0; i < columnCount; i++ )
+		{
+			boolean foundMatchingTable = false;			
+			ResultColumn	column = (ResultColumn) rcl.elementAt( i );
+
+			if (column.getTableName() != null) {
+				for (int j = 0; j < tableCount; j++) {
+					FromTable fromTable = (FromTable) ((SelectNode)resultSet).
+							fromList.elementAt(j);
+					final String tableName;
+					if ( fromTable instanceof CurrentOfNode ) { 
+						tableName = ((CurrentOfNode)fromTable).
+								getBaseCursorTargetTableName().getTableName();
+					} else { 
+						tableName = fromTable.getBaseTableName();
+					}
+
+					if (column.getTableName().equals(tableName)) {
+						foundMatchingTable = true;
+						break;
+					}
+				}
+
+				if (!foundMatchingTable) {
+					throw StandardException.newException(
+							SQLState.LANG_COLUMN_NOT_FOUND, 
+							column.getTableName() + "." + column.getName());
+				}
+			}
+
+			/* The table name is
+			 * unnecessary for an update.  More importantly, though, it
+			 * creates a problem in the degenerate case with a positioned
+			 * update.  The user must specify the base table name for a
+			 * positioned update.  If a correlation name was specified for
+			 * the cursor, then a match for the ColumnReference would not
+			 * be found if we didn't null out the name.  (Aren't you
+			 * glad you asked?)
+			 */
+			column.clearTableName();
+		}
+	}
+	
+	/**
+	 * Normalize synonym column references to have the name of the base table. 
+	 *
+	 * @param rcl	    The result column list of the target table
+	 * @param fromTable The table name to set the column refs to
+	 * 
+	 * @exception StandardException		Thrown on error
+	 */
+	private	void normalizeSynonymColumns(
+    ResultColumnList    rcl, 
+    FromTable           fromTable)
+		throws StandardException
+	{
+		if (fromTable.getCorrelationName() != null) 
+        { 
+            return; 
+        }
+		
+		TableName tableNameNode;
+		if (fromTable instanceof CurrentOfNode)
+		{ 
+			tableNameNode = 
+                ((CurrentOfNode) fromTable).getBaseCursorTargetTableName(); 
+		}
+		else 
+		{ 
+			tableNameNode = makeTableName(null, fromTable.getBaseTableName()); 
+		}
+		
+		super.normalizeSynonymColumns(rcl, tableNameNode);
+	}
+	
 } // end of UpdateNode

Modified: db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/master/synonym.out
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/master/synonym.out?view=diff&rev=448016&r1=448015&r2=448016
==============================================================================
--- db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/master/synonym.out (original)
+++ db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/master/synonym.out Tue Sep 19 17:09:18 2006
@@ -408,4 +408,375 @@
 0 rows inserted/updated/deleted
 ij> drop table table1;
 0 rows inserted/updated/deleted
+ij> -- DERBY-1784
+create schema test1;
+0 rows inserted/updated/deleted
+ij> create schema test2;
+0 rows inserted/updated/deleted
+ij> create table test1.t1 ( id bigint not null );
+0 rows inserted/updated/deleted
+ij> insert into test1.t1 values 1;
+1 row inserted/updated/deleted
+ij> create synonym test2.t1 for test1.t1;
+0 rows inserted/updated/deleted
+ij> set schema test1;
+0 rows inserted/updated/deleted
+ij> select t1.id from t1;
+ID                  
+--------------------
+1                   
+ij> set schema test2;
+0 rows inserted/updated/deleted
+ij> select id from t1;
+ID                  
+--------------------
+1                   
+ij> select id from test2.t1;
+ID                  
+--------------------
+1                   
+ij> select t1.id from t1;
+ID                  
+--------------------
+1                   
+ij> select t1.id from test2.t1;
+ID                  
+--------------------
+1                   
+ij> select test2.t1.id from t1;
+ID                  
+--------------------
+1                   
+ij> select test2.t1.id from test2.t1;
+ID                  
+--------------------
+1                   
+ij> drop synonym t1;
+0 rows inserted/updated/deleted
+ij> drop table test1.t1;
+0 rows inserted/updated/deleted
+ij> set schema app;
+0 rows inserted/updated/deleted
+ij> create table A (id integer);
+0 rows inserted/updated/deleted
+ij> insert into A values 29;
+1 row inserted/updated/deleted
+ij> create synonym B for A;
+0 rows inserted/updated/deleted
+ij> select a.id from a;
+ID         
+-----------
+29         
+ij> select b.id from b;
+ID         
+-----------
+29         
+ij> select b.id from b as b;
+ID         
+-----------
+29         
+ij> select b.id from (select b.id from b) as b;
+ID         
+-----------
+29         
+ij> select b.id from (select b.id from b as b) as b;
+ID         
+-----------
+29         
+ij> drop synonym B;
+0 rows inserted/updated/deleted
+ij> drop table A;
+0 rows inserted/updated/deleted
+ij> create table t1 (i int, j int);
+0 rows inserted/updated/deleted
+ij> create view v1 as select * from t1;
+0 rows inserted/updated/deleted
+ij> insert into t1 values (1, 10);
+1 row inserted/updated/deleted
+ij> create synonym s1 for t1;
+0 rows inserted/updated/deleted
+ij> create synonym sv1 for v1;
+0 rows inserted/updated/deleted
+ij> -- should fail
+select t1.i from s1;
+ERROR 42X04: Column 'T1.I' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'T1.I' is not a column in the target table.
+ij> select v1.i from sv1;
+ERROR 42X04: Column 'V1.I' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'V1.I' is not a column in the target table.
+ij> select sv1.i from sv1 as w1;
+ERROR 42X04: Column 'SV1.I' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'SV1.I' is not a column in the target table.
+ij> select s1.j from s1 where s1.k = 1;
+ERROR 42X04: Column 'S1.K' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'S1.K' is not a column in the target table.
+ij> select s1.j from s1 where w1.i = 1;
+ERROR 42X04: Column 'W1.I' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'W1.I' is not a column in the target table.
+ij> select * from s1 where w1.i = 1;
+ERROR 42X04: Column 'W1.I' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'W1.I' is not a column in the target table.
+ij> select s1.j from s1 as w1 where w1.i = 1;
+ERROR 42X04: Column 'S1.J' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'S1.J' is not a column in the target table.
+ij> select w1.j from s1 as w1 where s1.i = 1;
+ERROR 42X04: Column 'S1.I' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'S1.I' is not a column in the target table.
+ij> select s1.j from s1 where t1.i = 1;
+ERROR 42X04: Column 'T1.I' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'T1.I' is not a column in the target table.
+ij> select s1.j from s1 group by t1.j;
+ERROR 42X04: Column 'T1.J' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'T1.J' is not a column in the target table.
+ij> select s1.j from s1 group by s1.j having t1.j > 0;
+ERROR 42X04: Column 'T1.J' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'T1.J' is not a column in the target table.
+ij> insert into s1 (t1.i) values 100;
+ERROR 42X55: Table name 'T1' should be the same as 'S1'.
+ij> update s1 set t1.i=1;
+ERROR 42X55: Table name 'T1' should be the same as 'S1'.
+ij> delete from s1 where t1.i=100;
+ERROR 42X04: Column 'T1.I' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'T1.I' is not a column in the target table.
+ij> -- ok
+select s1.i from s1;
+I          
+-----------
+1          
+ij> select s1.i from s1 as s1;
+I          
+-----------
+1          
+ij> select s1.i from s1 where i = 1;
+I          
+-----------
+1          
+ij> select s1.i from s1 where s1.i = 1;
+I          
+-----------
+1          
+ij> select s1.i from s1 as s1 where i = 1;
+I          
+-----------
+1          
+ij> select w1.i from s1 as w1 where w1.i = 1;
+I          
+-----------
+1          
+ij> select sv1.i from sv1;
+I          
+-----------
+1          
+ij> select sv1.i from sv1 as sv1;
+I          
+-----------
+1          
+ij> select sv1.i from sv1 where i = 1;
+I          
+-----------
+1          
+ij> select sv1.i from sv1 where sv1.i = 1;
+I          
+-----------
+1          
+ij> select sv1.i from sv1 as sv1 where i = 1;
+I          
+-----------
+1          
+ij> select wv1.i from sv1 as wv1 where wv1.i = 1;
+I          
+-----------
+1          
+ij> select s1.i, s1.i from s1;
+I          |I          
+-----------------------
+1          |1          
+ij> select sv1.i, sv1.i from sv1;
+I          |I          
+-----------------------
+1          |1          
+ij> select * from s1;
+I          |J          
+-----------------------
+1          |10         
+ij> select * from s1 where i = 1;
+I          |J          
+-----------------------
+1          |10         
+ij> select * from s1 where s1.i = 1;
+I          |J          
+-----------------------
+1          |10         
+ij> select * from s1 as s1;
+I          |J          
+-----------------------
+1          |10         
+ij> select * from s1 as w1;
+I          |J          
+-----------------------
+1          |10         
+ij> select * from sv1;
+I          |J          
+-----------------------
+1          |10         
+ij> select * from sv1 as sv1;
+I          |J          
+-----------------------
+1          |10         
+ij> select * from sv1 as w1;
+I          |J          
+-----------------------
+1          |10         
+ij> select * from sv1 where i = 1;
+I          |J          
+-----------------------
+1          |10         
+ij> select * from sv1 where sv1.i = 1;
+I          |J          
+-----------------------
+1          |10         
+ij> select s1.i from (select s1.i from s1) as s1;
+I          
+-----------
+1          
+ij> select sv1.i from (select sv1.i from sv1) as sv1;
+I          
+-----------
+1          
+ij> create table t2 (i int, j int);
+0 rows inserted/updated/deleted
+ij> insert into t2 values (1, 100), (1, 100), (2, 200);
+3 rows inserted/updated/deleted
+ij> create view v2 as select * from t2;
+0 rows inserted/updated/deleted
+ij> create synonym s2 for t2;
+0 rows inserted/updated/deleted
+ij> create synonym sv2 for v2;
+0 rows inserted/updated/deleted
+ij> select s2.j from s2 group by s2.j order by s2.j;
+J          
+-----------
+100        
+200        
+ij> select s2.j from s2 group by s2.j having s2.j > 100 order by s2.j;
+J          
+-----------
+200        
+ij> select s1.i, s1.j from (select s1.i, s2.j from s1,s2 where s1.i=s2.i) as s1;
+I          |J          
+-----------------------
+1          |100        
+1          |100        
+ij> select sv2.j from sv2 group by sv2.j order by sv2.j;
+J          
+-----------
+100        
+200        
+ij> select sv2.j from sv2 group by sv2.j having sv2.j > 100 order by sv2.j;
+J          
+-----------
+200        
+ij> select sv1.i, sv1.j from (select sv1.i, sv2.j from sv1,sv2 where sv1.i=sv2.i) as sv1;
+I          |J          
+-----------------------
+1          |100        
+1          |100        
+ij> select max(s2.i) from s2;
+1          
+-----------
+2          
+ij> select max(sv2.i) from sv2;
+1          
+-----------
+2          
+ij> select * from s1 inner join s2 on (s1.i = s2.i);
+I          |J          |I          |J          
+-----------------------------------------------
+1          |10         |1          |100        
+1          |10         |1          |100        
+ij> select * from sv1 inner join sv2 on (sv1.i = sv2.i);
+I          |J          |I          |J          
+-----------------------------------------------
+1          |10         |1          |100        
+1          |10         |1          |100        
+ij> select s1.* from s1;
+I          |J          
+-----------------------
+1          |10         
+ij> select sv1.* from sv1;
+I          |J          
+-----------------------
+1          |10         
+ij> create table t3 (i int, j int);
+0 rows inserted/updated/deleted
+ij> insert into t3 values (10, 0), (11, 0), (12, 0);
+3 rows inserted/updated/deleted
+ij> create synonym s3 for t3;
+0 rows inserted/updated/deleted
+ij> insert into s1 (s1.i, s1.j) values (2, 20);
+1 row inserted/updated/deleted
+ij> insert into app.s1 (s1.i, s1.j) values (3, 30);
+1 row inserted/updated/deleted
+ij> insert into app.s1 (app.s1.i, s1.j) values (4, 40);
+1 row inserted/updated/deleted
+ij> insert into app.s1 (app.s1.i, app.s1.j) values (5, 50);
+1 row inserted/updated/deleted
+ij> update s1 set s1.j = 1;
+5 rows inserted/updated/deleted
+ij> update app.s1 set s1.j = 2;
+5 rows inserted/updated/deleted
+ij> update app.s1 set app.s1.j = 3;
+5 rows inserted/updated/deleted
+ij> update s1 set s1.j = 4 where s1.i = 3;
+1 row inserted/updated/deleted
+ij> update app.s1 set app.s1.j = 5 where app.s1.i = 4;
+1 row inserted/updated/deleted
+ij> delete from s1 where s1.i = 4;
+1 row inserted/updated/deleted
+ij> delete from app.s1 where app.s1.i = 5;
+1 row inserted/updated/deleted
+ij> update app.s1 set s1.j = s1.i, s1.i = s1.j;
+3 rows inserted/updated/deleted
+ij> select * from s1;
+I          |J          
+-----------------------
+3          |1          
+3          |2          
+4          |3          
+ij> update app.s1 set s1.j = s1.i, s1.i = s1.j;
+3 rows inserted/updated/deleted
+ij> select * from s1;
+I          |J          
+-----------------------
+1          |3          
+2          |3          
+3          |4          
+ij> delete from s1;
+3 rows inserted/updated/deleted
+ij> -- should fail
+insert into s1 (s1.i) select s1.i from s3;
+ERROR 42X04: Column 'S1.I' is either not in any table in the FROM list or appears within a join specification and is outside the scope of the join specification or appears in a HAVING clause and is not in the GROUP BY list. If this is a CREATE or ALTER TABLE  statement then 'S1.I' is not a column in the target table.
+ij> -- ok
+insert into s1 (s1.i) select s3.i from s3;
+3 rows inserted/updated/deleted
+ij> insert into s1 select * from s3;
+3 rows inserted/updated/deleted
+ij> select * from s1;
+I          |J          
+-----------------------
+10         |NULL       
+11         |NULL       
+12         |NULL       
+10         |0          
+11         |0          
+12         |0          
+ij> -- clean up  
+drop synonym s3;
+0 rows inserted/updated/deleted
+ij> drop synonym sv2;
+0 rows inserted/updated/deleted
+ij> drop synonym s2;
+0 rows inserted/updated/deleted
+ij> drop synonym s1;
+0 rows inserted/updated/deleted
+ij> drop synonym sv1;
+0 rows inserted/updated/deleted
+ij> drop view v2;
+0 rows inserted/updated/deleted
+ij> drop view v1;
+0 rows inserted/updated/deleted
+ij> drop table t3;
+0 rows inserted/updated/deleted
+ij> drop table t2;
+0 rows inserted/updated/deleted
+ij> drop table t1;
+0 rows inserted/updated/deleted
 ij> 

Modified: db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/suites/derbylang.runall
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/suites/derbylang.runall?view=diff&rev=448016&r1=448015&r2=448016
==============================================================================
--- db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/suites/derbylang.runall (original)
+++ db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/suites/derbylang.runall Tue Sep 19 17:09:18 2006
@@ -125,6 +125,7 @@
 lang/subquery2.sql
 lang/subqueryFlattening.sql
 lang/supersimple.sql
+lang/synonym.sql
 lang/syscat.sql
 lang/tempRestrictions.sql
 lang/timestampArith.java

Modified: db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/tests/lang/synonym.sql
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/tests/lang/synonym.sql?view=diff&rev=448016&r1=448015&r2=448016
==============================================================================
--- db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/tests/lang/synonym.sql (original)
+++ db/derby/code/branches/10.1/java/testing/org/apache/derbyTesting/functionTests/tests/lang/synonym.sql Tue Sep 19 17:09:18 2006
@@ -258,3 +258,141 @@
 drop view view1;
 drop table table1;
 
+-- DERBY-1784
+create schema test1;
+create schema test2;
+create table test1.t1 ( id bigint not null );
+insert into test1.t1 values 1;
+create synonym test2.t1 for test1.t1;
+set schema test1;
+select t1.id from t1;
+set schema test2;
+select id from t1;
+select id from test2.t1;
+select t1.id from t1;
+select t1.id from test2.t1;
+select test2.t1.id from t1;
+select test2.t1.id from test2.t1;
+drop synonym t1;
+drop table test1.t1;
+
+set schema app;
+create table A (id integer);
+insert into A values 29;
+create synonym B for A;
+select a.id from a;
+select b.id from b;
+select b.id from b as b;
+select b.id from (select b.id from b) as b;
+select b.id from (select b.id from b as b) as b;
+drop synonym B;
+drop table A;
+
+create table t1 (i int, j int);
+create view v1 as select * from t1;
+insert into t1 values (1, 10);
+create synonym s1 for t1;
+create synonym sv1 for v1;
+-- should fail
+select t1.i from s1;
+select v1.i from sv1;
+select sv1.i from sv1 as w1;
+select s1.j from s1 where s1.k = 1;
+select s1.j from s1 where w1.i = 1;
+select * from s1 where w1.i = 1;
+select s1.j from s1 as w1 where w1.i = 1;
+select w1.j from s1 as w1 where s1.i = 1;
+select s1.j from s1 where t1.i = 1;
+select s1.j from s1 group by t1.j;
+select s1.j from s1 group by s1.j having t1.j > 0;
+insert into s1 (t1.i) values 100;
+update s1 set t1.i=1;
+delete from s1 where t1.i=100;
+
+-- ok
+select s1.i from s1;
+select s1.i from s1 as s1;
+select s1.i from s1 where i = 1;
+select s1.i from s1 where s1.i = 1;
+select s1.i from s1 as s1 where i = 1;
+select w1.i from s1 as w1 where w1.i = 1;
+select sv1.i from sv1;
+select sv1.i from sv1 as sv1;
+select sv1.i from sv1 where i = 1;
+select sv1.i from sv1 where sv1.i = 1;
+select sv1.i from sv1 as sv1 where i = 1;
+select wv1.i from sv1 as wv1 where wv1.i = 1;
+
+select s1.i, s1.i from s1;
+select sv1.i, sv1.i from sv1;
+select * from s1;
+select * from s1 where i = 1;
+select * from s1 where s1.i = 1;
+select * from s1 as s1;
+select * from s1 as w1;
+select * from sv1;
+select * from sv1 as sv1;
+select * from sv1 as w1;
+select * from sv1 where i = 1;
+select * from sv1 where sv1.i = 1;
+select s1.i from (select s1.i from s1) as s1;
+select sv1.i from (select sv1.i from sv1) as sv1;
+
+create table t2 (i int, j int);
+insert into t2 values (1, 100), (1, 100), (2, 200);
+create view v2 as select * from t2;
+create synonym s2 for t2;
+create synonym sv2 for v2;
+select s2.j from s2 group by s2.j order by s2.j;
+select s2.j from s2 group by s2.j having s2.j > 100 order by s2.j;
+select s1.i, s1.j from (select s1.i, s2.j from s1,s2 where s1.i=s2.i) as s1;
+select sv2.j from sv2 group by sv2.j order by sv2.j;
+select sv2.j from sv2 group by sv2.j having sv2.j > 100 order by sv2.j;
+select sv1.i, sv1.j from (select sv1.i, sv2.j from sv1,sv2 where sv1.i=sv2.i) as sv1;
+select max(s2.i) from s2;
+select max(sv2.i) from sv2;
+select * from s1 inner join s2 on (s1.i = s2.i);
+select * from sv1 inner join sv2 on (sv1.i = sv2.i);
+select s1.* from s1;
+select sv1.* from sv1;
+
+create table t3 (i int, j int);
+insert into t3 values (10, 0), (11, 0), (12, 0);
+create synonym s3 for t3;
+insert into s1 (s1.i, s1.j) values (2, 20);
+insert into app.s1 (s1.i, s1.j) values (3, 30);
+insert into app.s1 (app.s1.i, s1.j) values (4, 40);
+insert into app.s1 (app.s1.i, app.s1.j) values (5, 50);
+update s1 set s1.j = 1;
+update app.s1 set s1.j = 2;
+update app.s1 set app.s1.j = 3;
+update s1 set s1.j = 4 where s1.i = 3;
+update app.s1 set app.s1.j = 5 where app.s1.i = 4;
+delete from s1 where s1.i = 4;
+delete from app.s1 where app.s1.i = 5;
+update app.s1 set s1.j = s1.i, s1.i = s1.j;
+select * from s1;
+update app.s1 set s1.j = s1.i, s1.i = s1.j;
+select * from s1;
+delete from s1;
+
+-- should fail
+insert into s1 (s1.i) select s1.i from s3;
+
+-- ok
+insert into s1 (s1.i) select s3.i from s3;
+insert into s1 select * from s3;
+select * from s1;
+
+-- clean up  
+drop synonym s3;
+drop synonym sv2;
+drop synonym s2;
+drop synonym s1;
+drop synonym sv1;
+drop view v2;
+drop view v1;
+drop table t3;
+drop table t2;
+drop table t1;
+