IGNITE-12590: Fix (remove) check KEY at the MERGE command (#7321)
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index a029cba..bcb3316 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -66,6 +66,14 @@
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-indexing</artifactId>
             <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-indexing</artifactId>
+            <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
 
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index 58af83d..662d971 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -77,6 +77,7 @@
 import org.apache.ignite.jdbc.thin.JdbcThinResultSetSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinSchemaCaseSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinSelectAfterAlterTable;
+import org.apache.ignite.jdbc.thin.JdbcThinSqlMergeTest;
 import org.apache.ignite.jdbc.thin.JdbcThinStatementCancelSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinStatementSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinStatementTimeoutSelfTest;
@@ -228,6 +229,7 @@
 
     JdbcThinPreparedStatementLeakTest.class,
     JdbcThinTransactionsLeaksMvccTest.class,
+    JdbcThinSqlMergeTest.class,
 })
 public class IgniteJdbcDriverTestSuite {
     /**
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSqlMergeTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSqlMergeTest.java
new file mode 100644
index 0000000..83774e3
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSqlMergeTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.ignite.jdbc.thin;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.internal.processors.query.SqlMergeTest;
+
+/**
+ * Tests SQL MERGE command via JDBC thin driver.
+ */
+public class JdbcThinSqlMergeTest extends SqlMergeTest {
+    /** JDBC thin URL. */
+    private static final String JDBC_THIN_URL = "jdbc:ignite:thin://127.0.0.1";
+
+    /** Connection. */
+    private static Connection conn;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        conn = DriverManager.getConnection(JDBC_THIN_URL);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        conn.close();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected List<List<?>> sql(String sql) throws Exception {
+        try (Statement stmt = conn.createStatement()) {
+            stmt.execute(sql);
+
+            int updCnt = stmt.getUpdateCount();
+
+            if (updCnt == -1) {
+                ResultSet rs = stmt.getResultSet();
+
+                int cols = rs.getMetaData().getColumnCount();
+
+                List<List<?>> res = new ArrayList<>();
+
+                while (rs.next()) {
+                    List<Object> row = new ArrayList<>();
+
+                    for (int i = 0; i < cols; ++i)
+                        row.add(rs.getObject(i + 1));
+
+                    res.add(row);
+                }
+
+                return res;
+            }
+            else
+                return Collections.singletonList(Collections.singletonList((long)updCnt));
+        }
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
index 8e29864..22b4114 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
@@ -418,7 +418,7 @@
 
             // Do actual parsing.
             if (CommandProcessor.isCommand(prepared)) {
-                GridSqlStatement cmdH2 = new GridSqlQueryParser(false).parse(prepared);
+                GridSqlStatement cmdH2 = new GridSqlQueryParser(false, log).parse(prepared);
 
                 QueryParserResultCommand cmd = new QueryParserResultCommand(null, cmdH2, false);
 
@@ -464,7 +464,7 @@
             }
 
             // Parse SELECT.
-            GridSqlQueryParser parser = new GridSqlQueryParser(false);
+            GridSqlQueryParser parser = new GridSqlQueryParser(false, log);
 
             GridSqlQuery selectStmt = (GridSqlQuery)parser.parse(prepared);
 
@@ -543,7 +543,8 @@
                         newQry.isEnforceJoinOrder(),
                         locSplit,
                         idx,
-                        paramsCnt
+                        paramsCnt,
+                        log
                     );
                 }
             }
@@ -560,7 +561,8 @@
                     newQry.isEnforceJoinOrder(),
                     locSplit,
                     idx,
-                    paramsCnt
+                    paramsCnt,
+                    log
                 );
             }
 
@@ -665,7 +667,7 @@
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
 
         // Prepare AST.
-        GridSqlQueryParser parser = new GridSqlQueryParser(false);
+        GridSqlQueryParser parser = new GridSqlQueryParser(false, log);
 
         GridSqlStatement stmt = parser.parse(prepared);
 
@@ -710,7 +712,8 @@
                 planKey,
                 stmt,
                 mvccEnabled,
-                idx
+                idx,
+                log
             );
         }
         catch (Exception e) {
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index 1d07e89..707a58a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -28,6 +28,7 @@
 import java.util.List;
 import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
@@ -103,12 +104,13 @@
         QueryDescriptor planKey,
         GridSqlStatement stmt,
         boolean mvccEnabled,
-        IgniteH2Indexing idx
+        IgniteH2Indexing idx,
+        IgniteLogger log
     ) throws IgniteCheckedException {
         if (stmt instanceof GridSqlMerge || stmt instanceof GridSqlInsert)
-            return planForInsert(planKey, stmt, idx, mvccEnabled);
+            return planForInsert(planKey, stmt, idx, mvccEnabled, log);
         else if (stmt instanceof GridSqlUpdate || stmt instanceof GridSqlDelete)
-            return planForUpdate(planKey, stmt, idx, mvccEnabled);
+            return planForUpdate(planKey, stmt, idx, mvccEnabled, log);
         else
             throw new IgniteSQLException("Unsupported operation: " + stmt.getSQL(),
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
@@ -129,7 +131,8 @@
         QueryDescriptor planKey,
         GridSqlStatement stmt,
         IgniteH2Indexing idx,
-        boolean mvccEnabled
+        boolean mvccEnabled,
+        IgniteLogger log
     ) throws IgniteCheckedException {
         GridSqlQuery sel = null;
 
@@ -147,8 +150,13 @@
 
         List<GridSqlElement[]> elRows = null;
 
+        UpdateMode mode;
+
         if (stmt instanceof GridSqlInsert) {
+            mode = UpdateMode.INSERT;
+
             GridSqlInsert ins = (GridSqlInsert) stmt;
+
             target = ins.into();
 
             tbl = DmlAstUtils.gridTableForElement(target);
@@ -170,6 +178,8 @@
             rowsNum = isTwoStepSubqry ? 0 : ins.rows().size();
         }
         else if (stmt instanceof GridSqlMerge) {
+            mode = UpdateMode.MERGE;
+
             GridSqlMerge merge = (GridSqlMerge) stmt;
 
             target = merge.into();
@@ -260,12 +270,11 @@
                 mvccEnabled,
                 planKey,
                 selectSql,
-                tbl.dataTable().cacheName()
+                tbl.dataTable().cacheName(),
+                log
             );
         }
 
-        UpdateMode mode = stmt instanceof GridSqlMerge ? UpdateMode.MERGE : UpdateMode.INSERT;
-
         List<List<DmlArgument>> rows = null;
 
         if (elRows != null) {
@@ -343,7 +352,8 @@
         QueryDescriptor planKey,
         GridSqlStatement stmt,
         IgniteH2Indexing idx,
-        boolean mvccEnabled
+        boolean mvccEnabled,
+        IgniteLogger log
     ) throws IgniteCheckedException {
         GridSqlElement target;
 
@@ -441,7 +451,8 @@
                         mvccEnabled,
                         planKey,
                         selectSql,
-                        tbl.dataTable().cacheName()
+                        tbl.dataTable().cacheName(),
+                        log
                     );
                 }
 
@@ -475,7 +486,8 @@
                         mvccEnabled,
                         planKey,
                         selectSql,
-                        tbl.dataTable().cacheName()
+                        tbl.dataTable().cacheName(),
+                        log
                     );
                 }
 
@@ -883,7 +895,8 @@
         boolean mvccEnabled,
         QueryDescriptor planKey,
         String selectQry,
-        String cacheName
+        String cacheName,
+        IgniteLogger log
     )
         throws IgniteCheckedException {
         if ((!mvccEnabled && !planKey.skipReducerOnUpdate()) || planKey.batched())
@@ -893,7 +906,7 @@
             // Get a new prepared statement for derived select query.
             try (PreparedStatement stmt = conn.prepareStatement(selectQry)) {
                 Prepared prep = GridSqlQueryParser.prepared(stmt);
-                GridSqlQuery selectStmt = (GridSqlQuery)new GridSqlQueryParser(false).parse(prep);
+                GridSqlQuery selectStmt = (GridSqlQuery)new GridSqlQueryParser(false, log).parse(prep);
 
                 GridCacheTwoStepQuery qry = GridSqlQuerySplitter.split(
                     conn,
@@ -904,7 +917,8 @@
                     planKey.enforceJoinOrder(),
                     false,
                     idx,
-                    prep.getParameters().size()
+                    prep.getParameters().size(),
+                    log
                 );
 
                 boolean distributed =
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlMerge.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlMerge.java
index f9b084b..5b348bc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlMerge.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlMerge.java
@@ -29,9 +29,6 @@
     private GridSqlColumn[] cols;
 
     /** */
-    private GridSqlColumn[] keys;
-
-    /** */
     private List<GridSqlElement[]> rows;
 
     /** Insert subquery. */
@@ -51,16 +48,6 @@
         }
         buff.append("\n)\n");
 
-        if (keys != null) {
-            buff.append("KEY(\n");
-            buff.resetCount();
-            for (GridSqlColumn c : keys) {
-                buff.appendExceptFirst(",\n");
-                buff.append(c.getSQL());
-            }
-            buff.append(")\n");
-        }
-
         if (!rows.isEmpty()) {
             buff.append("VALUES\n");
             StatementBuilder valuesBuff = new StatementBuilder();
@@ -128,15 +115,4 @@
         this.cols = cols;
         return this;
     }
-
-    /** */
-    public GridSqlColumn[] keys() {
-        return keys;
-    }
-
-    /** */
-    public GridSqlMerge keys(GridSqlColumn[] keys) {
-        this.keys = keys;
-        return this;
-    }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index 573df3c..c605df8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -28,8 +28,10 @@
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.QueryIndex;
@@ -38,8 +40,6 @@
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryUtils;
-import org.apache.ignite.internal.processors.query.h2.dml.DmlAstUtils;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.util.typedef.F;
 import org.h2.command.Command;
@@ -527,6 +527,9 @@
     /** */
     private final Map<String, Integer> optimizedTableFilterOrder;
 
+    /** */
+    private final IgniteLogger log;
+
     /**
      * We have a counter instead of a simple flag, because
      * a flag can be reset earlier than needed in case of
@@ -540,9 +543,14 @@
     /**
      * @param useOptimizedSubqry If we have to find correct order for table filters in FROM clause.
      *                           Relies on uniqueness of table filter aliases.
+     * @param log Logger.
      */
-    public GridSqlQueryParser(boolean useOptimizedSubqry) {
-        optimizedTableFilterOrder = useOptimizedSubqry ? new HashMap<String, Integer>() : null;
+    public GridSqlQueryParser(boolean useOptimizedSubqry, IgniteLogger log) {
+        assert Objects.nonNull(log);
+
+        optimizedTableFilterOrder = useOptimizedSubqry ? new HashMap<>() : null;
+
+        this.log = log;
     }
 
     /**
@@ -856,29 +864,11 @@
 
         res.columns(cols);
 
-        Column[] srcKeys = MERGE_KEYS.get(merge);
-
-        GridH2Table intoTbl = DmlAstUtils.gridTableForElement(tbl).dataTable();
-
-        GridH2RowDescriptor rowDesc = intoTbl.rowDescriptor();
-
-        GridSqlColumn[] keys = new GridSqlColumn[srcKeys.length];
-
-        for (int i = 0; i < srcKeys.length; i++) {
-            String colName = srcKeys[i].getName();
-
-            int colId = intoTbl.getColumn(colName).getColumnId();
-
-            if (!rowDesc.isKeyColumn(colId) && !F.eq(colName, rowDesc.type().affinityKey())) {
-                throw new IgniteSQLException("Invalid column name in KEYS clause of MERGE - it may include only " +
-                    "key and/or affinity columns: " + colName, IgniteQueryErrorCode.PARSING);
-            }
-
-            keys[i] = new GridSqlColumn(srcKeys[i], tbl, null, null, colName);
+        if (!F.isEmpty(MERGE_KEYS.get(merge))) {
+            log.warning("The search row by explicit KEY isn't supported. The primary key is always used to search row " +
+                "[sql=" + merge.getSQL() + ']');
         }
 
-        res.keys(keys);
-
         List<Expression[]> srcRows = MERGE_ROWS.get(merge);
         if (!srcRows.isEmpty()) {
             List<GridSqlElement[]> rows = new ArrayList<>(srcRows.size());
@@ -889,7 +879,7 @@
                     row[i] = parseExpression(srcRow[i], false);
 
                     if (row[i] == null) {
-                        throw new IgniteSQLException("DEFAULT values are unsupported for MERGE.",
+                        throw new IgniteSQLException("Explicit DEFAULT values are unsupported for MERGE.",
                             IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
                     }
                 }
@@ -899,7 +889,7 @@
             res.rows(rows);
         }
         else {
-            res.rows(Collections.<GridSqlElement[]>emptyList());
+            res.rows(Collections.emptyList());
             res.query(parseQuery(MERGE_QUERY.get(merge)));
         }
 
@@ -947,7 +937,7 @@
                     row[i] = parseExpression(srcRow[i], false);
 
                     if (row[i] == null) {
-                        throw new IgniteSQLException("DEFAULT values are unsupported for MERGE.",
+                        throw new IgniteSQLException("Explicit DEFAULT values are unsupported for INSERT.",
                             IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
                     }
                 }
@@ -1929,8 +1919,8 @@
      * @param useOptimizedSubqry Whether to user optimized subquery.
      * @return Parsed query.
      */
-    public static GridSqlQuery parseQuery(Prepared prepared, boolean useOptimizedSubqry) {
-        return (GridSqlQuery)new GridSqlQueryParser(useOptimizedSubqry).parse(prepared);
+    public static GridSqlQuery parseQuery(Prepared prepared, boolean useOptimizedSubqry, IgniteLogger log) {
+        return (GridSqlQuery)new GridSqlQueryParser(useOptimizedSubqry, log).parse(prepared);
     }
 
     /**
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 04968ea..dc5786e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -33,6 +33,7 @@
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
@@ -201,7 +202,8 @@
         boolean enforceJoinOrder,
         boolean locSplit,
         IgniteH2Indexing idx,
-        int paramsCnt
+        int paramsCnt,
+        IgniteLogger log
     ) throws SQLException, IgniteCheckedException {
         SplitterContext.set(distributedJoins);
 
@@ -215,7 +217,8 @@
                 enforceJoinOrder,
                 locSplit,
                 idx,
-                paramsCnt
+                paramsCnt,
+                log
             );
         }
         finally {
@@ -246,7 +249,8 @@
         boolean enforceJoinOrder,
         boolean locSplit,
         IgniteH2Indexing idx,
-        int paramsCnt
+        int paramsCnt,
+        IgniteLogger log
     ) throws SQLException, IgniteCheckedException {
         final boolean explain = qry.explain();
 
@@ -267,7 +271,9 @@
         // Here we will have correct normalized AST with optimized join order.
         // The distributedJoins parameter is ignored because it is not relevant for
         // the REDUCE query optimization.
-        qry = GridSqlQueryParser.parseQuery(prepare(conn, qry.getSQL(), false, enforceJoinOrder), true);
+        qry = GridSqlQueryParser.parseQuery(
+            prepare(conn, qry.getSQL(), false, enforceJoinOrder),
+            true, log);
 
         // Do the actual query split. We will update the original query AST, need to be careful.
         splitter.splitQuery(qry);
@@ -286,7 +292,7 @@
 
                 allCollocated &= isCollocated((Query)prepared0);
 
-                mapSqlQry.query(GridSqlQueryParser.parseQuery(prepared0, true).getSQL());
+                mapSqlQry.query(GridSqlQueryParser.parseQuery(prepared0, true, log).getSQL());
             }
 
             // We do not need distributed joins if all MAP queries are collocated.
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQueryFailingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQueryFailingTest.java
new file mode 100644
index 0000000..f7e8a0e
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQueryFailingTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.ignite.internal.processors.cache;
+
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class IgniteCacheMergeSqlQueryFailingTest extends IgniteCacheAbstractInsertSqlQuerySelfTest {
+    /**
+     *
+     */
+    @Test
+    public void testSqlMergeIntoQuery() throws Exception {
+        ignite(0).cache("S2P").query(new SqlFieldsQuery("CREATE TABLE " +
+                "USERPUBSTATICDATA (BOOK VARCHAR, DESK VARCHAR, TRADERS VARCHAR, REGION VARCHAR, LOB VARCHAR, " +
+                "EXCLUDE VARCHAR, TRANSIT VARCHAR, MAPBOOKTOTHISBOOK VARCHAR, " +
+                "CONSTRAINT USERPUBSTATICDATA_PK PRIMARY KEY (BOOK,DESK)) " +
+                "WITH \"template=replicated\"").setSchema("PUBLIC"));
+
+        Ignite ignite = startClientGrid(3);
+
+        IgniteCache srvCache = ignite(0).cache("S2P").withKeepBinary();
+
+        IgniteCache clientCache = ignite.cache("S2P").withKeepBinary();
+
+        srvCache.query(new SqlFieldsQuery(
+            "MERGE INTO USERPUBSTATICDATA(BOOK, DESK, TRADERS, REGION, LOB, EXCLUDE, TRANSIT, MAPBOOKTOTHISBOOK) " +
+                "VALUES('CADOIS', 'FRT TOR', 'Robin Das/Dave Carlson', 'Toronto', 'FRT', null, null, 'CADOIS');").setSchema("PUBLIC"));
+
+        srvCache.query(new SqlFieldsQuery(
+            "MERGE INTO USERPUBSTATICDATA(BOOK, DESK, TRADERS, REGION, LOB, EXCLUDE, TRANSIT, MAPBOOKTOTHISBOOK) " +
+                "VALUES('CADOIS', 'FRT TOR 1', 'Robin Das/Dave Carlson 1', 'Toronto', 'FRT', null, null, 'CADOIS');").setSchema("PUBLIC"));
+
+        clientCache.query(new SqlFieldsQuery(
+            "MERGE INTO USERPUBSTATICDATA(BOOK, DESK, TRADERS, REGION, LOB, EXCLUDE, TRANSIT, MAPBOOKTOTHISBOOK) " +
+                "VALUES('CADOIS', 'FRT TOR', 'Robin Das/Dave Carlson 2', 'Toronto', 'FRT', null, null, 'CADOIS');").setSchema("PUBLIC"));
+
+        clientCache.query(new SqlFieldsQuery(
+            "MERGE INTO USERPUBSTATICDATA(BOOK, TRADERS, REGION, LOB, EXCLUDE, TRANSIT, MAPBOOKTOTHISBOOK) " +
+                "VALUES('CADOIS', 'Robin Das/Dave Carlson 2', 'Toronto', 'FRT', null, null, 'CADOIS');").setSchema("PUBLIC"));
+
+        List<List<?>> res = clientCache.query(
+            new SqlFieldsQuery("SELECT TRADERS FROM USERPUBSTATICDATA WHERE BOOK='CADOIS'").setSchema("PUBLIC")).getAll();
+
+        assertEquals("Robin Das/Dave Carlson 2", res.get(0).get(0));
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlMergeOnClientNodeTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlMergeOnClientNodeTest.java
new file mode 100644
index 0000000..debb37d
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlMergeOnClientNodeTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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.ignite.internal.processors.query;
+
+import org.apache.ignite.internal.IgniteEx;
+
+/**
+ * Tests for SQL MERGE on client node.
+ */
+public class SqlMergeOnClientNodeTest extends SqlMergeTest {
+    /** Node. */
+    protected static IgniteEx cli;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        cli = startClientGrid(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        node = cli;
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlMergeTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlMergeTest.java
new file mode 100644
index 0000000..fe1e00f
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlMergeTest.java
@@ -0,0 +1,201 @@
+/*
+ * 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.ignite.internal.processors.query;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.index.AbstractIndexingCommonTest;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.junit.Test;
+
+/**
+ * Tests for SQL MERGE.
+ */
+public class SqlMergeTest extends AbstractIndexingCommonTest {
+    /** Node. */
+    private static IgniteEx srv;
+
+    /** Node. */
+    protected IgniteEx node;
+
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        srv = startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        for (String cache : srv.cacheNames())
+            srv.cache(cache).destroy();
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        node = srv;
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void test() throws Exception {
+        sql("CREATE TABLE test1 (id INT, id2 INT, name VARCHAR, PRIMARY KEY (id, id2))");
+
+        checkMergeQuery("MERGE INTO test1 (id, id2, name) VALUES (1, 2, 'Kyle')", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test1 WHERE id = 1",
+            Arrays.asList(1, 2, "Kyle"));
+
+        checkMergeQuery("MERGE INTO test1 (id2, id, name) VALUES (3, 2, 'Santa Claus')", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test1 WHERE id = 2",
+            Arrays.asList(2, 3, "Santa Claus"));
+
+        checkMergeQuery("MERGE INTO test1 (name, id, id2) VALUES ('Holy Jesus', 1, 2), ('Kartman', 3, 4)", 2L);
+        checkSqlResults("SELECT id, id2, name FROM test1 WHERE id = 1",
+            Arrays.asList(1, 2, "Holy Jesus"));
+        checkSqlResults("SELECT id, id2, name FROM test1 WHERE id = 3",
+            Arrays.asList(3, 4, "Kartman"));
+
+        checkMergeQuery("MERGE INTO test1 (id, id2, name) " +
+            "SELECT id, id2 * 1000, UPPER(name) FROM test1 WHERE id < 2", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test1 WHERE id = 1 AND id2 = 2000",
+            Arrays.asList(1, 2000, "HOLY JESUS"));
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testCheckKeysWarning() throws Exception {
+        LogListener logLsnr = LogListener
+            .matches("The search row by explicit KEY isn't supported. The primary key is always used to search row")
+            .build();
+
+        ListeningTestLogger listeningTestLogger = testLog();
+
+        listeningTestLogger.registerListener(logLsnr);
+
+        sql("CREATE TABLE test2 (id INT, id2 INT, name VARCHAR, PRIMARY KEY (id, id2))");
+
+        checkMergeQuery("MERGE INTO test2 (id, id2, name) KEY(_key) VALUES (100, 1, 'Bob')", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test2 WHERE id = 100",
+            Arrays.asList(100, 1, "Bob"));
+        assertTrue(logLsnr.check());
+        logLsnr.reset();
+
+        checkMergeQuery("MERGE INTO test2 (id2, id, name) KEY(_key) VALUES (2, 100, 'Alice')", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test2 WHERE id = 100 AND id2 = 2",
+            Arrays.asList(100, 2, "Alice"));
+        assertTrue(logLsnr.check());
+        logLsnr.reset();
+
+        checkMergeQuery("MERGE INTO test2 (id, id2, name) KEY(id, id2) VALUES (3, 5, 'Stan')", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test2 WHERE id = 3",
+            Arrays.asList(3, 5, "Stan"));
+        assertTrue(logLsnr.check());
+        logLsnr.reset();
+
+        checkMergeQuery("MERGE INTO test2 (id, id2, name) KEY(id2, id) VALUES (1, 100, 'Satan')", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test2 WHERE id = 1",
+            Arrays.asList(1, 100, "Satan"));
+        assertTrue(logLsnr.check());
+        logLsnr.reset();
+
+        checkMergeQuery("MERGE INTO test2 (id2, id, name) KEY(id) VALUES (15, 32, 'Kyle')", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test2 WHERE id = 32",
+            Arrays.asList(32, 15, "Kyle"));
+        assertTrue(logLsnr.check());
+        logLsnr.reset();
+
+        checkMergeQuery("MERGE INTO test2 (name, id, id2) KEY(id2) VALUES ('Morlock', 13, 12)", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test2 WHERE id = 13",
+            Arrays.asList(13, 12, "Morlock"));
+        assertTrue(logLsnr.check());
+        logLsnr.reset();
+
+        checkMergeQuery("MERGE INTO test2 (id, name, id2) KEY(_key, id) VALUES (10, 'Warlock', 52)", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test2 WHERE id = 10",
+            Arrays.asList(10, 52, "Warlock"));
+        assertTrue(logLsnr.check());
+        logLsnr.reset();
+
+        checkMergeQuery("MERGE INTO test2 (id, id2, name) KEY(name) VALUES (10, -11, 'Sherlock')", 1L);
+        checkSqlResults("SELECT id, id2, name FROM test2 WHERE id = 10 and id2=-11",
+            Arrays.asList(10, -11, "Sherlock"));
+        assertTrue(logLsnr.check());
+        logLsnr.reset();
+
+    }
+
+    /**
+     * @param sql MERGE query.
+     */
+    private void checkMergeQuery(String sql, long expectedUpdateCounts) throws Exception {
+        List<List<?>> resMrg = sql(sql);
+
+        assertEquals(1, resMrg.size());
+        assertEquals(1, resMrg.get(0).size());
+        assertEquals(expectedUpdateCounts, resMrg.get(0).get(0));
+    }
+
+    /**
+     * @param sql SELECT query to check merge result.
+     * @param expectedRow Expected results of the SELECT.
+     */
+    private void checkSqlResults(String sql, List<?> expectedRow) throws Exception {
+        List<List<?>> res = sql(sql);
+        assertEquals(1, res.size());
+        assertEquals(expectedRow, res.get(0));
+    }
+
+    /**
+     * @param sql SQL query.
+     * @return Results.
+     */
+    protected List<List<?>> sql(String sql) throws Exception {
+        GridQueryProcessor qryProc = node.context().query();
+
+        SqlFieldsQuery qry = new SqlFieldsQuery(sql).setSchema("PUBLIC");
+
+        return qryProc.querySqlFields(qry, true).getAll();
+    }
+
+    /**
+     * Setup and return test log.
+     *
+     * @return Test logger.
+     */
+    private ListeningTestLogger testLog() {
+        ListeningTestLogger testLog = new ListeningTestLogger(false, log);
+
+        GridTestUtils.setFieldValue(((IgniteH2Indexing)node.context().query().getIndexing()).parser(), "log", testLog);
+
+        return testLog;
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index e88d04a..0e73d7b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -342,7 +342,7 @@
         Prepared prepared = parse("select Person.old, p1.old, p1.addrId from Person, Person p1 " +
             "where exists(select 1 from sch2.Address a where a.id = p1.addrId)");
 
-        GridSqlSelect select = (GridSqlSelect)new GridSqlQueryParser(false).parse(prepared);
+        GridSqlSelect select = (GridSqlSelect)new GridSqlQueryParser(false, log).parse(prepared);
 
         GridSqlJoin join = (GridSqlJoin)select.from();
 
@@ -382,53 +382,6 @@
 
     /** */
     @Test
-    public void testParseMerge() throws Exception {
-        /* Plain rows w/functions, operators, defaults, and placeholders. */
-        checkQuery("merge into Person(old, name) values(5, 'John')");
-        checkQuery("merge into Person(name) values(null)");
-        checkQuery("merge into Person(name) values(null), (null)");
-        checkQuery("merge into Person(name, parentName) values(null, null), (?, ?)");
-        checkQuery("merge into Person(old, name) values(5, 'John',), (6, 'Jack')");
-        checkQuery("merge into Person(old, name) values(5 * 3, null,)");
-        checkQuery("merge into Person(old, name) values(ABS(-8), 'Max')");
-        checkQuery("merge into Person(old, name) values(5, 'Jane'), (null, null), (6, 'Jill')");
-        checkQuery("merge into Person(old, name, parentName) values(8 * 7, null, 'Unknown')");
-        checkQuery("merge into Person(old, name, parentName) values" +
-            "(2016 - 1828, CONCAT('Leo', 'Tolstoy'), CONCAT(?, 'Tolstoy'))," +
-            "(?, 'AlexanderPushkin', null)," +
-            "(ABS(1821 - 2016), CONCAT('Fyodor', null, UPPER(CONCAT(SQRT(?), 'dostoevsky'))), null)");
-        checkQuery("merge into Person(date, old, name, parentName, addrId) values " +
-            "('20160112', 1233, 'Ivan Ivanov', 'Peter Ivanov', 123)");
-        checkQuery("merge into Person(date, old, name, parentName, addrId) values " +
-            "(CURRENT_DATE(), RAND(), ASCII('Hi'), INSERT('Leo Tolstoy', 4, 4, 'Max'), ASCII('HI'))");
-        checkQuery("merge into Person(date, old, name, parentName, addrId) values " +
-            "(TRUNCATE(TIMESTAMP '2015-12-31 23:59:59'), POWER(3,12), NULL, NULL, NULL)");
-        checkQuery("merge into Person(old, name) select ASCII(parentName), INSERT(parentName, 4, 4, 'Max') from " +
-            "Person where date='2011-03-12'");
-
-        /* KEY clause. */
-        checkQuery("merge into Person(_key, old, name) key(_key) values('a', 5, 'John')");
-        checkQuery("merge into SCH3.Person(id, old, name) key(id) values(1, 5, 'John')");
-        checkQuery("merge into SCH3.Person(_key, old, name) key(_key) values(?, 5, 'John')");
-        checkQuery("merge into SCH3.Person(_key, id, old, name) key(_key, id) values(?, ?, 5, 'John')");
-        assertParseThrows("merge into Person(old, name) key(name) values(5, 'John')", IgniteSQLException.class,
-            "Invalid column name in KEYS clause of MERGE - it may include only key and/or affinity columns: NAME");
-        assertParseThrows("merge into SCH3.Person(id, stuff, old, name) key(stuff) values(1, 'x', 5, 'John')",
-            IgniteSQLException.class, "Invalid column name in KEYS clause of MERGE - it may include only key and/or " +
-                "affinity columns: STUFF");
-
-        /* Subqueries. */
-        checkQuery("merge into Person(old, name) select old, parentName from Person");
-        checkQuery("merge into Person(old, name) select old, parentName from Person where old > 5");
-        checkQuery("merge into Person(old, name) select 5, 'John'");
-        checkQuery("merge into Person(old, name) select p1.old, 'Name' from person p1 join person p2 on " +
-            "p2.name = p1.parentName where p2.old > 30");
-        checkQuery("merge into Person(old) select 5 from Person UNION select street from sch2.Address limit ? " +
-            "offset ?");
-    }
-
-    /** */
-    @Test
     public void testParseInsert() throws Exception {
         /* Plain rows w/functions, operators, defaults, and placeholders. */
         checkQuery("insert into Person(old, name) values(5, 'John')");
@@ -727,7 +680,7 @@
             @Override public Object call() throws Exception {
                 Prepared p = parse(sql);
 
-                return new GridSqlQueryParser(false).parse(p);
+                return new GridSqlQueryParser(false, log).parse(p);
             }
         }, exCls, msg);
     }
@@ -738,7 +691,7 @@
     private void assertCreateIndexEquals(GridSqlCreateIndex exp, String sql) throws Exception {
         Prepared prepared = parse(sql);
 
-        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+        GridSqlStatement stmt = new GridSqlQueryParser(false, log).parse(prepared);
 
         assertTrue(stmt instanceof GridSqlCreateIndex);
 
@@ -751,7 +704,7 @@
     private void assertDropIndexEquals(GridSqlDropIndex exp, String sql) throws Exception {
         Prepared prepared = parse(sql);
 
-        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+        GridSqlStatement stmt = new GridSqlQueryParser(false, log).parse(prepared);
 
         assertTrue(stmt instanceof GridSqlDropIndex);
 
@@ -786,7 +739,7 @@
     private void assertCreateTableEquals(GridSqlCreateTable exp, String sql) throws Exception {
         Prepared prepared = parse(sql);
 
-        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+        GridSqlStatement stmt = new GridSqlQueryParser(false, log).parse(prepared);
 
         assertTrue(stmt instanceof GridSqlCreateTable);
 
@@ -848,7 +801,7 @@
     private void assertAlterTableAddColumnEquals(GridSqlAlterTableAddColumn exp, String sql) throws Exception {
         Prepared prepared = parse(sql);
 
-        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+        GridSqlStatement stmt = new GridSqlQueryParser(false, log).parse(prepared);
 
         assertTrue(stmt instanceof GridSqlAlterTableAddColumn);
 
@@ -924,7 +877,7 @@
     private void assertDropTableEquals(GridSqlDropTable exp, String sql) throws Exception {
         Prepared prepared = parse(sql);
 
-        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+        GridSqlStatement stmt = new GridSqlQueryParser(false, log).parse(prepared);
 
         assertTrue(stmt instanceof GridSqlDropTable);
 
@@ -1115,7 +1068,7 @@
     private void checkQuery(String qry) throws Exception {
         Prepared prepared = parse(qry);
 
-        GridSqlStatement gQry = new GridSqlQueryParser(false).parse(prepared);
+        GridSqlStatement gQry = new GridSqlQueryParser(false, log).parse(prepared);
 
         String res = gQry.getSQL();
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index 5bda57e..b752574 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -216,6 +216,8 @@
 import org.apache.ignite.internal.processors.query.RunningQueriesTest;
 import org.apache.ignite.internal.processors.query.SqlIllegalSchemaSelfTest;
 import org.apache.ignite.internal.processors.query.SqlIncompatibleDataTypeExceptionTest;
+import org.apache.ignite.internal.processors.query.SqlMergeOnClientNodeTest;
+import org.apache.ignite.internal.processors.query.SqlMergeTest;
 import org.apache.ignite.internal.processors.query.SqlNestedQuerySelfTest;
 import org.apache.ignite.internal.processors.query.SqlPushDownFunctionTest;
 import org.apache.ignite.internal.processors.query.SqlQueryHistoryFromClientSelfTest;
@@ -600,7 +602,12 @@
     RowCountTableStatisticsSurvivesNodeRestartTest.class,
 
     SqlViewExporterSpiTest.class,
-    SystemViewSelfTest.class
+    SystemViewSelfTest.class,
+
+    IgniteCacheMergeSqlQuerySelfTest.class,
+    SqlMergeTest.class,
+    SqlMergeOnClientNodeTest.class
+
 })
 public class IgniteBinaryCacheQueryTestSuite {
 }