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 ab...@apache.org on 2006/12/19 23:10:57 UTC

svn commit: r488827 - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/sql/dictionary/ engine/org/apache/derby/impl/sql/catalog/ engine/org/apache/derby/impl/sql/compile/ testing/org/apache/derbyTesting/functionTests/master/ testing/org/apach...

Author: abrown
Date: Tue Dec 19 14:10:56 2006
New Revision: 488827

URL: http://svn.apache.org/viewvc?view=rev&rev=488827
Log:
DERBY-2152: Support diagnostic "table functions" for querying Derby diagnostic
VTIs that take parameters.  The table functions are exposed via the TABLE
constructor syntax:

  SELECT <rcList> from
    TABLE ( <sysdiag-schema>.<vti-table-name> (<arg-list>) )
    [ AS ] corrlationName

 Note that:

  a. We only support VTI table function names that are in the SYSCS_DIAG schema
  b. The correlation name *is* required, though use of the "AS" keyword
     is optional (section 7.6 of the SQL 2003 spec, "<table primary>").
  c. The argument list can be empty if the underlying VTI supports it.

The VTI table names that have been added are as follows:

  SYSCS_DIAG.SPACE_TABLE maps to org.apache.derby.diag.SpaceTable
  SYSCS_DIAG.ERROR_LOG_READER maps to org.apache.derby.diag.ErrorLogReader
  SYSCS_DIAG.STATEMENT_DURATION maps to org.apache.derby.diag.StatementDuration

Added:
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/SysDiagVTIMappingTest.java   (with props)
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/sys_vti_test_derby.tstlog   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionary.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NewInvocationNode.java
    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/valuesclause.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/build.xml

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionary.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionary.java?view=diff&rev=488827&r1=488826&r2=488827
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionary.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionary.java Tue Dec 19 14:10:56 2006
@@ -1699,10 +1699,22 @@
 	 * the virtual table. Assumes the descriptor is
 	 * of type TableDescriptor.VTI_TYPE.
 	 */
-	public String getVTIClass(TableDescriptor td)
+	public String getVTIClassForTable(TableDescriptor td)
 		throws StandardException;
 
-	
+	/**
+	 * Return the Java class to use for the VTI to which the received table
+	 * function name maps.  The table function name is a qualified name and
+	 * thus has two parts: a schema name and a function name.
+	 *
+	 * @param funcSchema Schema part of the function name
+	 * @param funcName Actual name of the function
+	 * @return Java class name to which <funcSchema>.<funcName> maps, or
+	 *  null if no mapping is found.
+	 */
+	public String getVTIClassForTableFunction(String funcSchema,
+		String funcName) throws StandardException;
+
 	/**
 	 * Adds a descriptor to a system catalog identified by the catalogNumber. 
 	 *

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java?view=diff&rev=488827&r1=488826&r2=488827
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java Tue Dec 19 14:10:56 2006
@@ -1781,7 +1781,7 @@
 						TableDescriptor.DEFAULT_LOCK_GRANULARITY);
 			
 			// ensure a vti class exists
-			if (getVTIClass(td) != null)
+			if (getVTIClassForTable(td) != null)
 				return td;
 			
 			// otherwise just standard search
@@ -10408,7 +10408,7 @@
     	}
     } // end of getUncachedRoutinePermsDescriptor
  
-	private String[][] DIAG_VTI_CLASSES =
+	private String[][] DIAG_VTI_TABLE_CLASSES =
 	{
 			{"LOCK_TABLE", "org.apache.derby.diag.LockTable"},
 			{"STATEMENT_CACHE", "org.apache.derby.diag.StatementCache"},
@@ -10418,10 +10418,18 @@
 			
 	};
 	
+	private String[][] DIAG_VTI_TABLE_FUNCTION_CLASSES =
+	{
+			{"SPACE_TABLE", "org.apache.derby.diag.SpaceTable"},
+			{"ERROR_LOG_READER", "org.apache.derby.diag.ErrorLogReader"},
+			{"STATEMENT_DURATION", "org.apache.derby.diag.StatementDuration"},
+	};
+
 	/**
-	 * @see org.apache.derby.iapi.sql.dictionary.DataDictionary#getVTIClass(org.apache.derby.iapi.sql.dictionary.TableDescriptor)
+	 * @see DataDictionary#getVTIClassForTable(TableDescriptor)
 	 */
-	public String getVTIClass(TableDescriptor td) throws StandardException {
+	public String getVTIClassForTable(TableDescriptor td)
+		throws StandardException {
 		
 		if (SanityManager.DEBUG)
 		{
@@ -10429,10 +10437,32 @@
 				SanityManager.THROWASSERT("getVTIClass: Invalid table type " + td);
 		}
 		
-		for (int i = 0; i < DIAG_VTI_CLASSES.length; i++)
+		for (int i = 0; i < DIAG_VTI_TABLE_CLASSES.length; i++)
 		{
-			String[] entry = DIAG_VTI_CLASSES[i];
+			String[] entry = DIAG_VTI_TABLE_CLASSES[i];
 			if (entry[0].equals(td.getDescriptorName()))
+				return entry[1];	
+		}	
+		
+		return null;
+	}
+
+	/**
+	 * @see DataDictionary#getVTIClassForTableFunction(String, String)
+	 */
+	public String getVTIClassForTableFunction(String funcSchema,
+		String funcName) throws StandardException
+	{
+		/* For now we only recognize table function names that are in
+		 * the "SYSCS_DIAG" schema; for anything else just return null.
+		 */
+		if (!SchemaDescriptor.STD_SYSTEM_DIAG_SCHEMA_NAME.equals(funcSchema))
+			return null;
+
+		for (int i = 0; i < DIAG_VTI_TABLE_FUNCTION_CLASSES.length; i++)
+		{
+			String[] entry = DIAG_VTI_TABLE_FUNCTION_CLASSES[i];
+			if (entry[0].equals(funcName))
 				return entry[1];	
 		}	
 		

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java?view=diff&rev=488827&r1=488826&r2=488827
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java Tue Dec 19 14:10:56 2006
@@ -2110,7 +2110,7 @@
 		if (tableDescriptor.getTableType() == TableDescriptor.VTI_TYPE) {
 			ResultSetNode vtiNode = getNodeFactory().mapTableAsVTI(
 					tableDescriptor,
-					dataDictionary.getVTIClass(tableDescriptor),
+					dataDictionary.getVTIClassForTable(tableDescriptor),
 					getCorrelationName(),
 					resultColumns,
 					getProperties(),

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NewInvocationNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NewInvocationNode.java?view=diff&rev=488827&r1=488826&r2=488827
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NewInvocationNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NewInvocationNode.java Tue Dec 19 14:10:56 2006
@@ -37,6 +37,8 @@
 import org.apache.derby.iapi.sql.compile.CompilerContext;
 
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
+import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
+import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 
 import org.apache.derby.iapi.reference.SQLState;
 
@@ -68,7 +70,9 @@
 	 * Initializer for a NewInvocationNode. Parameters are:
 	 *
 	 * <ul>
-	 * <li>javaClassName		The full package.class name of the class</li>
+	 * <li>javaClassName		The full package.class name of the class
+	 * 	                    	(as a String), or else a TableName object
+	 *                  		that maps to the full class name </li>
 	 * <li>parameterList		The parameter list for the constructor</li>
 	 * </ul>
 	 *
@@ -83,7 +87,50 @@
 		super.init("<init>");
 		addParms((Vector) params);
 
-		this.javaClassName = (String) javaClassName;
+		/* If javaClassName is a String then it is the full package
+		 * class name for the class to be invoked, so just store it
+		 * locally.
+		 */
+		if (javaClassName instanceof String)
+			this.javaClassName = (String) javaClassName;
+		else
+		{
+			/* javaClassName is a TableName object representing a table
+			 * function name that maps to some VTI class name.  For
+			 * example, in the following query:
+			 *
+			 *   select * from TABLE(SYSCS_DIAG.SPACE_TABLE(?)) x
+			 *
+			 * javaClassName will be a TableName object representing
+			 * the table function name "SYSCS_DIAG.SPACE_TABLE".  So
+			 * we need to look up that TableName to figure out what
+			 * the corresponding target class name should be.  We
+			 * figure that out by using the data dictionary.
+			 */
+			TableName funcName = (TableName)javaClassName;
+
+			/* If no schema was specified then we want to default to the
+			 * current schema; that's what the following line does.
+			 */
+			String funcSchema =
+				getSchemaDescriptor(funcName.getSchemaName()).getSchemaName();
+
+			this.javaClassName =
+				getDataDictionary().getVTIClassForTableFunction(
+					funcSchema, funcName.getTableName());
+
+			/* If javaClassName is still null at this point then we
+			 * could not find the target class for the received
+			 * table function name.
+			 */
+			if (this.javaClassName == null)
+			{
+				throw StandardException.newException(
+					SQLState.LANG_NO_SUCH_METHOD_ALIAS,
+					funcName.getFullTableName());
+			}
+		}
+
 		this.delimitedIdentifier =
 				 ((Boolean) delimitedIdentifier).booleanValue();
 	}

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?view=diff&rev=488827&r1=488826&r2=488827
==============================================================================
--- 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 Tue Dec 19 14:10:56 2006
@@ -7375,6 +7375,61 @@
 }
 
 /*
+ * <A NAME="vtiTableConstruct">vtiTableConstruct</A>
+ *
+ * Parse a TABLE() constructor that corresponds to an internal
+ * VTI invocation.  For example:
+ *
+ *    TABLE ( <qualifiedName> (arg1, arg2, ...) )
+ *
+ * where <qualifiedName> is a table name that Derby will map internally
+ * to a VTI (ex. "SYSCS_DIAG.SPACE_TABLE").  The list of arguments
+ * will then be passed to the VTI when it is invoked (DERBY-2152).
+ *
+ * An example query where this might occur is as follows:
+ *
+ *   SELECT * FROM TABLE(SYSCS_DIAG.SPACE_TABLE('APP', 'T1')) x
+ *
+ * in which case SYSCS_DIAG.SPACE_TABLE will be mapped (internally)
+ * to the "org.apache.derby.diag.SpaceTable" diagnostic VTI.  Thus
+ * the equivalent call prior to DERBY-2152 would have been:
+ *
+ *   SELECT * FROM NEW org.apache.derby.diag.SpaceTable('APP', 'T1')) x
+ *
+ * Note that this latter syntax is still supported.
+ */
+JavaToSQLValueNode
+vtiTableConstruct() throws StandardException :
+{
+    QueryTreeNode newNode = null;
+    Vector parameterList = new Vector();
+    TableName vtiTableName = null;
+}
+{
+    <TABLE> <LEFT_PAREN>
+        vtiTableName = qualifiedName(Limits.MAX_IDENTIFIER_LENGTH)
+        methodCallParameterList(parameterList)
+    <RIGHT_PAREN>
+    {
+        newNode = nodeFactory.getNode(C_NodeTypes.NEW_INVOCATION_NODE,
+                    vtiTableName,
+                    parameterList, 
+                    lastTokenDelimitedIdentifier,
+                    getContextManager());
+
+        /*
+        ** Assume this is being returned to the SQL domain.  If it turns
+        ** out that this is being returned to the Java domain, we will
+        ** get rid of this node.
+        */
+        return (JavaToSQLValueNode) nodeFactory.getNode(
+                    C_NodeTypes.JAVA_TO_SQL_VALUE_NODE,
+                    newNode,
+                    getContextManager());
+    }
+}
+
+/*
  * <A NAME="staticMethodInvocation">staticMethodInvocation</A>
  */
 ValueNode
@@ -8554,14 +8609,19 @@
 	FromTable tableReference;
 }
 {
-	<TABLE>
-	LOOKAHEAD( {
-				getToken(1).kind == LEFT_PAREN &&
+	/* If we have a table constructor and the expression is a SELECT
+	 * query or a VALUES query then we read the <TABLE> keyword;
+	 * otherwise we leave the <TABLE> token in the queue and let
+	 * other types of expressions (namely, vtiTableConstruct())
+	 * deal with it accordingly.  DERBY-2152.
+	 */
+	LOOKAHEAD({ getToken(1).kind == TABLE &&
+				getToken(2).kind == LEFT_PAREN &&
 				(
-					getToken(2).kind == SELECT ||
-					getToken(2).kind == VALUES
+					getToken(3).kind == SELECT ||
+					getToken(3).kind == VALUES
 				)
-			} )
+			}) <TABLE>
 	tableReference = tableReferenceTypes(false)
 	{
 		fromList.addFromTable(tableReference);
@@ -8656,8 +8716,10 @@
 	 * a reserved word.
 	 */
 	/* identifier() used to be correlationName() */
-	LOOKAHEAD( { newInvocationFollows(1) } )
-	javaToSQLNode = newInvocation() [ <AS> ] correlationName = identifier(Limits.MAX_IDENTIFIER_LENGTH, true)
+	(LOOKAHEAD({ newInvocationFollows(1) }) javaToSQLNode = newInvocation()
+	| javaToSQLNode = vtiTableConstruct()
+	)
+		[ <AS> ] correlationName = identifier(Limits.MAX_IDENTIFIER_LENGTH, true)
 		[ <LEFT_PAREN> derivedRCL = derivedColumnList() <RIGHT_PAREN> ]
 		[ optionalTableClauses = optionalTableProperties() ]
 		(LOOKAHEAD( { joinedTableExpressionFollows() } )

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/valuesclause.out
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/valuesclause.out?view=diff&rev=488827&r1=488826&r2=488827
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/valuesclause.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/valuesclause.out Tue Dec 19 14:10:56 2006
@@ -413,13 +413,13 @@
 ij> -- should fail because <TABLE> can appear in front of derived table
 select *
 from  table target;
-ERROR 42X01: Syntax error: Encountered "" at line 3, column 13.
+ERROR 42X01: Syntax error: Encountered "target" at line 3, column 13.
 ij> select *
 from  table (target);
-ERROR 42X01: Syntax error: Encountered "" at line 2, column 13.
+ERROR 42X01: Syntax error: Encountered ")" at line 2, column 20.
 ij> select *
 from  table (target as q);
-ERROR 42X01: Syntax error: Encountered "" at line 2, column 13.
+ERROR 42X01: Syntax error: Encountered "as" at line 2, column 21.
 ij> drop table sub;
 0 rows inserted/updated/deleted
 ij> drop table target;

Added: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/SysDiagVTIMappingTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/SysDiagVTIMappingTest.java?view=auto&rev=488827
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/SysDiagVTIMappingTest.java (added)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/SysDiagVTIMappingTest.java Tue Dec 19 14:10:56 2006
@@ -0,0 +1,655 @@
+/*
+ *
+ * Derby - Class org.apache.derbyTesting.functionTests.tests.lang.SysDiagVTIMappingTest
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, 
+ * software distributed under the License is distributed on an 
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 
+ * either express or implied. See the License for the specific 
+ * language governing permissions and limitations under the License.
+ */
+package org.apache.derbyTesting.functionTests.tests.lang;
+
+import junit.framework.Test;
+import junit.framework.TestSuite;
+
+import org.apache.derbyTesting.junit.JDBC;
+import org.apache.derbyTesting.junit.BaseJDBCTestCase;
+import org.apache.derbyTesting.junit.SecurityManagerSetup;
+import org.apache.derbyTesting.junit.SupportFilesSetup;
+import org.apache.derbyTesting.junit.TestConfiguration;
+
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.sql.CallableStatement;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+
+public final class SysDiagVTIMappingTest extends BaseJDBCTestCase {
+
+    // Name of the log file to use when testing VTIs that expect one.
+    private static final String testLogFile = "sys_vti_test_derby.tstlog";
+
+    /**
+     * Public constructor required for running test as standalone JUnit.
+     */
+    public SysDiagVTIMappingTest(String name)
+    {
+        super(name);
+    }
+
+    public static Test suite()
+    {
+        TestSuite suite = new TestSuite("Diagnostic VTI Table Mappings");
+        suite.addTest(TestConfiguration.
+            defaultSuite(SysDiagVTIMappingTest.class));
+
+        /* Some of the VTIs that are tested in this class require a derby.log
+         * file.  We have a test log file stored in the tests/lang directory,
+         * and since the VTIs are going to try to read it, the test log file
+         * must be in a directory for which Derby has read access.  By
+         * using a SupportFilesSetup wrapper, we copy the test log file to
+         * the "extin" directory, which has the required permissions.
+         */
+        return SecurityManagerSetup.noSecurityManager(
+            new SupportFilesSetup(suite,
+                new String [] {
+                    "functionTests/tests/lang/" + testLogFile
+                }));
+    }
+
+    public void setUp() throws Exception
+    {
+        createStatement().execute("create table app.t1 (i int, c varchar(10))");
+        createStatement().execute("insert into app.t1 values (1, 'one'), "
+            + "(2, 'two'), (4, 'four')");
+    }
+
+    public void tearDown() throws Exception
+    {
+        createStatement().execute("drop table app.t1");
+    }
+
+    /**
+     * We use the SpaceTable VTI as our primary test VTI to verify that
+     * that VTI table mappings in general are working as expected.  So
+     * this method does a lot more than the other VTI test methods;
+     * the other test methods just do sanity checks to make sure that
+     * the mapping from "SYSCS_DIAG.<vti_table>" to the actual VTI
+     * class names in question is working correctly.
+     */
+    public void testSpaceTable() throws Exception
+    {
+        Statement st = createStatement();
+        st.executeUpdate("set schema APP");
+        
+        // Should fail because SPACE_TABLE is not defined in APP 
+        // schema.
+        
+        assertStatementError("42Y03", st,
+            "select * from TABLE(SPACE_TABLE('APP')) x");
+        
+        assertStatementError("42Y03", st,
+            "select * from TABLE(APP.SPACE_TABLE('APP', 'T1')) x");
+        
+        // Should fail due to extra "TABLE" keyword.
+        
+        assertStatementError("42X01", st,
+            "select * from TABLE TABLE(SYSCS_DIAG.SPACE_TABLE('T1')) x");
+        
+        assertStatementError("42X01", st,
+            "select * from TABLE TABLE (select * from t1) x");
+        
+        // Should fail because the specified schema does not exist.
+        
+        assertStatementError("42Y07", st,
+            "select * from TABLE(SYSCS_DIAG.SPACE_TABLE('T1', 'APP')) x");
+        
+        assertStatementError("42Y07", st,
+            "select * from "
+            + "TABLE(SYSCS_DIAG.SPACE_TABLE('NOTTHERE', 'T1')) x");
+        
+        // Should fail because SPACE_TABLE is not defined in APP schema.
+        
+        st.executeUpdate("set schema SYSCS_DIAG");
+        assertStatementError("42Y03", st,
+            "select * from TABLE(APP.SPACE_TABLE('APP', 'T1')) x");
+        
+        // All remaining test cases in this method should succeed.
+        
+        st.executeUpdate("set schema APP");
+
+        // These should all return 1 row for APP.T1.
+        
+        // Two-argument direct call.
+        ResultSet rs = st.executeQuery(
+            "select * from TABLE(SYSCS_DIAG.SPACE_TABLE('APP', 'T1')) x");
+        
+        String [] expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        
+        String [][] expRS = new String [][]
+        {
+            {"T1", "0", "1", "0", "1", "4096", "0"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+        
+        // Single-argument direct execution.
+        rs = st.executeQuery(
+            " select * from TABLE(SYSCS_DIAG.SPACE_TABLE('T1')) x");
+        
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        
+        expRS = new String [][]
+        {
+            {"T1", "0", "1", "0", "1", "4096", "0"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+        
+        // Two-argument prepare and execute.
+        PreparedStatement pSt = prepareStatement(
+            "select * from TABLE(SYSCS_DIAG.SPACE_TABLE(?, ?)) x");
+
+        pSt.setString(1, "APP");
+        pSt.setString(2, "T1");
+
+        rs = pSt.executeQuery();
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+             "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        
+        expRS = new String [][]
+        {
+            {"T1", "0", "1", "0", "1", "4096", "0"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+        
+        // Single-argument prepare and execute.
+        pSt = prepareStatement(
+            "select * from TABLE(SYSCS_DIAG.SPACE_TABLE(?)) x");
+
+        pSt.setString(1, "T1");
+
+        rs = pSt.executeQuery();
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        
+        expRS = new String [][]
+        {
+            {"T1", "0", "1", "0", "1", "4096", "0"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+        
+        // Statment should succeed but there will be no rows 
+        // because the tables do not exist.
+        
+        rs = st.executeQuery(
+            "select * from TABLE(SYSCS_DIAG.SPACE_TABLE('APP')) x");
+        
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        JDBC.assertDrainResults(rs, 0);
+        
+        rs = st.executeQuery(
+            "select * from TABLE(SYSCS_DIAG.SPACE_TABLE('APP', "
+            + "'NOTTHERE')) x");
+        
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        JDBC.assertDrainResults(rs, 0);
+        
+        rs = st.executeQuery(
+            "select * from "
+            + "TABLE(SYSCS_DIAG.SPACE_TABLE('SYSCS_DIAG', 'NOTTHERE')) x");
+        
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        JDBC.assertDrainResults(rs, 0);
+        
+        // Should see zero rows since LOCK_TABLE does not exist as 
+        // an actual base table (it's another VTI).
+        
+        rs = st.executeQuery(
+            "select * from "
+            + "TABLE(SYSCS_DIAG.SPACE_TABLE('SYSCS_DIAG', 'LOCK_TABLE')) x");
+        
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        JDBC.assertDrainResults(rs, 0);
+        
+        // Similar tests but in the SYSCS_DIAG schema, in which 
+        // case the schema-name for SPACE_TABLE should not be required.
+        
+        st.executeUpdate("set schema syscs_diag");
+        
+        // Should see 1 row for APP.T1.
+        
+        pSt = prepareStatement(
+            "select * from TABLE(SPACE_TABLE(?, ?)) x");
+        
+        pSt.setString(1, "APP");
+        pSt.setString(2, "T1");
+
+        rs = pSt.executeQuery();
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        
+        expRS = new String [][]
+        {
+            {"T1", "0", "1", "0", "1", "4096", "0"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+        
+        // Should see zero rows since T1 does not exist within 
+        // schema "SYSCS_DIAG".
+        
+        pSt = prepareStatement("select * from TABLE(SPACE_TABLE(?)) x");
+        pSt.setString(1, "T1");
+
+        rs = pSt.executeQuery();
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        JDBC.assertDrainResults(rs, 0);
+        
+        // Should see zero rows since LOCK_TABLE does not exist as 
+        // an actual base table (it's another VTI).
+        
+        rs = st.executeQuery(
+            "select * from TABLE(SPACE_TABLE('LOCK_TABLE')) x");
+        
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        JDBC.assertDrainResults(rs, 0);
+        
+        // Simple check to ensure that we we can join with the VTI.
+        
+        st.executeUpdate("set schema app");
+        rs = st.executeQuery(
+            "select cast (conglomeratename as varchar(30)), t1.* from"
+            + "  TABLE(SYSCS_DIAG.SPACE_TABLE('APP', 'T1')) x,"
+            + "  t1"
+            + " where x.conglomeratename is not null");
+        
+        expColNames = new String [] {"1", "I", "C"};
+        JDBC.assertColumnNames(rs, expColNames);
+        
+        expRS = new String [][]
+        {
+            {"T1", "1", "one"},
+            {"T1", "2", "two"},
+            {"T1", "4", "four"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+
+        // Make sure old way of calling still works until it is 
+        // deprecated.
+        
+        st.executeUpdate("set schema APP");
+        rs = st.executeQuery(
+            "SELECT * FROM NEW org.apache.derby.diag.SpaceTable('T1') as x");
+        
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        
+        expRS = new String [][]
+        {
+            {"T1", "0", "1", "0", "1", "4096", "0"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+        
+        rs = st.executeQuery(
+            "SELECT * FROM NEW "
+            + "org.apache.derby.diag.SpaceTable('APP', 'T1') as x");
+        
+        expColNames = new String [] {
+            "CONGLOMERATENAME", "ISINDEX", "NUMALLOCATEDPAGES", "NUMFREEPAGES",
+            "NUMUNFILLEDPAGES", "PAGESIZE", "ESTIMSPACESAVING"};
+        JDBC.assertColumnNames(rs, expColNames);
+        
+        expRS = new String [][]
+        {
+            {"T1", "0", "1", "0", "1", "4096", "0"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+        
+        // Now do some sanity checking to make sure SPACE_TABLE cannot be
+        // used in any illegal ways.        
+
+        checkIllegalUses(st, "space_table", "SpaceTable",
+            "('APP', 'T1')", "conglomeratename");
+
+        // Clean-up.
+        getConnection().rollback();
+        st.close();
+    }
+
+    /**
+     * Just run a couple of sanity checks to makes sure the table
+     * mapping for org.apache.derby.diag.StatementDuration() works
+     * correctly and fails where it is supposed to.
+     */
+    public void testStatementDuration() throws Exception
+    {
+        Statement st = createStatement();
+        st.executeUpdate("set schema APP");
+
+        // Do a simple check to make sure the VTI mapping works.
+        
+        java.net.URL logURL = SupportFilesSetup.getReadOnlyURL(testLogFile);
+        String vtiArg = "('" + logURL.getFile() + "')";
+        ResultSet rs = st.executeQuery(
+            "select * from "
+            + "TABLE(SYSCS_DIAG.STATEMENT_DURATION" + vtiArg + ") x");
+        
+        String [] expColNames = new String [] {
+            "TS", "THREADID", "XID", "LCCID", "LOGTEXT", "DURATION"};
+
+        JDBC.assertColumnNames(rs, expColNames);
+        String [][] expRS = new String [][]
+        {
+            {"2006-12-15 16:14:58.280", "main,5,main", "1111", "0",
+                "(DATABASE = ugh), (DRDAID = null), Cleanup action starting",
+                "0"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+
+        // Make sure old way of calling still works until it is 
+        // deprecated.
+        
+        rs = st.executeQuery(
+            "SELECT * FROM NEW "
+            + "org.apache.derby.diag.StatementDuration" + vtiArg + " as x");
+        
+        expColNames = new String [] {
+            "TS", "THREADID", "XID", "LCCID", "LOGTEXT", "DURATION"};
+
+        JDBC.assertColumnNames(rs, expColNames);
+        expRS = new String [][]
+        {
+            {"2006-12-15 16:14:58.280", "main,5,main", "1111", "0",
+                "(DATABASE = ugh), (DRDAID = null), Cleanup action starting",
+                "0"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+
+        // And finally, do the usual checks for invalid uses.
+
+        checkIllegalUses(st, "statement_duration", "StatementDuration",
+            vtiArg, "logtext");
+    }
+
+    /**
+     * Just run a couple of sanity checks to makes sure the table
+     * mapping for org.apache.derby.diag.ErrorLogReader() works
+     * correctly and fails where it is supposed to.
+     */
+    public void testErrorLogReader() throws Exception
+    {
+        Statement st = createStatement();
+        st.executeUpdate("set schema APP");
+
+        // Do a simple check to make sure the VTI mapping works.
+        
+        java.net.URL logURL = SupportFilesSetup.getReadOnlyURL(testLogFile);
+        String vtiArg = "('" + logURL.getFile() + "')";
+        ResultSet rs = st.executeQuery(
+            " select * from "
+            + "TABLE(SYSCS_DIAG.ERROR_LOG_READER" + vtiArg + ") x");
+        
+        String [] expColNames = new String [] {
+            "TS", "THREADID", "XID", "LCCID", "DATABASE", "DRDAID", "LOGTEXT"};
+
+        JDBC.assertColumnNames(rs, expColNames);
+        String [][] expRS = new String [][]
+        {
+            {"2006-12-15 16:14:58.280", "main,5,main", "1111", "0", "ugh",
+                "null", "Cleanup action starting"},
+            {"2006-12-15 16:14:58.280", "main,5,main", "1111", "0", "ugh",
+                "null", "Failed Statement is: select * from oops"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+
+        // Make sure old way of calling still works until it is 
+        // deprecated.
+        
+        rs = st.executeQuery(
+            "SELECT * FROM NEW" +
+            " org.apache.derby.diag.ErrorLogReader" + vtiArg + " as x");
+        
+        expColNames = new String [] {
+            "TS", "THREADID", "XID", "LCCID", "DATABASE", "DRDAID", "LOGTEXT"};
+
+        JDBC.assertColumnNames(rs, expColNames);
+        expRS = new String [][]
+        {
+            {"2006-12-15 16:14:58.280", "main,5,main", "1111", "0", "ugh",
+                "null", "Cleanup action starting"},
+            {"2006-12-15 16:14:58.280", "main,5,main", "1111", "0", "ugh",
+                "null", "Failed Statement is: select * from oops"}
+        };
+
+        JDBC.assertFullResultSet(rs, expRS, true);
+
+        // And finally, do the usual checks for invalid uses.
+
+        checkIllegalUses(st, "error_log_reader", "ErrorLogReader",
+            vtiArg, "logtext");
+    }
+
+    /**
+     * Tests to make sure that attempts to use the TABLE constructor
+     * with things other than the VTI diagnostic table functions
+     * do not work (with the exception of SELECT and VALUES queries,
+     * which should work as normal).
+     */
+    public void testInvalidTableFunctions() throws Exception
+    {
+        // Sanity check: make sure SELECT and VALUES clauses still  work.
+        
+        Statement st = createStatement();
+        st.executeUpdate("set schema APP");
+
+        ResultSet rs = st.executeQuery(
+            "select * from table (select * from t1) x");
+        
+        String [] expColNames = new String [] {"I", "C"};
+        JDBC.assertColumnNames(rs, expColNames);
+        
+        String [][] expRS = new String [][]
+        {
+            {"1", "one"},
+            {"2", "two"},
+            {"4", "four"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+        
+        rs = st.executeQuery(
+            " select * from table (values (1, 2), (2, 3)) x");
+        
+        expColNames = new String [] {"1", "2"};
+        JDBC.assertColumnNames(rs, expColNames);
+        
+        expRS = new String [][]
+        {
+            {"1", "2"},
+            {"2", "3"}
+        };
+        
+        JDBC.assertFullResultSet(rs, expRS, true);
+        
+        // Use of TABLE constructor with regular tables should fail.
+
+        assertStatementError("42X01", st, "select * from table (app.t1) x");
+        assertStatementError("42Y03", st, "select * from table (app.t1()) x");
+
+        // Use of TABLE constructor with VTI tables (as opposed to VTI
+        // table *functions*) should fail.
+
+        assertStatementError("42X01", st,
+            "select * from table (syscs_diag.lock_table) x");
+
+        assertStatementError("42Y03", st,
+            "select * from table (syscs_diag.lock_table()) x");
+
+        assertStatementError("42X01", st,
+            "select * from table (syscs_diag.transaction_table) x");
+
+        assertStatementError("42Y03", st,
+            "select * from table (syscs_diag.transaction_table()) x");
+
+        assertStatementError("42X01", st,
+            "select * from table (syscs_diag.statement_cache) x");
+
+        assertStatementError("42Y03", st,
+            "select * from table (syscs_diag.statement_cache()) x");
+
+        assertStatementError("42X01", st,
+            "select * from table (syscs_diag.error_messages) x");
+
+        assertStatementError("42Y03", st,
+            "select * from table (syscs_diag.error_messages()) x");
+
+        // Clean-up.
+        getConnection().rollback();
+        st.close();
+    }
+
+    /* All statements in this method should fail because a VTI table-
+     * mapping that takes arguments can only be used as part of the TABLE 
+     * constructor.  Any other uses of, or attempts to modify, such a
+     * VTI table should throw an appropriate error.
+     */
+    private void checkIllegalUses(Statement st, String vtiTableName,
+        String vtiMethodName, String args, String colName) throws SQLException
+    {
+        assertStatementError("42X05", st,
+            "select * from syscs_diag." + vtiTableName);
+        
+        assertStatementError("42X01", st,
+            "select * from syscs_diag." + vtiTableName + args + " x");
+        
+        assertStatementError("42X01", st,
+            "select * from (syscs_diag." + vtiTableName + args + ") x");
+        
+        assertStatementError("42Y55", st,
+            "drop table syscs_diag." + vtiTableName);
+        
+        assertStatementError("42X01", st,
+            "drop table syscs_diag." + vtiTableName + args);
+        
+        assertStatementError("42X62", st,
+            "drop function syscs_diag." + vtiTableName);
+        
+        assertStatementError("42X01", st,
+            "drop function syscs_diag." + vtiTableName + args);
+        
+        assertStatementError("42X62", st,
+            "alter table syscs_diag." + vtiTableName + " add column bad int");
+        
+        assertStatementError("42X01", st,
+            "alter table syscs_diag." + vtiTableName + args
+            + " add column bad int");
+        
+        assertStatementError("42X05", st,
+            "update syscs_diag." + vtiTableName
+            + " set " + colName + " = NULL");
+        
+        assertStatementError("42X01", st,
+            "update syscs_diag." + vtiTableName + args + " set "
+            + colName + "  = NULL");
+        
+        assertStatementError("42X05", st,
+            "delete from syscs_diag." + vtiTableName + " where 1 = 1");
+        
+        assertStatementError("42X01", st,
+            "delete from syscs_diag." + vtiTableName + args + " where 1 = 1");
+        
+        assertStatementError("42X05", st,
+            "insert into syscs_diag." + vtiTableName + " values('bad')");
+        
+        assertStatementError("42X05", st,
+            "insert into syscs_diag." + vtiTableName + " (" + colName
+            + ") values('bad')");
+        
+        assertStatementError("42X01", st,
+            "insert into syscs_diag." + vtiTableName + args
+            + " values('bad')");
+        
+        CallableStatement cSt = prepareCall(
+            "call SYSCS_UTIL.SYSCS_COMPRESS_TABLE(?, ?, 1)");
+        cSt.setString(1, "SYSCS_DIAG");
+        cSt.setString(2, vtiTableName.toUpperCase());
+        assertStatementError("42X62", cSt);
+        
+        cSt = prepareCall(
+            "call SYSCS_UTIL.SYSCS_INPLACE_COMPRESS_TABLE(?, ?, 1, 1, 1)");
+        cSt.setString(1, "SYSCS_DIAG");
+        cSt.setString(2, vtiTableName.toUpperCase());
+        assertStatementError("42X05", cSt);
+
+        assertStatementError("42X08", st,
+            "update new org.apache.derby.diag." + vtiMethodName + args
+            + " set " + colName + " = NULL");
+        
+        assertStatementError("42X08", st,
+            "delete from new org.apache.derby.diag." + vtiMethodName + args
+            + " where 1 = 0");
+
+        // Simple check to verify same restrictions hold true if current
+        // schema is "SYSCS_DIAG".
+
+        st.execute("set schema syscs_diag");
+        assertStatementError("42X01", st,
+            "select * from " + vtiTableName + args + " x");
+        
+        assertStatementError("42X01", st,
+            "select * from (" + vtiTableName + args + ") x");
+
+        st.execute("set schema app");
+    }
+}

Propchange: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/SysDiagVTIMappingTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java?view=diff&rev=488827&r1=488826&r2=488827
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java Tue Dec 19 14:10:56 2006
@@ -70,6 +70,7 @@
         suite.addTest(SQLAuthorizationPropTest.suite());
         suite.addTest(TimeHandlingTest.suite());
         suite.addTest(VTITest.suite());
+        suite.addTest(SysDiagVTIMappingTest.suite());
         suite.addTest(UpdatableResultSetTest.suite());
 
         // Add the XML tests, which exist as a separate suite

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/build.xml
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/build.xml?view=diff&rev=488827&r1=488826&r2=488827
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/build.xml (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/build.xml Tue Dec 19 14:10:56 2006
@@ -171,7 +171,7 @@
   <target name="copyfiles">
     <copy todir="${out.dir}/${derby.testing.functest.dir}/tests/lang">
       <fileset dir="${derby.testing.src.dir}/${derby.testing.functest.dir}/tests/lang" 
-        includes="*.sql,*.properties,*.subsql,*.policy,*.jar,xmlTestFiles/*"/>  
+        includes="*.sql,*.properties,*.subsql,*.policy,*.jar,*.tstlog,xmlTestFiles/*"/>  
     </copy>
   </target> 
 

Added: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/sys_vti_test_derby.tstlog
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/sys_vti_test_derby.tstlog?view=auto&rev=488827
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/sys_vti_test_derby.tstlog (added)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/sys_vti_test_derby.tstlog Tue Dec 19 14:10:56 2006
@@ -0,0 +1,35 @@
+ ----------------------------------------------------------------
+ 2006-12-15 16:14:53.694 GMT:
+ Booting Derby version The Apache Software Foundation - Apache Derby - 10.3.0.0 alpha - (486911M): instance c013800d-010f-86e4-eedb-0000001755b0
+on database directory C:\private\test\other\derby-2152\ugh  
+
+Database Class Loader started - derby.database.classpath=''
+2006-12-15 16:14:58.280 GMT Thread[main,5,main] (XID = 1111), (SESSIONID = 0), (DATABASE = ugh), (DRDAID = null), Cleanup action starting
+2006-12-15 16:14:58.280 GMT Thread[main,5,main] (XID = 1111), (SESSIONID = 0), (DATABASE = ugh), (DRDAID = null), Failed Statement is: select * from oops
+ERROR 42X05: Table/View 'OOPS' does not exist.
+	at org.apache.derby.iapi.error.StandardException.newException(StandardException.java:305)
+	at org.apache.derby.impl.sql.compile.FromBaseTable.bindTableDescriptor(FromBaseTable.java:2385)
+	at org.apache.derby.impl.sql.compile.FromBaseTable.bindNonVTITables(FromBaseTable.java:2108)
+	at org.apache.derby.impl.sql.compile.FromList.bindTables(FromList.java:301)
+	at org.apache.derby.impl.sql.compile.SelectNode.bindNonVTITables(SelectNode.java:463)
+	at org.apache.derby.impl.sql.compile.DMLStatementNode.bindTables(DMLStatementNode.java:200)
+	at org.apache.derby.impl.sql.compile.DMLStatementNode.bind(DMLStatementNode.java:138)
+	at org.apache.derby.impl.sql.compile.CursorNode.bindStatement(CursorNode.java:237)
+	at org.apache.derby.impl.sql.GenericStatement.prepMinion(GenericStatement.java:314)
+	at org.apache.derby.impl.sql.GenericStatement.prepare(GenericStatement.java:88)
+	at org.apache.derby.impl.sql.conn.GenericLanguageConnectionContext.prepareInternalStatement(GenericLanguageConnectionContext.java:741)
+	at org.apache.derby.impl.jdbc.EmbedStatement.execute(EmbedStatement.java:568)
+	at org.apache.derby.impl.jdbc.EmbedStatement.execute(EmbedStatement.java:517)
+	at org.apache.derby.impl.tools.ij.ij.executeImmediate(ij.java:321)
+	at org.apache.derby.impl.tools.ij.utilMain.doCatch(utilMain.java:517)
+	at org.apache.derby.impl.tools.ij.utilMain.runScriptGuts(utilMain.java:373)
+	at org.apache.derby.impl.tools.ij.utilMain.go(utilMain.java:268)
+	at org.apache.derby.impl.tools.ij.Main.go(Main.java:204)
+	at org.apache.derby.impl.tools.ij.Main.mainCore(Main.java:170)
+	at org.apache.derby.impl.tools.ij.Main14.main(Main14.java:56)
+	at org.apache.derby.tools.ij.main(ij.java:71)
+Cleanup action completed
+
+2006-12-15 16:15:04.199 GMT:
+Shutting down instance c013800d-010f-86e4-eedb-0000001755b0
+----------------------------------------------------------------

Propchange: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/sys_vti_test_derby.tstlog
------------------------------------------------------------------------------
    svn:eol-style = native