You are viewing a plain text version of this content. The canonical link for it is here.
Posted to derby-commits@db.apache.org by da...@apache.org on 2009/10/23 20:05:18 UTC

svn commit: r829155 [2/2] - in /db/derby/code/trunk: java/engine/org/apache/derby/iapi/sql/compile/ java/engine/org/apache/derby/iapi/sql/execute/ java/engine/org/apache/derby/impl/sql/compile/ java/engine/org/apache/derby/impl/sql/execute/ java/engine...

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=829155&r1=829154&r2=829155&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 Fri Oct 23 18:05:03 2009
@@ -778,6 +778,7 @@
 					  					whereClause, /* WHERE clause */
 										null, /* GROUP BY list */
 										null, /* having clause */
+										null, /* window list */
 										getContextManager());
 
 		StatementNode retval =
@@ -818,6 +819,7 @@
 					  					whereClause, /* WHERE clause */
 										null, /* GROUP BY list */
 										null, /* having clause */
+										null, /* window list */
 										getContextManager());
 
 		StatementNode retval =
@@ -1089,6 +1091,37 @@
 		return retval;
 	}
 
+
+	/**
+	 * Determine whether the next sequence of tokens can be the beginning of a
+	 * window or aggregate function.
+	 * @return	TRUE iff the next set of tokens is the beginning of a
+	 *			window or aggregate function
+	 */
+	private boolean windowOrAggregateFunctionFollows()
+	{
+		boolean retval = false;
+
+		switch (getToken(1).kind) {
+
+		case ROWNUMBER:
+
+		// case RANK:
+		// case DENSE_RANK:
+		// case PERCENT_RANK:
+		// case CUME_DIST:
+
+			retval = true;
+			break;
+
+		default:
+			retval = aggregateFollows();
+			break;
+		}
+
+		return retval;
+	}
+
 	/**
 	 * Determine whether the next sequence of tokens can be the beginning
 	 * of a miscBuiltins().
@@ -2152,6 +2185,7 @@
 |	<VIEW: "view">
 |	<WHENEVER: "whenever">
 |	<WHERE: "where">
+|   <WINDOW: "window">
 |	<WITH: "with">
 |	<WORK: "work">
 |	<WRITE: "write">
@@ -2282,7 +2316,9 @@
 |	<LONG: "long">
 |	<LTRIM: "ltrim">
 |	<NONE: "none">
+|	<OVER: "over">
 |	<ROLE: "role">
+|	<ROWNUMBER: "row_number">
 |	<RTRIM: "rtrim">
 |	<SUBSTR:	"substr">
 |	<XML:	"xml">
@@ -6524,10 +6560,12 @@
 		return value;
 	}
 |
-	// This LOOKAHEAD is required to distinguish an aggregateNode from
+        // This LOOKAHEAD is required to distinguish an
+        // window/aggregate function from
 	// miscBuiltins(). Both can start with an identifier.
-	LOOKAHEAD( { aggregateFollows() } )
-	value = aggregateNode()
+
+	LOOKAHEAD( { windowOrAggregateFunctionFollows() } )
+	value = windowOrAggregateFunctionNode()
 	{
 		return value;
 	}
@@ -8681,12 +8719,14 @@
 	GroupByList	groupByList = null;
 	ValueNode	havingClause = null;
 	Token		whereToken;
+	WindowList  windows = null;
 }
 {
 	fromList = fromClause()
 	[ whereToken = <WHERE> whereClause = whereClause(whereToken) ]
 	[ groupByList = groupByClause() ]
 	[ havingClause = havingClause() ]
+	[ windows = windowClause() ]
 	{
  
 		// fix for HAVING without GROUP BY, makes sure we get one
@@ -8711,6 +8751,7 @@
 							whereClause,
 							groupByList,
 							havingClause,
+							windows,
 							getContextManager());
 
 		return selectNode;
@@ -9393,15 +9434,16 @@
 ** aggregate nodes.
 */
 ValueNode
-aggregateNode() throws StandardException :
+windowOrAggregateFunctionNode() throws StandardException :
 {
-	ValueNode agg;
+	ValueNode winOrAgg;
+	QueryTreeNode window = null;
 }
 {
 	<COUNT> <LEFT_PAREN>
 	( <ASTERISK>
 	{
-			agg = (ValueNode) nodeFactory.getNode(
+			winOrAgg = (ValueNode) nodeFactory.getNode(
 								C_NodeTypes.AGGREGATE_NODE,
 								null,
 								CountAggregateDefinition.class, 
@@ -9410,20 +9452,76 @@
 								getContextManager());
 	}
 	|
-		agg = aggregateExpression("COUNT", CountAggregateDefinition.class)
+		winOrAgg = aggregateExpression("COUNT", CountAggregateDefinition.class)
 	)
-	<RIGHT_PAREN>
+		<RIGHT_PAREN> [ window = overClause() ]
+	{
+		if (window != null) {
+			winOrAgg =  (ValueNode) nodeFactory.getNode(
+				C_NodeTypes.AGGREGATE_WINDOW_FUNCTION_NODE,
+				window,
+				winOrAgg,
+				getContextManager());
+		}
+
+		return winOrAgg;
+	}
+|
+	winOrAgg = generalAggregate() [ window = overClause() ]
 	{
-		return agg;
+		if (window != null) {
+			winOrAgg =  (ValueNode) nodeFactory.getNode(
+				C_NodeTypes.AGGREGATE_WINDOW_FUNCTION_NODE,
+				window,
+				winOrAgg,
+				getContextManager());
+		}
+
+		return winOrAgg;
 	}
 |
-	agg = generalAggregate()
+	<ROWNUMBER> <LEFT_PAREN> <RIGHT_PAREN> window = overClause()
 	{
-		return agg;
+		winOrAgg = (ValueNode) nodeFactory.getNode(
+			C_NodeTypes.ROW_NUMBER_FUNCTION_NODE,
+			null,
+			window,
+			getContextManager());
+		return winOrAgg;
 	}
 }
 
 
+QueryTreeNode
+overClause() throws StandardException :
+{
+	String windowRef;
+	OrderByList orderCols = null;
+}
+{
+  LOOKAHEAD({ getToken(2).kind == LEFT_PAREN ||
+			  getToken(2).kind == IDENTIFIER })
+	<OVER>
+  ( <LEFT_PAREN> [ orderCols = orderByClause() ] <RIGHT_PAREN>
+	{
+		return (QueryTreeNode) nodeFactory.getNode(
+			C_NodeTypes.WINDOW_DEFINITION_NODE,
+			null,
+			orderCols,
+			getContextManager());
+	}
+ |
+   windowRef = identifier(Limits.MAX_IDENTIFIER_LENGTH, true)
+   {
+	   return (QueryTreeNode) nodeFactory.getNode(
+			C_NodeTypes.WINDOW_REFERENCE_NODE,
+			windowRef,
+			getContextManager());
+   }
+ )
+
+}
+
 
 ValueNode
 aggregateExpression(String aggName, Class aggClass) throws StandardException :
@@ -9677,6 +9775,44 @@
 	}
 }
 
+
+WindowList
+windowClause() throws StandardException :
+{
+	WindowList windows = new WindowList();
+	windows.setContextManager(getContextManager());
+}
+{
+	<WINDOW> windows = windowDefinition(windows)
+	( <COMMA> windows = windowDefinition(windows) )*
+	{
+		return windows;
+	}
+}
+
+
+WindowList
+windowDefinition(WindowList wl) throws StandardException :
+{
+	String windowName;
+	OrderByList orderCols = null;
+}
+{
+	windowName = identifier(Limits.MAX_IDENTIFIER_LENGTH, true)
+	<AS>
+	<LEFT_PAREN> [ orderCols = orderByClause() ] <RIGHT_PAREN>
+	{
+		wl.addWindow(
+			(WindowDefinitionNode)nodeFactory.getNode(
+				C_NodeTypes.WINDOW_DEFINITION_NODE,
+				windowName,
+				orderCols,
+				getContextManager()));
+
+		return wl;
+	}
+}
+
 StatementNode
 schemaDefinition() throws StandardException :
 {
@@ -13895,6 +14031,7 @@
 	|	tok = <OLD>
 	|	tok = <OLD_TABLE>
 	|	tok = <OJ>
+	|	tok = <OVER>
 	|	tok = <PASCAL>
 	|	tok = <PASSING>
 	|	tok = <PLI>
@@ -13918,6 +14055,7 @@
  	|	tok = <ROLLUP>
 	|	tok = <ROW>
 //	|	tok = <ROW_COUNT>
+	|	tok = <ROWNUMBER>
 	|   tok = <RR>
 	|   tok = <RS>
 	|	tok = <SCALE>

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java?rev=829155&r1=829154&r2=829155&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java Fri Oct 23 18:05:03 2009
@@ -829,6 +829,33 @@
 	}
 
 	/**
+		@see ResultSetFactory#getWindowResultSet
+		@exception StandardException	Thrown on error
+	 */
+	public NoPutResultSet getWindowResultSet(
+								Activation activation,
+								NoPutResultSet source,
+								GeneratedMethod rowAllocator,
+								int resultSetNumber,
+								int erdNumber,
+								GeneratedMethod restriction,
+								double optimizerEstimatedRowCount,
+								double optimizerEstimatedCost)
+		throws StandardException
+	{
+		return new WindowResultSet(
+								activation,
+								source,
+								rowAllocator,
+								resultSetNumber,
+								erdNumber,
+								restriction,
+								optimizerEstimatedRowCount,
+								optimizerEstimatedCost);
+	}
+
+
+	/**
 		@see ResultSetFactory#getNestedLoopJoinResultSet
 		@exception StandardException thrown on error
 	 */

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RealResultSetStatisticsFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RealResultSetStatisticsFactory.java?rev=829155&r1=829154&r2=829155&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RealResultSetStatisticsFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RealResultSetStatisticsFactory.java Fri Oct 23 18:05:03 2009
@@ -70,6 +70,7 @@
 import org.apache.derby.impl.sql.execute.UpdateResultSet;
 import org.apache.derby.impl.sql.execute.VTIResultSet;
 import org.apache.derby.impl.sql.execute.DependentResultSet;
+import org.apache.derby.impl.sql.execute.WindowResultSet;
 
 import org.apache.derby.iapi.sql.execute.RunTimeStatistics;
 import org.apache.derby.impl.sql.execute.rts.RealAnyResultSetStatistics;
@@ -107,6 +108,7 @@
 import org.apache.derby.impl.sql.execute.rts.RealRowCountStatistics;
 import org.apache.derby.impl.sql.execute.rts.ResultSetStatistics;
 import org.apache.derby.impl.sql.execute.rts.RunTimeStatisticsImpl;
+import org.apache.derby.impl.sql.execute.rts.RealWindowResultSetStatistics;
 
 import org.apache.derby.iapi.reference.SQLState;
 
@@ -836,6 +838,24 @@
 											rrs.optimizerEstimatedRowCount,
 											rrs.optimizerEstimatedCost);
 		}
+		else if (rs instanceof WindowResultSet)
+		{
+			WindowResultSet wrs = (WindowResultSet) rs;
+
+			return new RealWindowResultSetStatistics(
+											wrs.numOpens,
+											wrs.rowsSeen,
+											wrs.rowsFiltered,
+											wrs.constructorTime,
+											wrs.openTime,
+											wrs.nextTime,
+											wrs.closeTime,
+											wrs.resultSetNumber,
+											wrs.optimizerEstimatedRowCount,
+											wrs.optimizerEstimatedCost,
+											getResultSetStatistics(wrs.source)
+											);
+		}
 		else if (rs instanceof SetOpResultSet)
 		{
 			SetOpResultSet srs = (SetOpResultSet) rs;

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/WindowResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/WindowResultSet.java?rev=829155&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/WindowResultSet.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/WindowResultSet.java Fri Oct 23 18:05:03 2009
@@ -0,0 +1,328 @@
+/*
+
+   Derby - Class org.apache.derby.impl.sql.execute.WindowResultSet
+
+   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.derby.impl.sql.execute;
+
+import org.apache.derby.iapi.services.sanity.SanityManager;
+import org.apache.derby.iapi.services.loader.GeneratedMethod;
+
+import org.apache.derby.iapi.sql.Activation;
+import org.apache.derby.iapi.sql.execute.ExecRow;
+import org.apache.derby.iapi.sql.execute.NoPutResultSet;
+
+import org.apache.derby.iapi.types.DataValueDescriptor;
+
+import org.apache.derby.iapi.reference.SQLState;
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.services.io.FormatableBitSet;
+
+/**
+ * WindowResultSet
+ *
+ * This ResultSet handles a window function ResultSet.
+ *
+ * The ResultSet is opened using openCore().  Each row is fetched and any
+ * restrictions evaluated for each row in a do-while loop in getNextRowCore().
+ * The ResultSet is closed using closeCore().
+ *
+ */
+class WindowResultSet extends NoPutResultSetImpl
+{
+    private GeneratedMethod restriction = null;
+    private GeneratedMethod row;
+
+
+    /**
+     * Source result set,
+     */
+    public NoPutResultSet source = null;
+
+
+    /**
+     * Cumulative time needed to evalute any restriction on this result set.
+     */
+    public long restrictionTime;
+
+    private FormatableBitSet referencedColumns;
+    private ExecRow allocatedRow;
+    private long rownumber;
+
+    /**
+     *  Constructor
+     *
+     *  @param  activation          The activation
+     *  @param  source              Source result set
+     *  @param  rowAllocator
+     *  @param  resultSetNumber     The resultSetNumber
+     *  @param  erdNumber           Int for ResultDescription
+	                                (so it can be turned back into an object)
+     *  @param  restriction         Restriction
+     *  @param  optimizerEstimatedRowCount  The optimizer's estimated number
+     *                                      of rows.
+     *  @param  optimizerEstimatedCost      The optimizer's estimated cost
+     */
+
+    WindowResultSet(Activation activation,
+        NoPutResultSet         source,
+        GeneratedMethod        rowAllocator,
+        int                    resultSetNumber,
+        int                    erdNumber,
+        GeneratedMethod        restriction,
+        double                 optimizerEstimatedRowCount,
+        double                 optimizerEstimatedCost)
+    {
+
+        super(activation,
+            resultSetNumber,
+            optimizerEstimatedRowCount,
+            optimizerEstimatedCost);
+
+        if (SanityManager.DEBUG) {
+            SanityManager.ASSERT(activation != null,
+                                 "activation expected to be non-null");
+            SanityManager.ASSERT(resultSetNumber >= 0,
+                                 "resultSetNumber expected to be >= 0");
+        }
+
+        this.restriction = restriction;
+        this.source = source;
+        this.row = rowAllocator;
+        this.allocatedRow = null;
+        this.rownumber = 0;
+
+        if (erdNumber != -1) {
+            this.referencedColumns =
+                (FormatableBitSet)(activation.getPreparedStatement().
+                                   getSavedObject(erdNumber));
+        }
+
+        recordConstructorTime();
+    }
+
+
+    /**
+     * Open this ResultSet.
+     *
+     * @exception StandardException thrown if cursor finished.
+     */
+    public void openCore() throws StandardException {
+        if (SanityManager.DEBUG) {
+            SanityManager.ASSERT(!isOpen,
+                "WindowResultSet already open");
+        }
+
+        beginTime = getCurrentTimeMillis();
+
+        /* Call into the source openCore() */
+        source.openCore();
+
+        isOpen = true;
+        rownumber = 0;
+        numOpens++;
+        openTime += getElapsedMillis(beginTime);
+    }
+
+    /**
+     * Reopen this ResultSet.
+     *
+     * @exception StandardException thrown if cursor finished.
+     */
+    public void reopenCore() throws StandardException {
+        if (SanityManager.DEBUG) {
+            SanityManager.ASSERT(isOpen, "WindowResultSet already open");
+        }
+
+        beginTime = getCurrentTimeMillis();
+
+        /* Reopen the source */
+        source.reopenCore();
+
+        rownumber = 0;
+        numOpens++;
+        openTime += getElapsedMillis(beginTime);
+    }
+
+    /**
+     * Return the requested values computed from the next row (if any) for which
+     * the restriction evaluates to true.
+     * <p>
+     * Restriction and projection parameters are evaluated for each row.
+     *
+     * @exception StandardException thrown on failure.
+     * @exception StandardException ResultSetNotOpen thrown if not yet open.
+     *
+     * @return the next row in the result
+     */
+    public ExecRow getNextRowCore() throws StandardException
+    {
+        ExecRow sourceRow = null;
+        ExecRow retval = null;
+        boolean restrict = false;
+        DataValueDescriptor restrictBoolean;
+        long beginRT = 0;
+
+        beginTime = getCurrentTimeMillis();
+
+        if (!isOpen) {
+            throw StandardException.newException(
+                SQLState.LANG_RESULT_SET_NOT_OPEN, "next");
+        }
+
+        /*
+         * Loop until we get a row from the source that qualifies, or there are
+         * no more rows to qualify. For each iteration fetch a row from the
+         * source, and evaluate against the restriction if any.
+         */
+        ExecRow tmpRow = null;
+
+        do {
+            sourceRow = source.getNextRowCore();
+
+            if (sourceRow != null) {
+                this.rownumber++;
+                tmpRow = getAllocatedRow();
+                populateFromSourceRow(sourceRow, tmpRow);
+                setCurrentRow(tmpRow);
+
+                /* Evaluate any restrictions */
+                restrictBoolean = (DataValueDescriptor) ((restriction == null) ?
+                                    null : restriction.invoke(activation));
+
+                restrictionTime += getElapsedMillis(beginRT);
+
+                // if the result is null, we make it false --
+                // so the row won't be returned.
+                restrict = (restrictBoolean == null) ||
+                    ((!restrictBoolean.isNull()) &&
+                    restrictBoolean.getBoolean());
+
+                if (!restrict) {
+                    rowsFiltered++;
+                    clearCurrentRow();
+                }
+
+                /* Update the run time statistics */
+                rowsSeen++;
+                retval = currentRow;
+            } else {
+                clearCurrentRow();
+                retval = null;
+            }
+        } while ((sourceRow != null) && (!restrict));
+
+        nextTime += getElapsedMillis(beginTime);
+        return retval;
+    }
+
+    /**
+     * If the result set has been opened, close the open scan, else throw.
+     *
+     * @exception StandardException thrown on error
+     */
+    public void close() throws StandardException {
+        beginTime = getCurrentTimeMillis();
+
+        if (isOpen) {
+            clearCurrentRow();
+
+            /*
+             * Make sure to close the source
+             */
+            source.close();
+            super.close();
+
+        } else if (SanityManager.DEBUG) {
+            SanityManager.DEBUG("CloseRepeatInfo",
+                                "Close of WindowResultSet repeated");
+        }
+
+        closeTime += getElapsedMillis(beginTime);
+    }
+
+
+
+    /**
+     * Copy columns from srcrow into destrow, or insert ROW_NUMBER.
+     * <p/>
+     * <b>FIXME</b>
+     * This is temporary. Window function treatment needs to generalized to
+     * work for other window functions.
+     *
+     * @exception StandardException thrown on failure to open
+     */
+    public void populateFromSourceRow(ExecRow srcrow, ExecRow destrow)
+        throws StandardException
+    {
+        int srcindex = 1;
+
+        try {
+            DataValueDescriptor[] columns = destrow.getRowArray();
+            for (int index = 0; index < columns.length; index++) {
+
+                if (referencedColumns != null &&
+                        !referencedColumns.get(index)) {
+                    columns[index].setValue((long)this.rownumber);
+                } else {
+                    destrow.setColumn(index+1, srcrow.getColumn(srcindex));
+                    srcindex++;
+                }
+            }
+        } catch (StandardException se) {
+            throw se;
+        } catch (Throwable t) {
+            throw StandardException.unexpectedUserException(t);
+        }
+    }
+
+    /**
+     * Return the total amount of time spent in this ResultSet
+     *
+     * @param type  CURRENT_RESULTSET_ONLY - time spent only in this ResultSet
+     *          ENTIRE_RESULTSET_TREE  - time spent in this ResultSet and below.
+     *
+     * @return long     The total amount of time spent (in milliseconds).
+     */
+    public long getTimeSpent(int type) {
+        long totTime = constructorTime + openTime + nextTime + closeTime;
+
+        if (type == NoPutResultSet.CURRENT_RESULTSET_ONLY) {
+            return totTime - source.getTimeSpent(ENTIRE_RESULTSET_TREE);
+        } else {
+            return totTime;
+        }
+    }
+
+    /**
+     * Cache the ExecRow for this result set.
+     *
+     * @return The cached ExecRow for this ResultSet
+     *
+     * @exception StandardException thrown on failure.
+     */
+    private ExecRow getAllocatedRow()
+        throws StandardException {
+
+        if (allocatedRow == null) {
+            allocatedRow = (ExecRow) row.invoke(activation);
+        }
+
+        return allocatedRow;
+    }
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/WindowResultSet.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/rts/RealWindowResultSetStatistics.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/rts/RealWindowResultSetStatistics.java?rev=829155&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/rts/RealWindowResultSetStatistics.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/rts/RealWindowResultSetStatistics.java Fri Oct 23 18:05:03 2009
@@ -0,0 +1,180 @@
+/*
+   Derby - Class org.apache.derby.impl.sql.execute.rts.RealWindowResultSetStatistics
+
+   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.derby.impl.sql.execute.rts;
+
+import org.apache.derby.iapi.services.i18n.MessageService;
+import org.apache.derby.impl.sql.execute.xplain.XPLAINUtil;
+import org.apache.derby.iapi.sql.execute.xplain.XPLAINVisitor;
+import org.apache.derby.iapi.reference.SQLState;
+
+/**
+ * ResultSetStatistics implementation for WindowResultSet.
+ */
+public class RealWindowResultSetStatistics
+    extends RealNoPutResultSetStatistics
+{
+
+    private ResultSetStatistics childResultSetStatistics;
+
+    /**
+     * Constructor.
+     *
+     */
+    public  RealWindowResultSetStatistics(
+                                int numOpens,
+                                int rowsSeen,
+                                int rowsFiltered,
+                                long constructorTime,
+                                long openTime,
+                                long nextTime,
+                                long closeTime,
+                                int resultSetNumber,
+                                double optimizerEstimatedRowCount,
+                                double optimizerEstimatedCost,
+                                ResultSetStatistics childResultSetStatistics
+                                )
+    {
+        super(
+            numOpens,
+            rowsSeen,
+            rowsFiltered,
+            constructorTime,
+            openTime,
+            nextTime,
+            closeTime,
+            resultSetNumber,
+            optimizerEstimatedRowCount,
+            optimizerEstimatedCost
+            );
+        this.childResultSetStatistics = childResultSetStatistics;
+
+    }
+
+    // ResultSetStatistics interface
+
+    /**
+     * Return the statement execution plan as a String.
+     *
+     * @param depth Indentation level.
+     *
+     * @return String   The statement execution plan as a String.
+     */
+    public String getStatementExecutionPlanText(int depth)
+    {
+        initFormatInfo(depth);
+        String WINDOWSPECIFICATION = "()";
+
+        return
+            indent + MessageService.getTextMessage(
+                            SQLState.RTS_WINDOW_RS) +
+            WINDOWSPECIFICATION + "\n" +
+            indent + MessageService.getTextMessage(
+                            SQLState.RTS_NUM_OPENS) +
+                            " = " + numOpens + "\n" +
+            indent + MessageService.getTextMessage(
+                            SQLState.RTS_ROWS_SEEN) +
+                            " = " + rowsSeen + "\n" +
+            dumpTimeStats(indent, subIndent) + "\n" +
+            dumpEstimatedCosts(subIndent) + "\n" +
+            indent + MessageService.getTextMessage(
+                SQLState.RTS_SOURCE_RS) + ":\n" +
+            childResultSetStatistics.
+                getStatementExecutionPlanText(sourceDepth) + "\n";
+    }
+
+    /**
+     * Return information on the scan nodes from the statement execution
+     * plan as a String.
+     *
+     * @param depth Indentation level.
+     * @param tableName if not NULL then print information for this table only
+     *
+     * @return String   The information on the scan nodes from the
+     *                  statement execution plan as a String.
+     */
+    public String getScanStatisticsText(String tableName, int depth)
+    {
+        return getStatementExecutionPlanText(depth);
+    }
+
+
+    // java.lang.Object override
+    //
+    public String toString()
+    {
+        return getStatementExecutionPlanText(0);
+    }
+
+
+    /**
+     * RealBasicNoPutResultSetStatistics override.
+     * @see RealBasicNoPutResultSetStatistics#getChildren
+     */
+    public java.util.Vector getChildren()
+    {
+        java.util.Vector children = new java.util.Vector();
+        children.addElement(childResultSetStatistics);
+        return children;
+    }
+
+
+    /**
+     * RealBasicNoPutResultSetStatistics override.
+     * @see RealBasicNoPutResultSetStatistics#getNodeOn
+     */
+    public String getNodeOn(){
+        return MessageService.getTextMessage(
+                                    SQLState.RTS_FOR_TAB_NAME,
+                                    "<WINDOW FUNCTION>");
+    }
+
+
+    /**
+     * RealBasicNoPutResultSetStatistics override.
+     * @see RealBasicNoPutResultSetStatistics#getNodeName
+     */
+    public String getNodeName(){
+        return MessageService.getTextMessage(SQLState.RTS_IRTBR);
+    }
+
+
+
+    // -----------------------------------------------------
+    // XPLAINable Implementation
+    // -----------------------------------------------------
+
+    public void accept(XPLAINVisitor visitor) {
+
+        // I have only one child
+        visitor.setNumberOfChildren(1);
+
+        // pre-order, depth-first traversal me first
+        visitor.visit(this);
+
+        // then my child
+        childResultSetStatistics.accept(visitor);
+    }
+
+
+    public String getRSXplainType() {
+        return XPLAINUtil.OP_WINDOW;
+    }
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/rts/RealWindowResultSetStatistics.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml?rev=829155&r1=829154&r2=829155&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml Fri Oct 23 18:05:03 2009
@@ -2784,6 +2784,23 @@
                 <text>A scalar value is expected, not a row set returned by a table function.</text>
             </msg>
 
+            <msg>
+                <name>42ZC0</name>
+                <text>Window '{0}' is not defined.</text>
+                <arg>windowName</arg>
+            </msg>
+
+            <msg>
+                <name>42ZC1</name>
+                <text>Only one window is supported.</text>
+            </msg>
+
+            <msg>
+                <name>42ZC2</name>
+                <text>Window function is illegal in this context: '{0}' clause</text>
+                <arg>clauseName</arg>
+            </msg>
+
         </family>
 
 
@@ -6735,6 +6752,11 @@
             </msg>
 
             <msg>
+                <name>43X9C.U</name>
+                <text>Window ResultSet: </text>
+            </msg>
+
+            <msg>
                 <name>43Y00.U</name>
                 <text>Scalar Aggregate ResultSet</text>
             </msg>

Modified: db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java?rev=829155&r1=829154&r2=829155&view=diff
==============================================================================
--- db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java (original)
+++ db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java Fri Oct 23 18:05:03 2009
@@ -1096,6 +1096,10 @@
     String LANG_NO_COSTING_CONSTRUCTOR                              = "42ZB5";
     String LANG_TABLE_FUNCTION_NOT_ALLOWED                   = "42ZB6";
 
+	String LANG_NO_SUCH_WINDOW                                         = "42ZC0";
+	String LANG_WINDOW_LIMIT_EXCEEDED                                  = "42ZC1";
+	String LANG_WINDOW_FUNCTION_CONTEXT_ERROR                          = "42ZC2";
+
 	//following 3 matches the DB2 sql states
 	String LANG_DECLARED_GLOBAL_TEMP_TABLE_ONLY_IN_SESSION_SCHEMA = "428EK";
 	String LANG_NOT_ALLOWED_FOR_DECLARED_GLOBAL_TEMP_TABLE = "42995";
@@ -1213,6 +1217,7 @@
 	String RTS_ROW_RS												   = "43X99.U";
 	String RTS_RC                                                      = "43X9A.U";
 	String RTS_RC_RS                                                   = "43X9B.U";
+	String RTS_WINDOW_RS                                               = "43X9C.U";
 
 	String RTS_SCALAR_AGG_RS										   = "43Y00.U";
 	String RTS_INDEX_KEY_OPT										   = "43Y01.U";

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/OLAPTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/OLAPTest.java?rev=829155&r1=829154&r2=829155&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/OLAPTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/OLAPTest.java Fri Oct 23 18:05:03 2009
@@ -25,12 +25,16 @@
 import java.sql.SQLException;
 import java.sql.PreparedStatement;
 import java.sql.Statement;
+import java.sql.ResultSetMetaData;
 
 import junit.framework.Test;
+import junit.framework.TestSuite;
+import junit.framework.Assert;
 
 import org.apache.derbyTesting.junit.BaseJDBCTestCase;
 import org.apache.derbyTesting.junit.JDBC;
 import org.apache.derbyTesting.junit.TestConfiguration;
+import org.apache.derbyTesting.junit.CleanDatabaseTestSetup;
 
 /**
  * OLAP functionality test.
@@ -39,10 +43,56 @@
  */ 
 public class OLAPTest extends BaseJDBCTestCase {
 
+	private final static String LANG_WINDOW_FUNCTION_CONTEXT_ERROR = "42ZC2";
+	private final static String NOT_IMPLEMENTED = "0A000";
+	private final static String LANG_SYNTAX_ERROR = "42X01";
+	private final static String LANG_COLUMN_NOT_FOUND =	"42X04";
+
 	public OLAPTest(String name) {
 		super(name);    
 	}
 
+	public static Test makeSuite() {
+        Test clean = new CleanDatabaseTestSetup(
+            new TestSuite(OLAPTest.class)) {
+                protected void decorateSQL(Statement s)
+                        throws SQLException
+                {
+                    getConnection().setAutoCommit(false);
+                    s.executeUpdate("create table t1 (a int, b int)");
+                    s.executeUpdate("create table t2 (x int)");
+                    s.executeUpdate("create table t3 (y int)");
+                    s.executeUpdate("create table t4 (a int, b int)");
+                    s.executeUpdate("create table t5 (a int, b int)");
+
+                    s.executeUpdate(
+                        "insert into t1 values (10,100),(20,200)," +
+                        "                      (30,300),(40,400)," +
+                        "                      (50,500)");
+                    s.executeUpdate(
+                        "insert into t2 values (1),(2),(3),(4),(5)");
+                    s.executeUpdate(
+                        "insert into t3 values (4),(5),(6),(7),(8)");
+                    s.executeUpdate(
+                        "insert into t4 values (10,100),(20,200)");
+                    s.executeUpdate(
+                        "insert into t5 values (1,1),(2,4),(3,4),(4,4),(5,9)");
+                    getConnection().commit();
+                }
+            };
+		return clean;
+	}
+
+
+	public static Test suite()
+    {
+		TestSuite suite = new TestSuite("OLAPTest");
+		suite.addTest(makeSuite());
+		suite.addTest(TestConfiguration.clientServerDecorator(makeSuite()));
+		return suite;
+    }
+
+
 	/**
 	 * Main test body
 	 * 
@@ -52,19 +102,13 @@
 		throws SQLException {
 		Statement s = createStatement();
 
-		s.executeUpdate("create table t1 (a int, b int)");
-		s.executeUpdate("create table t2 (x int)");
-		s.executeUpdate("create table t3 (y int)");
-		s.executeUpdate("create table t4 (a int, b int)");
-
-		s.executeUpdate("insert into t1 values (10,100),(20,200),(30,300),(40,400),(50,500)");
-		s.executeUpdate("insert into t2 values (1),(2),(3),(4),(5)");
-		s.executeUpdate("insert into t3 values (4),(5),(6),(7),(8)");		
-		s.executeUpdate("insert into t4 values (10,100),(20,200)");
-
 		/*
 		 * Positive testing of Statements
-		 *
+		 */
+
+
+
+		/*
 		 * Simple queries
 		 */		
 		ResultSet rs = s.executeQuery("select row_number() over (), t1.* from t1");
@@ -98,25 +142,45 @@
 		expectedRows = new String[][]{{"1", "4"}, {"2", "5"}};
 		JDBC.assertFullResultSet(rs, expectedRows);
 
+		// DERBY-4069: ORDER BY should be applied at the cursor level, that is
+		// *after* a windowing clause in the. So, with the original ordering
+		// here, the ROW_NUMBER should come backwards:
+
 		/* Ordering */
 		rs = s.executeQuery("select row_number() over () as r, t1.* from t1 order by b desc");
-		expectedRows = new String[][]{{"1", "50", "500"}, {"2", "40", "400"}, {"3", "30", "300"}, {"4", "20", "200"}, {"5", "10", "100"}};
+
+		expectedRows = new String[][]{{"5", "50", "500"},
+									  {"4", "40", "400"},
+									  {"3", "30", "300"},
+									  {"2", "20", "200"},
+									  {"1", "10", "100"}};
 		JDBC.assertFullResultSet(rs, expectedRows);
 
+        // DERBY-4069: ORDER BY should be applied at the cursor level, that is
+        // *after* a windowing clause in the. So, with the original ordering
+        // here, the ROW_NUMBER should come backwards:
+
 		/* Ordering on a column dropped in projection */
 		rs = s.executeQuery("select row_number() over () as r, t1.a from t1 order by b desc");
-		expectedRows = new String[][]{{"1", "50"}, {"2", "40"}, {"3", "30"}, {"4", "20"}, {"5", "10"}};
+
+		expectedRows = new String[][]{{"5", "50"},
+									  {"4", "40"},
+									  {"3", "30"},
+									  {"2", "20"},
+									  {"1", "10"}};
 		JDBC.assertFullResultSet(rs, expectedRows);
 
 		/* Only expressions in RCL */
 		rs = s.executeQuery("select row_number() over (), row_number() over (), 2*t1.a from t1");
 		expectedRows = new String[][]{{"1", "1", "20"}, {"2", "2","40"}, {"3", "3","60"}, {"4", "4", "80"}, {"5", "5", "100"}};
 		JDBC.assertFullResultSet(rs, expectedRows);
-		
+
+
+
 		/*
-		 * Subquerys 
-		 */ 
-			
+		 * Subqueries
+		 */
+
 		/* This query returned no rows at one time */
 		rs = s.executeQuery("select * from (select row_number() over () as r,x from t2,t3 where x=y) s(r,x) where r < 3");
 		expectedRows = new String[][]{{"1", "4"}, {"2", "5"}};
@@ -232,7 +296,19 @@
 		expectedRows = new String[][]{{"100", "20", "200"}, {"200", "40", "400"}};
 		JDBC.assertFullResultSet(rs, expectedRows);
 
-		/* Group by and having */
+		// Check that flattening does not happen when a window is used in a
+		// subquery
+
+		rs = s.executeQuery("select * from t5 o where o.a in " +
+							"(select x + row_number() over () from t2)");
+		expectedRows = new String[][]{{"2", "4"},
+									  {"4", "4"}};
+		JDBC.assertFullResultSet(rs, expectedRows);
+
+
+		/*
+		 * Group by and having
+		 */
 		rs = s.executeQuery("select r from (select a, row_number() over() as r, b from t1) x group by r");
 		expectedRows = new String[][]{{"1"}, {"2"}, {"3"}, {"4"}, {"5"}};
 		JDBC.assertFullResultSet(rs, expectedRows);
@@ -301,36 +377,221 @@
 										{"5", "ABC"}};
 		JDBC.assertFullResultSet(rs, expectedRows);
 		
-		rs = s.executeQuery("select * from (select distinct row_number() over (), 'ABC' from t1) tmp");
+		rs = s.executeQuery(
+			"select * from (select distinct row_number() over (), " +
+			"                               'ABC' from t1) tmp");
 		expectedRows = new String[][]{{"1", "ABC"},
 										{"2", "ABC"},
 										{"3", "ABC"},
 										{"4", "ABC"},
 										{"5", "ABC"}};
 		JDBC.assertFullResultSet(rs, expectedRows);
-		
-		/*
-		 * Negative testing of Statements
-		 */
 
-		// Missing required OVER () 
-		assertStatementError("42X01", s, "select row_number() as r, * from t1 where t1.a > 2");
+        // Test explicitly declared window
+        rs = s.executeQuery(
+            "select * from (select distinct row_number() over w, 'ABC' " +
+                            "from t1 window w as ()) tmp");
+        JDBC.assertFullResultSet(rs, expectedRows);
 
-		// Illegal where clause, r not a named column of t1.        
-		assertStatementError("42X04", s, "select row_number() over () as r, a from t1 where r < 3");
+        // DERBY-3634 Cannot use row_number() in ORDER BY clause
+        rs = s.executeQuery(
+            "select row_number() over () r, a from t1 order by r desc");
+        expectedRows = new String[][]{{"5", "50"},
+                                      {"4", "40"},
+                                      {"3", "30"},
+                                      {"2", "20"},
+                                      {"1", "10"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
 
-		// Illegal use of asterix with another column identifier.        
-		assertStatementError("42X01", s, "select row_number() over () as r, * from t1 where t1.a > 2");
+        rs = s.executeQuery(
+            "select a from t1 order by row_number() over () desc");
+        expectedRows = new String[][]{{"50"},
+                                      {"40"},
+                                      {"30"},
+                                      {"20"},
+                                      {"10"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
+
+        // Another case mentioned in DERBY-3634
+        rs = s.executeQuery("select a, row_number() over () from t1 except " +
+                            "select a, row_number() over () from t1");
+        JDBC.assertEmpty(rs);
+
+        // And yet another case mentioned in DERBY-3634 This actually also
+        // tests that ROW_NUMBER in an ORDER BY does not get optimized away if
+        // there is a restriction, see SelectNode's call to
+        // orderByList.removeConstantColumns(wherePredicates).
+        rs = s.executeQuery("select abs(a), row_number() over () c " +
+                            "from t1 where a > 30 and a <= 50 " +
+                            "order by c desc");
+        expectedRows = new String[][]{{"50", "2"},
+                                      {"40", "1"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
+
+
+        // DERBY-3635 Cannot build SELECT LIST expressions involving
+        // ROW_NUMBER()
+        rs = s.executeQuery("select row_number() over () + 10, a from t1 " +
+                            "order by row_number() over () desc");
+        expectedRows = new String[][]{{"15", "50"},
+                                      {"14", "40"},
+                                      {"13", "30"},
+                                      {"12", "20"},
+                                      {"11", "10"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
+
+
+        // Check that a non-used window doesn't cause problems
+        rs = s.executeQuery("select a from t1 window r as () order by a desc");
+        expectedRows = new String[][]{{"50"},
+                                      {"40"},
+                                      {"30"},
+                                      {"20"},
+                                      {"10"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
+
+        // Check that a row_number combined with group by works. Good to test
+        // this since windowing uses a similar rewrite mechanism to group by
+        // and could interfere (seen during development).
+        rs = s.executeQuery("select row_number() over r, b, sum(a) from t5 " +
+                            "group by b window r as ()");
+        expectedRows = new String[][]{{"1", "1", "1"},
+                                      {"2", "4", "9"},
+                                      {"3", "9", "5"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
+
+        // Check that a row_number combined with group by works. Good to test
+        // this since windowing uses a similar rewrite mechanism to group by
+        // and could interfere (seen during development).
+        rs = s.executeQuery("select row_number() over r, b, sum(a) from t5 " +
+                            "group by b window r as ()");
+        expectedRows = new String[][]{{"1", "1", "1"},
+                                      {"2", "4", "9"},
+                                      {"3", "9", "5"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
+
+        rs = s.executeQuery("select b, sum(a) from t5 " +
+                            "group by b window r as ()");
+        expectedRows = new String[][]{{"1", "1"},
+                                      {"4", "9"},
+                                      {"9", "5"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
+
+        // Check that row_number inside EXISTS works.
+        rs = s.executeQuery("SELECT * FROM t2 WHERE EXISTS " +
+                            "(SELECT ROW_NUMBER() OVER () FROM t5)");
+        expectedRows = new String[][]{{"1"},{"2"},{"3"},{"4"},{"5"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
+
+        /*
+         * Negative testing of Statements
+         */
+
+        // Missing required OVER ()
+        assertStatementError(
+            LANG_SYNTAX_ERROR, s,
+			"select row_number() as r, * from t1 where t1.a > 2");
+
+        // Illegal where clause, r not a named column of t1.
+        assertStatementError(
+            LANG_COLUMN_NOT_FOUND, s,
+            "select row_number() over () as r, a from t1 where r < 3");
+
+        // Illegal use of asterisk with another column identifier.
+        assertStatementError(
+            LANG_SYNTAX_ERROR, s,
+            "select row_number() over () as r, * from t1 where t1.a > 2");
+
+        // Order by in window specification
+        assertStatementError(
+            NOT_IMPLEMENTED,
+            s,
+            "select row_number() over (order by i) as r from t1");
+
+        // Other window function than row_number:
+        assertStatementError(NOT_IMPLEMENTED,
+                             s,
+                             "select max(i) over () from t1");
+
+        // Illegal context: WHERE, cf. SQL 2003, section 7.8 SR 2
+        assertStatementError(LANG_WINDOW_FUNCTION_CONTEXT_ERROR,
+                             s,
+                             "select * from t4 where row_number() over () > 3");
+
+        // But nested inside a subquery it should work:
+        rs = s.executeQuery("select * from t2 where x in " +
+                            "     (select row_number() over () from t4)");
+        expectedRows = new String[][]{{"1"},{"2"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
+
+        // Illegal context: GROUP BY
+        assertStatementError(LANG_WINDOW_FUNCTION_CONTEXT_ERROR,
+                             s,
+                             "select * from t4 group by row_number() over ()");
+
+        // But nested inside a subquery it should work.
+        // Fails: DERBY-4403, enable this test when that issue is fixed.
+        // rs = s.executeQuery("select * from t4 group by a + " +
+        //             "(select row_number() over () from t4 where a=10)");
+        // JDBC.assertEmpty(rs);
+
+
+        // Illegal context: HAVING, cf. SQL 2003, section 7.10 SR 4
+        assertStatementError(
+			LANG_WINDOW_FUNCTION_CONTEXT_ERROR,
+			s,
+			"select * from t4 group by a having b = row_number() over ()");
 
-		/*
-		 * Clean up the tables used.
-		 */
-		s.executeUpdate("drop table t1");
-		s.executeUpdate("drop table t2");
-		s.executeUpdate("drop table t3");
+        // But nested inside a subquery it should work:
+        rs = s.executeQuery(
+            "select sum(a) from t5 group by b " +
+            "   having b = (select row_number() over () + 3 " +
+            "                   from t5 where a=1)");
+        expectedRows = new String[][]{{"9"}};
+
+
+        // Illegal context: VALUES
+        assertStatementError(LANG_WINDOW_FUNCTION_CONTEXT_ERROR,
+                             s,
+                             "values row_number() over ()");
+
+        // But nested inside a subquery it should work:
+        rs = s.executeQuery("values 3 + " +
+                            "(select row_number() over () from t2 where x=1)");
+        expectedRows = new String[][]{{"4"}};
+        JDBC.assertFullResultSet(rs, expectedRows);
+
+        // Illegal context: Aggregate function, cf. SQL 2003, section 10.9 SR 7
+        // a)
+        assertStatementError(LANG_WINDOW_FUNCTION_CONTEXT_ERROR,
+                             s,
+                             "select sum(row_number() over ()) from t4");
+
+        // Illegal context: JOIN ON clause, cf. SQL 2003, section 7.7 SR 5
+        assertStatementError(
+            LANG_WINDOW_FUNCTION_CONTEXT_ERROR,
+            s,
+            "select * from t4 t_1 join t4 t_2 on " +
+            "                     t_1.a = row_number() over () + t_2.a");
+    }
+
+
+    public void testMetaData()
+        throws SQLException {
+
+        if (JDBC.vmSupportsJSR169()) {
+            // does not support metadata
+            return;
+        }
+
+        Statement s = createStatement();
+        ResultSet rs = s.executeQuery(
+            "select row_number() over () from sys.systables");
+        ResultSetMetaData rsmd = rs.getMetaData();
+
+        assertEquals(ResultSetMetaData.columnNoNulls, rsmd.isNullable(1));
+    }
 
-		s.close();
-	}
 
     private String makeString(int len)
     {
@@ -679,7 +940,7 @@
                 {"DEF","2","350","1","20"},
                 {null,"8","1075","4","215"},
             });
-        // Show a usage of disinct shipping aggregate, similar to the
+        // Show a usage of distinct shipping aggregate, similar to the
         // distinct count aggregate:
         JDBC.assertUnorderedResultSet( s.executeQuery(
             "select o.Customer, count(*) as items_per_customer, " +
@@ -883,8 +1144,4 @@
 
         s.close();
     }
-
-	public static Test suite() {
-		return TestConfiguration.defaultSuite(OLAPTest.class);
-	}
 }

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?rev=829155&r1=829154&r2=829155&view=diff
==============================================================================
--- 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 Fri Oct 23 18:05:03 2009
@@ -202,6 +202,7 @@
         suite.addTest(UserLobTest.suite());
         suite.addTest(OffsetFetchNextTest.suite());
         suite.addTest(SequenceTest.suite());
+        suite.addTest(OLAPTest.suite());
 
         return suite;
 	}

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/util/SQLStateConstants.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/util/SQLStateConstants.java?rev=829155&r1=829154&r2=829155&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/util/SQLStateConstants.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/util/SQLStateConstants.java Fri Oct 23 18:05:03 2009
@@ -101,8 +101,12 @@
         = "22006";
     public static final String DATA_EXCEPTION_INVALID_PARAMETER_VALUE 
         = "22023";
-    public static final String DATA_EXCEPTION_INVALID_PRECEDING_OR_FOLLOWING_SIZE_IN_WINDOW_FUNCTION 
-        = "22013";
+
+
+    // Derby uses 22013 for something else, cf. SQLState.LANG_SQRT_OF_NEG_NUMBER
+    // public static final String DATA_EXCEPTION_INVALID_PRECEDING_OR_FOLLOWING_SIZE_IN_WINDOW_FUNCTION
+    //    = "22013";
+
     public static final String DATA_EXCEPTION_INVALID_REGULAR_EXPRESSION 
         = "2201B";
     public static final String DATA_EXCEPTION_INVALID_REPEAT_ARGUMENT_IN_A_SAMPLE_CLAUSE 

Modified: db/derby/code/trunk/tools/jar/DBMSnodes.properties
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/tools/jar/DBMSnodes.properties?rev=829155&r1=829154&r2=829155&view=diff
==============================================================================
--- db/derby/code/trunk/tools/jar/DBMSnodes.properties (original)
+++ db/derby/code/trunk/tools/jar/DBMSnodes.properties Fri Oct 23 18:05:03 2009
@@ -140,9 +140,16 @@
 derby.module.cloudscapenodes.gq=org.apache.derby.impl.sql.compile.SetRoleNode
 derby.module.cloudscapenodes.gr=org.apache.derby.impl.sql.compile.GrantRoleNode
 derby.module.cloudscapenodes.gs=org.apache.derby.impl.sql.compile.RevokeRoleNode
-derby.module.cloudscapenodes.gv=org.apache.derby.impl.sql.compile.RowCountNode
-derby.module.cloudscapenodes.gw=org.apache.derby.impl.sql.compile.CreateSequenceNode
-derby.module.cloudscapenodes.gx=org.apache.derby.impl.sql.compile.DropSequenceNode
+derby.module.cloudscapenodes.gt=org.apache.derby.impl.sql.compile.RowCountNode
+derby.module.cloudscapenodes.gu=org.apache.derby.impl.sql.compile.CreateSequenceNode
+derby.module.cloudscapenodes.gv=org.apache.derby.impl.sql.compile.DropSequenceNode
+derby.module.cloudscapenodes.gw=org.apache.derby.impl.sql.compile.AggregateWindowFunctionNode
+derby.module.cloudscapenodes.gx=org.apache.derby.impl.sql.compile.RowNumberFunctionNode
+derby.module.cloudscapenodes.gy=org.apache.derby.impl.sql.compile.WindowDefinitionNode
+derby.module.cloudscapenodes.gz=org.apache.derby.impl.sql.compile.WindowFunctionNode
+derby.module.cloudscapenodes.ha=org.apache.derby.impl.sql.compile.WindowNode
+derby.module.cloudscapenodes.hb=org.apache.derby.impl.sql.compile.WindowReferenceNode
+derby.module.cloudscapenodes.hc=org.apache.derby.impl.sql.compile.WindowResultSetNode
 
 # Warning: make sure this file is properly terminated with a newline,
 # else the build can fail silently. Symptom: derby.jar lacks many