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 rh...@apache.org on 2013/08/21 15:26:06 UTC

svn commit: r1516157 - in /db/derby/code/trunk/java: engine/org/apache/derby/impl/sql/compile/ engine/org/apache/derby/loc/ shared/org/apache/derby/shared/common/reference/ testing/org/apache/derbyTesting/functionTests/tests/lang/

Author: rhillegas
Date: Wed Aug 21 13:26:06 2013
New Revision: 1516157

URL: http://svn.apache.org/r1516157
Log:
DERBY-3155: Add grammar and bind logic for MERGE statement; commit derby-3155-01-ac-grammar.diff.

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MatchingClauseNode.java   (with props)
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MergeNode.java   (with props)
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UnionNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/sqlgrammar.jj
    db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml
    db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/GeneratedColumnsHelper.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java?rev=1516157&r1=1516156&r2=1516157&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java Wed Aug 21 13:26:06 2013
@@ -803,7 +803,7 @@ class DeleteNode extends DMLModStatement
                                               null, /* optimizer override plan */
                                               getContextManager());
 
-        return new UpdateNode(tableName, sn, getContextManager());
+        return new UpdateNode(tableName, sn, false, getContextManager());
 
     }
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java?rev=1516157&r1=1516156&r2=1516157&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java Wed Aug 21 13:26:06 2013
@@ -306,6 +306,18 @@ public final class InsertNode extends DM
              ((UnionNode)resultSet).tableConstructor()) ||
             resultSet instanceof RowResultSetNode;
 
+        //
+        // For the MERGE statement, DEFAULT expressions in the SELECT node
+        // may have been replaced with generated expressions already.
+        //
+        ResultColumnList    tempRCL = resultSet.getResultColumns();
+        boolean defaultsWereReplaced = false;
+        for ( int i = 0; i < tempRCL.size(); i++ )
+        {
+            ResultColumn    rc = tempRCL.getResultColumn( i+1 );
+            if ( rc.wasDefaultColumn() ) { defaultsWereReplaced = true; }
+        }
+
         resultSet.replaceOrForbidDefaults(targetTableDescriptor,
                                           targetColumnList,
                                           isTableConstructor);
@@ -436,7 +448,7 @@ public final class InsertNode extends DM
 
         bindOffsetFetch(offset, fetchFirst);
 
-		resultSet = enhanceAndCheckForAutoincrement(resultSet, inOrder, colMap);
+		resultSet = enhanceAndCheckForAutoincrement( resultSet, inOrder, colMap, defaultsWereReplaced );
 
 		resultColumnList.checkStorableExpressions(resultSet.getResultColumns());
 		/* Insert a NormalizeResultSetNode above the source if the source
@@ -550,12 +562,18 @@ public final class InsertNode extends DM
 	 * @param resultSet			current node in the result set tree
 	 * @param inOrder			FALSE if the column list needs reordering
 	 * @param colMap            correspondence between RCLs
+	 * @param defaultsWereReplaced  true if DEFAULT clauses were replaced with generated expressions
 	 * @return a node representing the source for the insert
 	 *
 	 * @exception StandardException Thrown on error
 	 */
-	ResultSetNode enhanceAndCheckForAutoincrement(
-			ResultSetNode resultSet, boolean inOrder, int[] colMap)
+	ResultSetNode enhanceAndCheckForAutoincrement
+        (
+         ResultSetNode resultSet,
+         boolean inOrder,
+         int[] colMap,
+         boolean    defaultsWereReplaced
+         )
 		throws StandardException
 	{
 		/*
@@ -593,7 +611,7 @@ public final class InsertNode extends DM
 			// doesn't tell whether or not DEFAULT is specified at the leaf
 			// level, we need to skip it here to avoid false positives.
 		} else {
-			resultColumnList.forbidOverrides(resultSet.getResultColumns());
+			resultColumnList.forbidOverrides( resultSet.getResultColumns(), defaultsWereReplaced );
 		}
 
 		return resultSet;

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java?rev=1516157&r1=1516156&r2=1516157&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java Wed Aug 21 13:26:06 2013
@@ -801,36 +801,7 @@ class JoinNode extends TableOperatorNode
 		/* ON clause */
 		if (joinClause != null)
 		{
-			/* Create a new fromList with only left and right children before
-			 * binding the join clause. Valid column references in the join clause
-			 * are limited to columns from the 2 tables being joined. This
-			 * algorithm enforces that.
-			 */
-            FromList fromList = new FromList(
-                    getOptimizerFactory().doJoinOrderOptimization(), cm);
-
-			fromList.addElement((FromTable) leftResultSet);
-			fromList.addElement((FromTable) rightResultSet);
-
-            int previousReliability = orReliability( CompilerContext.ON_CLAUSE_RESTRICTION );
-			joinClause = joinClause.bindExpression(
-									  fromList, subqueryList,
-                                      aggregates);
-            cc.setReliability( previousReliability );
-
-			// SQL 2003, section 7.7 SR 5
-			SelectNode.checkNoWindowFunctions(joinClause, "ON");
-
-			/*
-			** We cannot have aggregates in the ON clause.
-			** In the future, if we relax this, we'll need
-            ** to be able to pass the list of aggregates up
-			** the tree.
-			*/
-            if (!aggregates.isEmpty())
-			{
-				throw StandardException.newException(SQLState.LANG_NO_AGGREGATES_IN_ON_CLAUSE);
-			}
+            joinClause = bindExpression( joinClause, true, true, "ON" );
 		}
 		/* USING clause */
 		else if (usingClause != null)
@@ -934,6 +905,69 @@ class JoinNode extends TableOperatorNode
 	}
 
     /**
+     * Bind an expression against the child tables of the JoinNode. May
+     * update the subquery and aggregate lists in the JoinNode. Assumes that
+     * the subquery and aggregate lists for the JoinNode have already been created.
+     *
+     * @return the bound expression
+     */
+    public  ValueNode   bindExpression
+        (
+         ValueNode expression,
+         boolean    useLeftChild,
+         boolean    useRightChild,
+         String expressionType
+         )
+        throws StandardException
+    {
+        ContextManager cm = getContextManager();
+        CompilerContext cc = getCompilerContext();
+
+        /* Create a new fromList with only left and right children before
+         * binding the join clause. Valid column references in the join clause
+         * are limited to columns from the 2 tables being joined. This
+         * algorithm enforces that.
+         */
+        FromList fromList = makeFromList( useLeftChild, useRightChild );
+
+        int previousReliability = orReliability( CompilerContext.ON_CLAUSE_RESTRICTION );
+        expression = expression.bindExpression( fromList, subqueryList, aggregates );
+        cc.setReliability( previousReliability );
+
+        // SQL 2003, section 7.7 SR 5
+        SelectNode.checkNoWindowFunctions( expression, expressionType );
+
+        /*
+        ** We cannot have aggregates in the ON clause.
+        ** In the future, if we relax this, we'll need
+        ** to be able to pass the list of aggregates up
+        ** the tree.
+        */
+        if ( !aggregates.isEmpty() )
+        {
+            throw StandardException.newException(SQLState.LANG_NO_AGGREGATES_IN_ON_CLAUSE);
+        }
+
+        return expression;
+    }
+
+    /** Make a FromList for binding */
+    public  FromList    makeFromList
+        (
+         boolean    useLeftChild,
+         boolean    useRightChild
+         )
+        throws StandardException
+    {
+        FromList fromList = new FromList( getOptimizerFactory().doJoinOrderOptimization(), getContextManager() );
+
+        if ( useLeftChild ) { fromList.addElement((FromTable) leftResultSet); }
+        if ( useRightChild ) { fromList.addElement((FromTable) rightResultSet); }
+
+        return fromList;
+    }
+    
+    /**
      * Generate a result column list with all the column names that appear on
      * both sides of the join operator. Those are the columns to use as join
      * columns in a natural join.

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MatchingClauseNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MatchingClauseNode.java?rev=1516157&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MatchingClauseNode.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MatchingClauseNode.java Wed Aug 21 13:26:06 2013
@@ -0,0 +1,244 @@
+/*
+
+   Derby - Class org.apache.derby.impl.sql.compile.MatchingClauseNode
+
+   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.compile;
+
+import java.util.ArrayList;
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.reference.SQLState;
+import org.apache.derby.iapi.services.context.ContextManager;
+import org.apache.derby.shared.common.sanity.SanityManager;
+
+/**
+ * Node representing a WHEN MATCHED or WHEN NOT MATCHED clause
+ * in a MERGE statement.
+ *
+ */
+
+public class MatchingClauseNode extends QueryTreeNode
+{
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTANTS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // STATE
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    // filled in by the constructor
+    private ValueNode           _matchingRefinement;
+    private ResultColumnList    _updateColumns;
+    private ResultColumnList    _insertColumns;
+    private ResultColumnList    _insertValues;
+
+    // filled in at bind() time
+    private DMLModStatementNode _dml;
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTRUCTORS/FACTORY METHODS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /**
+     * Constructor called by factory methods.
+     */
+    private MatchingClauseNode
+        (
+         ValueNode  matchingRefinement,
+         ResultColumnList   updateColumns,
+         ResultColumnList   insertColumns,
+         ResultColumnList   insertValues,
+         ContextManager     cm
+         )
+    {
+        super( cm );
+        
+        _matchingRefinement = matchingRefinement;
+        _updateColumns = updateColumns;
+        _insertColumns = insertColumns;
+        _insertValues = insertValues;
+    }
+
+    /** Make a WHEN MATCHED ... THEN UPDATE clause */
+    public  static  MatchingClauseNode   makeUpdateClause
+        (
+         ValueNode  matchingRefinement,
+         ResultColumnList   updateColumns,
+         ContextManager     cm
+         )
+    {
+        return new MatchingClauseNode( matchingRefinement, updateColumns, null, null, cm );
+    }
+
+    /** Make a WHEN MATCHED ... THEN DELETE clause */
+    public  static  MatchingClauseNode   makeDeleteClause
+        (
+         ValueNode  matchingRefinement,
+         ContextManager     cm
+         )
+    {
+        return new MatchingClauseNode( matchingRefinement, null, null, null, cm );
+    }
+
+    /** Make a WHEN NOT MATCHED ... THEN INSERT clause */
+    public  static  MatchingClauseNode   makeInsertClause
+        (
+         ValueNode  matchingRefinement,
+         ResultColumnList   insertColumns,
+         ResultColumnList   insertValues,
+         ContextManager     cm
+         )
+    {
+        return new MatchingClauseNode( matchingRefinement, null, insertColumns, insertValues, cm );
+    }
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // ACCESSORS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /** Return true if this is a WHEN MATCHED ... UPDATE clause */
+    public  boolean isUpdateClause()    { return (_updateColumns != null); }
+    
+    /** Return true if this is a WHEN NOT MATCHED ... INSERT clause */
+    public  boolean isInsertClause()    { return (_insertValues != null); }
+    
+    /** Return true if this is a WHEN MATCHED ... DELETE clause */
+    public  boolean isDeleteClause()    { return !( isUpdateClause() || isInsertClause() ); }
+
+    /** Return the bound DML statement--returns null if called before binding */
+    public  DMLModStatementNode getDML()    { return _dml; }
+    
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // bind() BEHAVIOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /** Bind this WHEN [ NOT ] MATCHED clause against the parent JoinNode */
+    public void    bind( JoinNode joinNode, FromTable targetTable )
+        throws StandardException
+    {
+        String  clauseType = isInsertClause() ? "WHEN NOT MATCHED" : "WHEN MATCHED";
+
+        // For WHEN NOT MATCHED clauses, the target table is not in scope.
+        boolean useTargetTable = !isInsertClause();
+
+        if ( _matchingRefinement != null )
+        {
+            _matchingRefinement = joinNode.bindExpression
+                ( _matchingRefinement, true, useTargetTable, clauseType );
+        }
+
+        if ( isDeleteClause() ) { bindDelete( joinNode, targetTable ); }
+        if ( isUpdateClause() ) { bindUpdate( joinNode, targetTable ); }
+        if ( isInsertClause() ) { bindInsert( joinNode, targetTable ); }
+    }
+
+    /** Bind a WHEN MATCHED ... THEN UPDATE clause */
+    private void    bindUpdate( JoinNode joinNode, FromTable targetTable )
+        throws StandardException
+    {
+        SelectNode  selectNode = new SelectNode
+            (
+             _updateColumns,
+             joinNode.makeFromList( true, true ),
+             null,      // where clause
+             null,      // group by list
+             null,      // having clause
+             null,      // window list
+             null,      // optimizer plan override
+             getContextManager()
+             );
+        _dml = new UpdateNode( targetTable.getTableName(), selectNode, true, getContextManager() );
+
+        _dml.bindStatement();
+    }
+    
+    /** Bind a WHEN MATCHED ... THEN DELETE clause */
+    private void    bindDelete( JoinNode joinNode, FromTable targetTable )
+        throws StandardException
+    {
+        SelectNode  selectNode = new SelectNode
+            (
+             null,      // select list
+             joinNode.makeFromList( true, true ),
+             null,      // where clause
+             null,      // group by list
+             null,      // having clause
+             null,      // window list
+             null,      // optimizer plan override
+             getContextManager()
+             );
+        _dml = new DeleteNode( targetTable.getTableName(), selectNode, getContextManager() );
+
+        _dml.bindStatement();
+    }
+
+    /** Bind a WHEN NOT MATCHED ... THEN INSERT clause */
+    private void    bindInsert( JoinNode joinNode, FromTable targetTable )
+        throws StandardException
+    {
+        // needed to make the SelectNode bind
+        _insertValues.replaceOrForbidDefaults( targetTable.getTableDescriptor(), _insertColumns, true );
+
+        // the VALUES clause may not mention columns in the target table
+        _insertValues.bindExpressions
+            (
+             joinNode.makeFromList( true, false ),
+             new SubqueryList( getContextManager() ),
+             new ArrayList<AggregateNode>()
+             );
+        
+        SelectNode  selectNode = new SelectNode
+            (
+             _insertValues,      // select list
+             joinNode.makeFromList( true, true ),
+             null,      // where clause
+             null,      // group by list
+             null,      // having clause
+             null,      // window list
+             null,      // optimizer plan override
+             getContextManager()
+             );
+        _dml = new InsertNode
+            (
+             targetTable.getTableName(),
+             _insertColumns,
+             selectNode,
+             null,      // targetProperties
+             null,      // order by cols
+             null,      // offset
+             null,      // fetch first
+             false,     // has JDBC limit clause
+             getContextManager()
+             );
+
+        _dml.bindStatement();
+    }
+
+}

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

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MergeNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MergeNode.java?rev=1516157&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MergeNode.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MergeNode.java Wed Aug 21 13:26:06 2013
@@ -0,0 +1,203 @@
+/*
+
+   Derby - Class org.apache.derby.impl.sql.compile.MergeNode
+
+   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.compile;
+
+import java.util.ArrayList;
+
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.reference.SQLState;
+import org.apache.derby.iapi.services.context.ContextManager;
+import org.apache.derby.iapi.sql.dictionary.DataDictionary;
+import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
+
+/**
+ * <p>
+ * A MergeNode represents a MERGE statement.  It is the top node of the
+ * query tree for that statement. The driving result set for a MERGE statement
+ * is essentially the following:
+ * </p>
+ *
+ * <pre>
+ * sourceTable LEFT OUTER JOIN targetTable ON searchCondition
+ * </pre>
+ */
+
+public final class MergeNode extends DMLModStatementNode
+{
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTANTS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // STATE
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    private FromTable   _targetTable;
+    private FromTable   _sourceTable;
+    private ValueNode   _searchCondition;
+    private ArrayList<MatchingClauseNode>   _matchingClauses;
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTRUCTOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /**
+     * <p>
+     * Constructor for a MergeNode.
+     * </p>
+     */
+    public  MergeNode
+        (
+         FromTable          targetTable,
+         FromTable          sourceTable,
+         ValueNode          searchCondition,
+         ArrayList<MatchingClauseNode>  matchingClauses,
+         ContextManager     cm
+         )
+        throws StandardException
+    {
+        super( null, cm );
+
+        _targetTable = targetTable;
+        _sourceTable = sourceTable;
+        _searchCondition = searchCondition;
+        _matchingClauses = matchingClauses;
+
+        makeJoin();
+    }
+
+    /**
+     * <p>
+     * Construct the left outer join which will drive the execution.
+     * </p>
+     */
+    private void    makeJoin() throws StandardException
+    {
+        resultSet = new HalfOuterJoinNode
+            (
+             _sourceTable,
+             _targetTable,
+             _searchCondition,
+             null,
+             false,
+             null,
+             getContextManager()
+             );
+    }
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // bind() BEHAVIOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    @Override
+	public void bindStatement() throws StandardException
+	{
+        DataDictionary  dd = getDataDictionary();
+
+        //
+        // Bind the left join. This binds _targetTable and _sourceTable.
+        //
+        bind( dd );
+
+        bindSearchCondition();
+
+        if ( !targetIsBaseTable() )
+        {
+            throw StandardException.newException( SQLState.LANG_TARGET_NOT_BASE_TABLE );
+        }
+
+        if ( !sourceIsBase_View_or_VTI() )
+        {
+            throw StandardException.newException( SQLState.LANG_SOURCE_NOT_BASE_VIEW_OR_VTI );
+        }
+
+        // source and target may not have the same correlation names
+        if ( getExposedName( _targetTable ).equals( getExposedName( _sourceTable ) ) )
+        {
+            throw StandardException.newException( SQLState.LANG_SAME_EXPOSED_NAME );
+        }
+
+        for ( MatchingClauseNode mcn : _matchingClauses )
+        {
+            mcn.bind( (JoinNode) resultSet, _targetTable );
+        }
+
+        throw StandardException.newException( SQLState.NOT_IMPLEMENTED, "MERGE" );
+	}
+
+    /** Get the exposed name of a FromTable */
+    private String  getExposedName( FromTable ft ) throws StandardException
+    {
+        return ft.getTableName().getTableName();
+    }
+
+    /**  Bind the search condition, the ON clause of the left join */
+    private void    bindSearchCondition()   throws StandardException
+    {
+        FromList    fromList = new FromList
+            ( getOptimizerFactory().doJoinOrderOptimization(), getContextManager() );
+
+        resultSet.bindResultColumns( fromList );
+    }
+
+    /** Return true if the target table is a base table */
+    private boolean targetIsBaseTable() throws StandardException
+    {
+        if ( !( _targetTable instanceof FromBaseTable) ) { return false; }
+
+        FromBaseTable   fbt = (FromBaseTable) _targetTable;
+        TableDescriptor desc = fbt.getTableDescriptor();
+        if ( desc == null ) { return false; }
+
+        return ( desc.getTableType() == TableDescriptor.BASE_TABLE_TYPE );
+    }
+
+    /** Return true if the source table is a base table, view, or table function */
+    private boolean sourceIsBase_View_or_VTI() throws StandardException
+    {
+        if ( _sourceTable instanceof FromVTI ) { return true; }
+        if ( !( _sourceTable instanceof FromBaseTable) ) { return false; }
+
+        FromBaseTable   fbt = (FromBaseTable) _sourceTable;
+        TableDescriptor desc = fbt.getTableDescriptor();
+        if ( desc == null ) { return false; }
+
+        switch( desc.getTableType() )
+        {
+        case TableDescriptor.BASE_TABLE_TYPE:
+        case TableDescriptor.VIEW_TYPE:
+            return true;
+
+        default:
+            return false;
+        }
+    }
+
+}

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

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java?rev=1516157&r1=1516156&r2=1516157&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java Wed Aug 21 13:26:06 2013
@@ -891,6 +891,7 @@ class ResultColumnList extends QueryTree
 						targetTableDescriptor,
 						index + 1
 					);
+            
             int colIdx = rc.getColumnPosition() - 1;
             if( SanityManager.DEBUG)
                 SanityManager.ASSERT( colIdx >= 0 && colIdx < targetTableDescriptor.getNumberOfColumns(),
@@ -4035,6 +4036,19 @@ class ResultColumnList extends QueryTree
     void forbidOverrides(ResultColumnList sourceRSRCL)
 		throws StandardException
 	{
+        forbidOverrides( sourceRSRCL, false );
+    }
+    
+	/**
+	 * check if any autoincrement or generated columns exist in the result column list.
+	 * called from insert or update where you cannot insert/update the value
+	 * of a generated or autoincrement column.
+	 *
+	 * @exception StandardException		If the column is an ai column
+	 */
+    void forbidOverrides(ResultColumnList sourceRSRCL, boolean defaultsWereReplaced )
+		throws StandardException
+	{
 		int size = size();
 
 		for (int index = 0; index < size; index++)
@@ -4046,7 +4060,7 @@ class ResultColumnList extends QueryTree
 
             if ( (cd != null) && cd.hasGenerationClause() )
             {
-                if ( (sourceRC != null) && !sourceRC.hasGenerationClause() && !sourceRC.wasDefaultColumn() )
+                if ( !defaultsWereReplaced && (sourceRC != null) && !sourceRC.hasGenerationClause() && !sourceRC.wasDefaultColumn() )
                 {
                     throw StandardException.newException(SQLState.LANG_CANT_OVERRIDE_GENERATION_CLAUSE, rc.getName());
                 }
@@ -4056,8 +4070,10 @@ class ResultColumnList extends QueryTree
 			
 			if ((cd != null) && (cd.isAutoincrement()))
 			{
-				if ((sourceRC != null) && 
-					(sourceRC.isAutoincrementGenerated()))
+				if (
+                    ( (sourceRC != null) && (sourceRC.isAutoincrementGenerated()) ) ||
+                    ( cd.isAutoincAlways() && defaultsWereReplaced )
+                    )
 				{
 					sourceRC.setColumnDescriptor(cd.getTableDescriptor(), cd);
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UnionNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UnionNode.java?rev=1516157&r1=1516156&r2=1516157&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UnionNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UnionNode.java Wed Aug 21 13:26:06 2013
@@ -195,10 +195,10 @@ class UnionNode extends SetOperatorNode
 		throws StandardException
 	{
 		if (tableConstructor()) {
-			leftResultSet = target.enhanceAndCheckForAutoincrement(
-					leftResultSet, inOrder, colMap);
-			rightResultSet = target.enhanceAndCheckForAutoincrement(
-					rightResultSet, inOrder, colMap);
+			leftResultSet = target.enhanceAndCheckForAutoincrement
+                ( leftResultSet, inOrder, colMap, false );
+			rightResultSet = target.enhanceAndCheckForAutoincrement
+                ( rightResultSet, inOrder, colMap, false );
 			if (!inOrder ||
 					resultColumns.size() < target.resultColumnList.size()) {
 				resultColumns = getRCLForInsert(target, colMap);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java?rev=1516157&r1=1516156&r2=1516157&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java Wed Aug 21 13:26:06 2013
@@ -78,6 +78,8 @@ public final class UpdateNode extends DM
 	protected FormatableBitSet 			readColsBitSet;
 	protected boolean 			positionedUpdate;
 
+    private     boolean         inMatchedClause;
+
 	/* Column name for the RowLocation in the ResultSet */
     static final String COLUMNNAME = "###RowLocationToUpdate";
 
@@ -86,14 +88,17 @@ public final class UpdateNode extends DM
 	 *
 	 * @param targetTableName	The name of the table to update
      * @param resultSet         The ResultSet that we will generate
+     * @param inMatchedClause   True if this UPDATE is part of a MATCHED ... THEN UPDATE clause of a MERGE statement.
      * @param cm                The context manager
 	 */
     UpdateNode(TableName targetTableName,
                ResultSetNode resultSet,
+               boolean  inMatchedClause,
                ContextManager cm)
 	{
         super(resultSet, cm);
         this.targetTableName = targetTableName;
+        this.inMatchedClause = inMatchedClause;
 	}
 
 	/**
@@ -321,10 +326,12 @@ public final class UpdateNode extends DM
 		** Get the result FromTable, which should be the only table in the
 	 	** from list.
 		*/
+        /*
         if (SanityManager.DEBUG) {
             SanityManager.ASSERT(resultSet.getFromList().size() == 1,
                     "More than one table in result from list in an update.");
         }
+        */
 		/* Normalize the SET clause's result column list for synonym */
 		if (synonymTableName != null)
 			normalizeSynonymColumns( resultSet.resultColumns, targetTable );
@@ -435,6 +442,7 @@ public final class UpdateNode extends DM
                                         afterColumns, affectedGeneratedColumns );
 
 				afterColumns = fbt.addColsToList(afterColumns, readColsBitSet);
+                
 				resultColumnList = fbt.addColsToList(resultColumnList, readColsBitSet);
 
 				/*
@@ -1328,11 +1336,15 @@ public final class UpdateNode extends DM
 			 * the cursor, then a match for the ColumnReference would not
 			 * be found if we didn't null out the name.  (Aren't you
 			 * glad you asked?)
+             *
+             * However, we need the table name if this UPDATE is part of a MERGE
+             * statement. If we clear the table name, then we will not be able to
+             * resolve which table (target or source) holds the column.
 			 */
-			column.clearTableName();
+			if ( !inMatchedClause ) { column.clearTableName(); }
 		}
 	}
-	
+
 	/**
 	 * Normalize synonym column references to have the name of the base table. 
 	 *

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=1516157&r1=1516156&r2=1516157&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 Wed Aug 21 13:26:06 2013
@@ -757,7 +757,7 @@ public class SQLParser
                                               getContextManager());
 
         StatementNode retval =
-            new UpdateNode(tableName, resultSet, getContextManager());
+            new UpdateNode(tableName, resultSet, false, getContextManager());
 
 		setUpAndLinkParameters();
 
@@ -2344,7 +2344,9 @@ TOKEN [IGNORE_CASE] :
 |	<LOCKS: "locks">
 |	<LOCKSIZE: "locksize">
 |	<LOGGED: "logged">
+|	<MATCHED: "matched">
 |	<MAXVALUE: "maxvalue">
+|	<MERGE: "merge">
 |	<MINVALUE: "minvalue">
 |       <MOD: "mod">
 |	<MODIFIES: "modifies">
@@ -2885,6 +2887,8 @@ proceduralStatement(Token[] tokenHolder)
 |
 	statementNode =  preparableSelectStatement(true)	
 |
+	statementNode =  mergeStatement()	
+|
 	statementNode =  callStatement()	
 )
 	{
@@ -3102,6 +3106,11 @@ preparableSQLDataStatement() throws Stan
 		return dmlStatement;
 	}
 |
+	dmlStatement =  mergeStatement()	
+	{
+		return dmlStatement;
+	}
+|
 	dmlStatement = preparableUpdateStatement()
 	{
 		return dmlStatement;
@@ -3555,6 +3564,98 @@ bareCallStatement() throws StandardExcep
 }
 
 /*
+ * <A NAME="mergeStatement">mergeStatement</A>
+ */
+StatementNode
+mergeStatement() throws StandardException :
+{
+	FromTable   targetTable;
+    FromTable       sourceTable;
+    ValueNode       searchCondition;
+    ArrayList<MatchingClauseNode>   matchingClauses;
+}
+{
+	<MERGE> <INTO> targetTable = tableFactor()
+    <USING> sourceTable = tableFactor()
+    searchCondition = joinCondition()
+    matchingClauses = matchingClauseList()
+	{
+        return new MergeNode
+        (
+            targetTable,
+            sourceTable,
+            searchCondition,
+            matchingClauses,
+            getContextManager()
+        );
+	}
+}
+
+/*
+ * <A NAME="matchingClauseList">matchingClauseList</A>
+ */
+ArrayList<MatchingClauseNode>
+matchingClauseList() throws StandardException :
+{
+    ArrayList<MatchingClauseNode>   matchingClauses = new ArrayList<MatchingClauseNode>();
+}
+{
+    matchingClause( matchingClauses ) ( <COMMA> matchingClause( matchingClauses ) ) *
+	{
+        return matchingClauses;
+	}
+}
+
+/*
+ * <A NAME="matchingClause">matchingClause</A>
+ */
+void
+matchingClause( ArrayList<MatchingClauseNode> matchingClauses ) throws StandardException :
+{
+    ValueNode   matchingRefinement = null;
+    ResultColumnList               updateColumns = null;
+    ResultColumnList               insertColumns = null;
+	ResultColumnList               insertValues = new ResultColumnList( getContextManager() );
+    MatchingClauseNode             matchingClause;
+}
+{
+    LOOKAHEAD ( { getToken(1).kind == WHEN && getToken(2).kind != NOT } )
+    <WHEN> <MATCHED> [ <AND> matchingRefinement = valueExpression() ]
+    <THEN>
+    (
+        <DELETE>
+        |
+        <UPDATE> <SET> updateColumns = setClauseList()
+    )
+	{
+        if ( updateColumns != null )
+        {
+            matchingClause = MatchingClauseNode.makeUpdateClause
+            ( matchingRefinement, updateColumns, getContextManager() );
+        }
+        else
+        {
+            matchingClause = MatchingClauseNode.makeDeleteClause
+            ( matchingRefinement, getContextManager() );
+        }
+
+        matchingClauses.add( matchingClause );
+	}
+|
+    LOOKAHEAD ( { getToken(1).kind == WHEN && getToken(2).kind == NOT } )
+    <WHEN> <NOT> <MATCHED> [ <AND> matchingRefinement = valueExpression() ]
+    <THEN>
+    <INSERT> [ <LEFT_PAREN> insertColumns = insertColumnList() <RIGHT_PAREN> ]
+    <VALUES> <LEFT_PAREN> rowValueConstructorList( insertValues ) <RIGHT_PAREN>
+	{
+        matchingClause = MatchingClauseNode.makeInsertClause
+        ( matchingRefinement, insertColumns, insertValues, getContextManager() );
+
+        matchingClauses.add( matchingClause );
+	}
+}
+
+/*
  * <A NAME="primaryExpression">primaryExpression</A>
  */
 ValueNode primaryExpression() throws StandardException :
@@ -14376,7 +14477,9 @@ nonReservedKeyword()  :
 	|	tok = <LOCKSIZE>
 	|	tok = <LOGGED>
 	|	tok = <LONG>
+	|	tok = <MATCHED>
 	|	tok = <MAXVALUE>
+	|	tok = <MERGE>
 	|	tok = <MINVALUE>
 	|	tok = <MESSAGE_LOCALE>
 	|	tok = <METHOD>

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=1516157&r1=1516156&r2=1516157&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 Wed Aug 21 13:26:06 2013
@@ -2248,6 +2248,21 @@ Guide.
             </msg>
 
             <msg>
+                <name>42XAK</name>
+                <text>The target table of a MERGE statement must be a base table.</text>
+            </msg>
+
+            <msg>
+                <name>42XAL</name>
+                <text>The source table of a MERGE statement must be a base table, view, or table function.</text>
+            </msg>
+
+            <msg>
+                <name>42XAM</name>
+                <text>The source and target tables of a MERGE statement may not have the same correlation name.</text>
+            </msg>
+
+            <msg>
                 <name>42Y00</name>
                 <text>Class '{0}' does not implement org.apache.derby.iapi.db.AggregateDefinition and thus cannot be used as an aggregate expression.</text>
                 <arg>className</arg>

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=1516157&r1=1516156&r2=1516157&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 Wed Aug 21 13:26:06 2013
@@ -942,6 +942,9 @@ public interface SQLState {
     String LANG_NEXT_VALUE_FOR_ILLEGAL                                      = "42XAH";    
     String LANG_SEQUENCE_REFERENCED_TWICE                                      = "42XAI";    
     String LANG_DUPLICATE_CS_CLAUSE                                      = "42XAJ";    
+    String LANG_TARGET_NOT_BASE_TABLE                                  = "42XAK";    
+    String LANG_SOURCE_NOT_BASE_VIEW_OR_VTI                        = "42XAL";    
+    String LANG_SAME_EXPOSED_NAME                                       = "42XAM";    
     String LANG_INVALID_USER_AGGREGATE_DEFINITION2                     = "42Y00";
 	String LANG_INVALID_CHECK_CONSTRAINT                               = "42Y01";
 	// String LANG_NO_ALTER_TABLE_COMPRESS_ON_TARGET_TABLE                = "42Y02";

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/GeneratedColumnsHelper.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/GeneratedColumnsHelper.java?rev=1516157&r1=1516156&r2=1516157&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/GeneratedColumnsHelper.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/GeneratedColumnsHelper.java Wed Aug 21 13:26:06 2013
@@ -47,6 +47,7 @@ public class GeneratedColumnsHelper exte
     //
     ///////////////////////////////////////////////////////////////////////////////////
 
+    protected static final    String NOT_IMPLEMENTED = "0A000";
     protected static final    String OBJECT_DOES_NOT_EXIST = "42X94";
     protected static final    String NONEXISTENT_OBJECT = "42Y55";
     protected static  final   String  REDUNDANT_CLAUSE = "42613";

Added: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.java?rev=1516157&view=auto
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.java (added)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.java Wed Aug 21 13:26:06 2013
@@ -0,0 +1,577 @@
+/*
+
+   Derby - Class org.apache.derbyTesting.functionTests.tests.lang.MergeStatementTest
+
+   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 java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.Connection;
+import java.sql.Statement;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.DriverManager;
+import java.util.ArrayList;
+import junit.framework.Test;
+import junit.framework.TestSuite;
+import org.apache.derby.iapi.util.StringUtil;
+import org.apache.derby.catalog.DefaultInfo;
+import org.apache.derbyTesting.junit.BaseJDBCTestCase;
+import org.apache.derbyTesting.junit.JDBC;
+import org.apache.derbyTesting.junit.DatabasePropertyTestSetup;
+import org.apache.derbyTesting.junit.JDBC;
+import org.apache.derbyTesting.junit.TestConfiguration;
+import org.apache.derbyTesting.junit.CleanDatabaseTestSetup;
+import org.apache.derbyTesting.junit.JDBC;
+
+/**
+ * <p>
+ * Test the MERGE statement (see DERBY-3155).
+ * </p>
+ */
+public class MergeStatementTest extends GeneratedColumnsHelper
+{
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTANTS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    private static  final   String      TEST_DBO = "TEST_DBO";
+    private static  final   String      RUTH = "RUTH";
+    private static  final   String      ALICE = "ALICE";
+    private static  final   String      FRANK = "FRANK";
+    private static  final   String      TONY = "TONY";
+    private static  final   String[]    LEGAL_USERS = { TEST_DBO, ALICE, RUTH, FRANK, TONY  };
+
+    private static  final   String      TARGET_MUST_BE_BASE = "42XAK";
+    private static  final   String      SOURCE_MUST_BE_BASE_VIEW_OR_VTI = "42XAL";
+    private static  final   String      SAME_EXPOSED_NAME = "42XAM";
+    private static  final   String      DUPLICATE_COLUMNS = "42X13";
+    private static  final   String      COLUMN_NOT_IN_TABLE = "42X14";
+    private static  final   String      COLUMN_COUNT_MISMATCH = "42802";
+    private static  final   String      DUPLICATE_SET_COLUMNS = "42X16";
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // STATE
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTRUCTOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /**
+     * Create a new instance.
+     */
+
+    public MergeStatementTest(String name)
+    {
+        super(name);
+    }
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // JUnit BEHAVIOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+
+    /**
+     * Construct top level suite in this JUnit test
+     */
+    public static Test suite()
+    {
+        TestSuite suite = (TestSuite) TestConfiguration.embeddedSuite(MergeStatementTest.class);
+
+        Test        cleanTest = new CleanDatabaseTestSetup( suite );
+        Test        authenticatedTest = DatabasePropertyTestSetup.builtinAuthentication
+            ( cleanTest, LEGAL_USERS, "MergeStatementPermissions" );
+        Test        authorizedTest = TestConfiguration.sqlAuthorizationDecorator( authenticatedTest );
+
+        return authorizedTest;
+    }
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // TESTS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /**
+     * <p>
+     * Test some bad syntax.
+     * </p>
+     */
+    public  void    test_001_badSyntax()
+        throws Exception
+    {
+        Connection  dboConnection = openUserConnection( TEST_DBO );
+
+        goodStatement
+            ( dboConnection,
+              "create table t1( c1 int generated always as identity, c2 int, c3 int generated always as ( c1 + c2 ), c1_4 int )" );
+        goodStatement( dboConnection, "create table t2( c1 int generated always as identity, c2 int, c3 int, c4 int, c5 varchar( 5 ) )" );
+        goodStatement( dboConnection, "create table t3( c1 int generated always as identity, c2 int, c3 int )" );
+        goodStatement( dboConnection, "create view v1 as select * from t1" );
+        goodStatement( dboConnection, "create view v2 as select * from t2" );
+        goodStatement
+            (
+             dboConnection,
+             "create function integerList()\n" +
+             "returns table( s_r int, s_nr int, ns_r int, ns_nr int )\n" +
+             "language java\n" +
+             "parameter style derby_jdbc_result_set\n" +
+             "no sql\n" +
+             "external name 'org.apache.derbyTesting.functionTests.tests.lang.RestrictedVTITest.integerList'\n"
+             );
+        goodStatement
+            (
+             dboConnection,
+             "create function illegalFunction() returns int\n" +
+             "language java parameter style java contains sql\n" +
+             "external name 'org.apache.derbyTesting.functionTests.tests.lang.MergeStatementTest.illegalFunction'\n"
+             );
+
+        // INSERT only allowed with NOT MATCHED
+        expectCompilationError
+            ( dboConnection, SYNTAX_ERROR,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched then insert\n"
+              );
+        // DELETE only allowed with MATCHED
+        expectCompilationError
+            ( dboConnection, SYNTAX_ERROR,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then delete\n"
+              );
+        // UPDATE only allowed with MATCHED
+        expectCompilationError
+            ( dboConnection, SYNTAX_ERROR,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then update\n"
+              );
+
+        // Target table must be a base table.
+        expectCompilationError
+            ( dboConnection, TARGET_MUST_BE_BASE,
+              "merge into v1\n" +
+              "using t2\n" +
+              "on v1.c1 = t2.c1\n" +
+              "when matched then delete\n"
+              );
+        expectCompilationError
+            ( dboConnection, TARGET_MUST_BE_BASE,
+              "merge into table( integerList() ) i\n" +
+              "using t2\n" +
+              "on i.s_r = t2.c1\n" +
+              "when matched then delete\n"
+              );
+        expectCompilationError
+            ( dboConnection, TARGET_MUST_BE_BASE,
+              "merge into (t1 left join t3 on t1.c3 = t3.c3)\n" +
+              "using t2\n" +
+              "on c4 is not null\n" +
+              "when matched then delete\n"
+              );
+        expectCompilationError
+            ( dboConnection, TARGET_MUST_BE_BASE,
+              "merge into v1 v\n" +
+              "using t2 t\n" +
+              "on v.c1 = t.c1\n" +
+              "when matched then delete\n"
+              );
+
+        // Source must be a base table, view, or table function.
+        expectCompilationError
+            ( dboConnection, SOURCE_MUST_BE_BASE_VIEW_OR_VTI,
+              "merge into t1\n" +
+              "using ( t3 left join t2 on t3.c3 = t2.c3 )\n" +
+              "on t1.c1 is not null\n" +
+              "when matched then delete\n"
+              );
+
+        // Source and target may not have the same correlation names.
+        expectCompilationError
+            ( dboConnection, SAME_EXPOSED_NAME,
+              "merge into t1 t2\n" +
+              "using t2\n" +
+              "on c4 is null\n" +
+              "when matched then delete\n"
+              );
+        expectCompilationError
+            ( dboConnection, SAME_EXPOSED_NAME,
+              "merge into t1\n" +
+              "using t2 t1\n" +
+              "on c4 is not null\n" +
+              "when matched then delete\n"
+              );
+        expectCompilationError
+            ( dboConnection, SAME_EXPOSED_NAME,
+              "merge into t1 v\n" +
+              "using t2 v\n" +
+              "on c4 is not null\n" +
+              "when matched then delete\n"
+              );
+
+        // ON clause may only refer to columns in the source and target tables
+        expectCompilationError
+            ( dboConnection, COLUMN_OUT_OF_SCOPE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t3.c1 = t2.c1\n" +
+              "when matched then delete\n"
+              );
+
+        //
+        // The matching refinement clauses can only mention columns in the
+        // source and target tables.
+        //
+        expectCompilationError
+            ( dboConnection, COLUMN_OUT_OF_SCOPE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched and t3.c2 is null then insert ( c2 ) values ( t2.c2 )\n"
+              );
+        expectCompilationError
+            ( dboConnection, COLUMN_OUT_OF_SCOPE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t3.c2 = t2.c2 then update set c3 = t2.c3\n"
+              );
+        expectCompilationError
+            ( dboConnection, COLUMN_OUT_OF_SCOPE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t3.c2 = t2.c2 then delete\n"
+              );
+
+        //
+        // The refining restriction of a WHEN NOT MATCHED clause may not
+        // reference columns in the target table.
+        //
+        expectCompilationError
+            ( dboConnection, COLUMN_OUT_OF_SCOPE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched and t1.c2 is null then insert ( c2 ) values ( t2.c2 )\n"
+              );
+
+        // Should fail at run time because the function modifies sql data.
+        // But for now, this doesn't make it past the bind() phase.
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1 and t2.c2 = illegalFunction()\n" +
+              "when matched then delete\n"
+              );
+
+        // Can only specify DEFAULT as the value of a generated column.
+        expectCompilationError
+            ( dboConnection, CANT_MODIFY_IDENTITY,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c1, c2 ) values ( t2.c1, t2.c2 )\n"
+              );
+        expectCompilationError
+            ( dboConnection, CANT_OVERRIDE_GENERATION_CLAUSE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c2, c3 ) values ( t2.c2, t2.c3 )\n"
+              );
+        expectCompilationError
+            ( dboConnection, CANT_MODIFY_IDENTITY,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t1.c2 != t2.c2 then update set c1 = t2.c2\n"
+              );
+        expectCompilationError
+            ( dboConnection, CANT_OVERRIDE_GENERATION_CLAUSE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t1.c2 != t2.c2 then update set c3 = t2.c2\n"
+              );
+
+        // and you can't update an identity column at all
+        expectCompilationError
+            ( dboConnection, CANT_MODIFY_IDENTITY,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t1.c2 != t2.c2 then update set c1 = default, c2 = t2.c2\n"
+              );
+
+        // Column may not appear twice in INSERT list.
+        expectCompilationError
+            ( dboConnection, DUPLICATE_COLUMNS,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c2, c2 ) values ( t2.c2, t2.c3 )\n"
+              );
+
+        // INSERTed column name must be in the target table
+        expectCompilationError
+            ( dboConnection, COLUMN_NOT_IN_TABLE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c2, c5 ) values ( t2.c2, t2.c3 )\n"
+              );
+
+        // INSERTed value must be storable in target column.
+        expectCompilationError
+            ( dboConnection, ILLEGAL_STORAGE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c2 ) values ( t2.c5 )\n"
+              );
+
+        // INSERT refinement clause can't mention columns in the target table
+        expectCompilationError
+            ( dboConnection, COLUMN_OUT_OF_SCOPE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c2 ) values ( c1_4 )\n"
+              );
+
+        // Must have same number of columns in INSERT and VALUES lists
+        expectCompilationError
+            ( dboConnection, COLUMN_COUNT_MISMATCH,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c2, c1_4 ) values ( t2.c2 )\n"
+              );
+        expectCompilationError
+            ( dboConnection, COLUMN_COUNT_MISMATCH,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c2 ) values ( t2.c2, t2.c3 )\n"
+              );
+
+        // Trigger tansition tables may not be used as target tables.
+        expectCompilationError
+            ( dboConnection, TARGET_MUST_BE_BASE,
+              "create trigger trig1 after update on t1\n" +
+              "referencing old table as old_cor new table as new_cor\n" +
+              "for each statement\n" +
+              "merge into new_cor\n" +
+              "using t2\n" +
+              "on new_cor.c1 = t2.c1\n" +
+              "when not matched then insert ( c2 ) values ( t2.c2 )\n"
+              );
+        expectCompilationError
+            ( dboConnection, TARGET_MUST_BE_BASE,
+              "create trigger trig2 after update on t1\n" +
+              "referencing old table as old_cor new table as new_cor\n" +
+              "for each statement\n" +
+              "merge into old_cor\n" +
+              "using t2\n" +
+              "on old_cor.c1 = t2.c1\n" +
+              "when not matched then insert ( c2 ) values ( t2.c2 )\n"
+              );
+
+        // Columns may not be SET twice in a MATCHED ... THEN UPDATE clause
+        expectCompilationError
+            ( dboConnection, DUPLICATE_SET_COLUMNS,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t1.c2 != t2.c2 then update set c2 = t2.c2, c2 = t2.c1\n"
+              );
+
+        // SET columns must be in target table
+        expectCompilationError
+            ( dboConnection, COLUMN_NOT_IN_TABLE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t1.c2 != t2.c2 then update set c5 = t2.c2\n"
+              );
+
+        // SET value must be storable in target column.
+        expectCompilationError
+            ( dboConnection, ILLEGAL_STORAGE,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t1.c2 != t2.c2 then update set c2 = t2.c5\n"
+              );
+
+        //
+        // The following syntax is actually good, but the compiler rejects these
+        // statements because we haven't finished implementing MERGE.
+        //
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using table( integerList() ) i\n" +
+              "on t1.c1 = i.s_r\n" +
+              "when matched then delete\n"
+              );
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using v2\n" +
+              "on t1.c1 = v2.c1\n" +
+              "when matched then delete\n"
+              );
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched then delete\n"
+              );
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t1.c2 = t2.c2 then delete\n"
+              );
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched then update set c2 = t2.c3\n"
+              );
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t1.c2 = t2.c2 then update set c2 = t2.c3\n"
+              );
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c2 ) values ( t2.c2 )\n"
+              );
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched and t2.c2 is null then insert ( c2 ) values ( t2.c2 )\n"
+              );
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c1, c2 ) values ( default, t2.c2 )\n"
+              );
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when not matched then insert ( c2, c3 ) values ( t2.c2, default )\n"
+              );
+
+        // Using a trigger transition table as a source table is probably ok.
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "create trigger trig3 after update on t2\n" +
+              "referencing old table as old_cor new table as new_cor\n" +
+              "for each statement\n" +
+              "merge into t1\n" +
+              "using new_cor\n" +
+              "on t1.c1 = new_cor.c1\n" +
+              "when not matched then insert ( c2 ) values ( new_cor.c2 )\n"
+              );
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "create trigger trig4 after update on t2\n" +
+              "referencing old table as old_cor new table as new_cor\n" +
+              "for each statement\n" +
+              "merge into t1\n" +
+              "using old_cor\n" +
+              "on t1.c1 = old_cor.c1\n" +
+              "when not matched then insert ( c2 ) values ( old_cor.c2 )\n"
+              );
+
+        // it's probably ok to specify default values for generated columns in MATCHED ... THEN UPDATE
+        expectCompilationError
+            ( dboConnection, NOT_IMPLEMENTED,
+              "merge into t1\n" +
+              "using t2\n" +
+              "on t1.c1 = t2.c1\n" +
+              "when matched and t1.c2 != t2.c2 then update set c3 = default, c2 = t2.c2\n"
+              );
+        
+
+        goodStatement( dboConnection, "drop function illegalFunction" );
+        goodStatement( dboConnection, "drop function integerList" );
+        goodStatement( dboConnection, "drop view v2" );
+        goodStatement( dboConnection, "drop view v1" );
+        goodStatement( dboConnection, "drop table t3" );
+        goodStatement( dboConnection, "drop table t2" );
+        goodStatement( dboConnection, "drop table t1" );
+    }
+    
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // ROUTINES
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /** Illegal function which performs sql updates */
+    public  static  int illegalFunction() throws Exception
+    {
+        Connection  conn = getNestedConnection();
+
+        conn.prepareStatement( "insert into t1( c2 ) values ( 1 )" ).executeUpdate();
+
+        return 1;
+    }
+
+    public  static  Connection  getNestedConnection()   throws SQLException
+    {
+        return DriverManager.getConnection( "jdbc:default:connection" );
+    }
+
+}

Propchange: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/MergeStatementTest.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?rev=1516157&r1=1516156&r2=1516157&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 Wed Aug 21 13:26:06 2013
@@ -240,6 +240,7 @@ public class _Suite extends BaseTestCase
         suite.addTest(Derby6131.suite());
         suite.addTest(NewOptimizerOverridesTest.suite());
         suite.addTest(XMLOptimizerTraceTest.suite());
+        suite.addTest(MergeStatementTest.suite());
         return suite;
 	}
 }