You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/12/05 12:38:05 UTC

[01/52] ignite git commit: IGNITE-2294: Implemented DML.

Repository: ignite
Updated Branches:
  refs/heads/master 6e8c35b42 -> 6d348aeda


http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeQueryBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeQueryBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeQueryBenchmark.java
new file mode 100644
index 0000000..2a466f1
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeQueryBenchmark.java
@@ -0,0 +1,116 @@
+/*
+ * 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.yardstick.cache.dml;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.yardstickframework.BenchmarkUtils.println;
+
+/**
+ * Ignite benchmark that performs SQL MERGE and query operations.
+ */
+public class IgniteSqlMergeQueryBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private AtomicInteger putCnt = new AtomicInteger();
+
+    /** */
+    private AtomicInteger qryCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        if (rnd.nextBoolean()) {
+            double salary = rnd.nextDouble() * args.range() * 1000;
+
+            double maxSalary = salary + 1000;
+
+            Collection<Cache.Entry<Integer, Object>> entries = executeQuery(salary, maxSalary);
+
+            for (Cache.Entry<Integer, Object> entry : entries) {
+                Object o = entry.getValue();
+
+                double s = o instanceof Person ? ((Person) o).getSalary() : ((BinaryObject) o).<Double>field("salary");
+
+                if (s < salary || s > maxSalary)
+                    throw new Exception("Invalid person retrieved [min=" + salary + ", max=" + maxSalary +
+                            ", person=" + o + ']');
+            }
+
+            qryCnt.getAndIncrement();
+        }
+        else {
+            int i = rnd.nextInt(args.range());
+
+            cache.query(new SqlFieldsQuery("merge into Person(_key, id, firstName, lastName, salary) " +
+                "values (?, ?, ?, ?, ?)").setArgs(i ,i, "firstName" + i, "lastName" + i, (double) i * 1000));
+
+            putCnt.getAndIncrement();
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onWarmupFinished() {
+        super.onWarmupFinished();
+    }
+
+    /**
+     * @param minSalary Min salary.
+     * @param maxSalary Max salary.
+     * @return Query result.
+     * @throws Exception If failed.
+     */
+    private Collection<Cache.Entry<Integer, Object>> executeQuery(double minSalary, double maxSalary) throws Exception {
+        SqlQuery qry = new SqlQuery(Person.class, "salary >= ? and salary <= ?");
+
+        qry.setArgs(minSalary, maxSalary);
+
+        return cache.query(qry).getAll();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("query").withKeepBinary();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() throws Exception {
+        println(cfg, "Finished sql query put benchmark [putCnt=" + putCnt.get() + ", qryCnt=" + qryCnt.get() + ']');
+
+        super.tearDown();
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlUpdateBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlUpdateBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlUpdateBenchmark.java
new file mode 100644
index 0000000..17eb1d2
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlUpdateBenchmark.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.yardstick.cache.dml;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person1;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ * Ignite benchmark that performs SQL UPDATE operations.
+ */
+public class IgniteSqlUpdateBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** {@inheritDoc} */
+    @Override public void setUp(final BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        final AtomicInteger i = new AtomicInteger();
+
+        Collection<Thread> setupThreads = new ArrayList<>(cfg.threads());
+
+        for (int j = 0; j < cfg.threads(); j++) {
+            Thread t = new Thread() {
+                /** {@inheritDoc} */
+                @Override public void run() {
+                    int k;
+
+                    while ((k = i.getAndIncrement()) < args.range()) {
+                        cache().put(k, new Person1(k));
+                        if (++k % 100000 == 0)
+                            BenchmarkUtils.println(cfg, "UPDATE setUp: have successfully put " + k + " items");
+                    }
+                }
+            };
+
+            setupThreads.add(t);
+
+            t.start();
+        }
+
+        for (Thread t : setupThreads)
+            t.join();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        cache().query(new SqlFieldsQuery("update Person1 set _val = ? where _key = ?")
+            .setArgs(new Person1(rnd.nextInt(args.range())), rnd.nextInt(args.range())));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlUpdateFilteredBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlUpdateFilteredBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlUpdateFilteredBenchmark.java
new file mode 100644
index 0000000..94a83df
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlUpdateFilteredBenchmark.java
@@ -0,0 +1,88 @@
+/*
+ * 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.yardstick.cache.dml;
+
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.yardstickframework.BenchmarkUtils.println;
+
+/**
+ * Ignite benchmark that performs put and SQL UPDATE operations.
+ */
+public class IgniteSqlUpdateFilteredBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private AtomicInteger putCnt = new AtomicInteger();
+
+    /** */
+    private AtomicInteger updCnt = new AtomicInteger();
+
+    /** */
+    private AtomicInteger updItemsCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        if (rnd.nextBoolean()) {
+            double salary = rnd.nextDouble() * args.range() * 1000;
+
+            double maxSalary = salary + 1000;
+
+            int res = (Integer) cache().query(new SqlFieldsQuery("update Person set salary = (salary - ?1 + ?2) / 2 " +
+                "where salary >= ?1 and salary <= ?2").setArgs(salary, maxSalary)).getAll().get(0).get(0);
+
+            updItemsCnt.getAndAdd(res);
+
+            updCnt.getAndIncrement();
+        }
+        else {
+            int i = rnd.nextInt(args.range());
+
+            cache.put(i, new Person(i, "firstName" + i, "lastName" + i, i * 1000));
+
+            putCnt.getAndIncrement();
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("query");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() throws Exception {
+        println(cfg, "Finished SQL UPDATE query benchmark [putCnt=" + putCnt.get() + ", updCnt=" + updCnt.get() +
+            ", updItemsCnt=" + updItemsCnt.get() + ']');
+
+        super.tearDown();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/SampleValue.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/SampleValue.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/SampleValue.java
index 20be370..19167f8 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/SampleValue.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/SampleValue.java
@@ -25,12 +25,14 @@ import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryReader;
 import org.apache.ignite.binary.BinaryWriter;
 import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
 
 /**
  * Entity class for benchmark.
  */
 public class SampleValue implements Externalizable, Binarylizable {
     /** */
+    @QuerySqlField
     private int id;
 
     /** */


[14/52] ignite git commit: Merge branch 'ignite-1.7.4' into 'ignite-1.8'.

Posted by vo...@apache.org.
Merge branch 'ignite-1.7.4' into 'ignite-1.8'.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d2b915e5
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d2b915e5
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d2b915e5

Branch: refs/heads/master
Commit: d2b915e50c8e3cbe07f565f14d14474342662550
Parents: e43ea5e 7d88c5b
Author: Andrey Novikov <an...@gridgain.com>
Authored: Thu Nov 24 18:01:31 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Thu Nov 24 18:01:31 2016 +0700

----------------------------------------------------------------------
 modules/web-console/docker/compose/backend/build.sh  | 2 +-
 modules/web-console/docker/compose/frontend/build.sh | 4 ++--
 modules/web-console/docker/standalone/build.sh       | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[26/52] ignite git commit: IGNITE-4304: Improved logging of deferred TX ack message in case remote node left the grid.

Posted by vo...@apache.org.
IGNITE-4304: Improved logging of deferred TX ack message in case remote node left the grid.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/5b2eacfc
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5b2eacfc
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5b2eacfc

Branch: refs/heads/master
Commit: 5b2eacfc25fcffad6f438b63e08d9187635f25d3
Parents: 43ec85f
Author: devozerov <vo...@gridgain.com>
Authored: Fri Nov 25 14:09:42 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 14:09:42 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/transactions/IgniteTxManager.java       | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5b2eacfc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index c72d7f7..e2e9868 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -53,7 +53,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvccFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheReturnCompletableWrapper;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cache.GridDeferredAckMessageSender;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheMappedVersion;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxFinishSync;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture;
@@ -282,9 +281,12 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
                 try {
                     cctx.io().send(nodeId, ackReq, GridIoPolicy.SYSTEM_POOL);
                 }
+                catch (ClusterTopologyCheckedException e) {
+                    if (log.isDebugEnabled())
+                        log.debug("Failed to send one phase commit ack to backup node because it left grid: " + nodeId);
+                }
                 catch (IgniteCheckedException e) {
-                    log.error("Failed to send one phase commit ack to backup node [backup=" +
-                        nodeId + ']', e);
+                    log.error("Failed to send one phase commit ack to backup node [backup=" + nodeId + ']', e);
                 }
                 finally {
                     cctx.kernalContext().gateway().readUnlock();


[40/52] ignite git commit: Ignite-4243: Added a Python/PHP examples for Redis. - Fixes #1258.

Posted by vo...@apache.org.
Ignite-4243: Added a Python/PHP examples for Redis. - Fixes #1258.

Signed-off-by: shroman <rs...@yahoo.com>

(cherry picked from commit 9ec0a6f)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/1073b2c1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/1073b2c1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/1073b2c1

Branch: refs/heads/master
Commit: 1073b2c14d68e252fb2970d489eab47f9a31d201
Parents: 99eb4e0
Author: shroman <rs...@yahoo.com>
Authored: Sat Nov 26 09:23:06 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Mon Nov 28 17:56:39 2016 +0700

----------------------------------------------------------------------
 examples/redis/redis-example.php | 82 +++++++++++++++++++++++++++++++++++
 examples/redis/redis-example.py  | 62 ++++++++++++++++++++++++++
 2 files changed, 144 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1073b2c1/examples/redis/redis-example.php
----------------------------------------------------------------------
diff --git a/examples/redis/redis-example.php b/examples/redis/redis-example.php
new file mode 100644
index 0000000..0054f16
--- /dev/null
+++ b/examples/redis/redis-example.php
@@ -0,0 +1,82 @@
+#!/bin/php
+
+<?php
+/*
+ * 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.
+ */
+
+/**
+ * To execute this script, you need to have Predis extension installed and Ignite running.
+ * See https://github.com/nrk/predis for Predis details.
+ *
+ * See https://apacheignite.readme.io/ for more details on Redis integration.
+ */
+
+// Load the library.
+require 'predis/autoload.php';
+Predis\Autoloader::register();
+
+// Connect.
+try {
+    $redis = new Predis\Client(array(
+        "host" => "localhost",
+        "port" => 11211));
+
+    echo ">>> Successfully connected to Redis. \n";
+
+    // Put entry to cache.
+    if ($redis->set('k1', '1'))
+        echo ">>> Successfully put entry in cache. \n";
+
+    // Check entry value.
+    echo(">>> Value for 'k1': " . $redis->get('k1') . "\n");
+
+    // Change entry's value.
+    if ($redis->set('k1', 'new_value'))
+        echo ">>> Successfully put entry in cache. \n";
+
+    // Check entry value.
+    echo(">>> Value for 'k1': " . $redis->get('k1') . "\n");
+
+    // Put entry to cache.
+    if ($redis->set('k2', '2'))
+        echo ">>> Successfully put entry in cache. \n";
+
+    // Check entry value.
+    echo(">>> Value for 'k2': " . $redis->get('k2') . "\n");
+
+    // Get two entries.
+    $val = $redis->mget('k1', 'k2');
+    echo(">>> Value for 'k1' and 'k2': " . var_dump($val) . "\n");
+
+    // Delete on entry.
+    if ($redis->del('k1'))
+        echo ">>> Successfully deleted 'k1'. \n";
+
+    // Db size.
+    echo ">>> Db size: " . $redis->dbsize() . "\n";
+
+    // Increment.
+    echo ">>> Incremented: " . $redis->incr("inc_k") . "\n";
+
+    // Increment by 5.
+    echo ">>> Incremented: " . $redis->incrby("inc_k", 5) . "\n";
+}
+catch (Exception $e) {
+    echo ">>> Couldn't connected to Redis.";
+    echo $e->getMessage();
+}
+?>

http://git-wip-us.apache.org/repos/asf/ignite/blob/1073b2c1/examples/redis/redis-example.py
----------------------------------------------------------------------
diff --git a/examples/redis/redis-example.py b/examples/redis/redis-example.py
new file mode 100644
index 0000000..15b847b
--- /dev/null
+++ b/examples/redis/redis-example.py
@@ -0,0 +1,62 @@
+'''
+ * 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.
+'''
+
+import redis
+'''
+To execute this, you will have redis-py installed and Ignite running.
+See https://github.com/andymccurdy/redis-py for the details on redis-py.
+
+See https://apacheignite.readme.io/ for more details on Redis integration.
+'''
+
+r = redis.StrictRedis(host='localhost', port=11211, db=0)
+
+# set entry.
+r.set('k1', 1)
+
+# check.
+print 'Value for "k1": %s' % r.get('k1')
+
+# change entry's value.
+r.set('k1', 'new_val')
+
+# check.
+print 'Value for "k1": %s' % r.get('k1')
+
+# set another entry.
+r.set('k2', 2)
+
+# check.
+print 'Value for "k2": %s' % r.get('k2')
+
+# get both values.
+print 'Values for "k1" and "k2": %s' % r.mget('k1', 'k2')
+
+# delete one entry.
+r.delete('k1')
+
+# check one entry left.
+print 'Values for "k1" and "k2": %s' % r.mget('k1', 'k2')
+
+# check db size
+print 'Db size: %d' % r.dbsize()
+
+# increment.
+print 'Value for incremented "inc_k" : %s' % r.incr('inc_k')
+
+# increment again.
+print 'Value for incremented "inc_k" : %s' % r.incr('inc_k')


[45/52] ignite git commit: IGNITE-4321 Cassandra modules (useless optional dependency removed)

Posted by vo...@apache.org.
IGNITE-4321 Cassandra modules (useless optional dependency removed)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ca28001f
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ca28001f
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ca28001f

Branch: refs/heads/master
Commit: ca28001f490df29df18d5f875d3d834715093c9f
Parents: d0e0eaa
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Nov 30 12:16:54 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Nov 30 12:16:54 2016 +0300

----------------------------------------------------------------------
 assembly/dependencies-fabric-lgpl.xml | 1 +
 assembly/dependencies-fabric.xml      | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ca28001f/assembly/dependencies-fabric-lgpl.xml
----------------------------------------------------------------------
diff --git a/assembly/dependencies-fabric-lgpl.xml b/assembly/dependencies-fabric-lgpl.xml
index 2b4cf62..82f63d8 100644
--- a/assembly/dependencies-fabric-lgpl.xml
+++ b/assembly/dependencies-fabric-lgpl.xml
@@ -132,6 +132,7 @@
                 <exclude>org.apache.ignite:ignite-apache-license-gen</exclude>
                 <exclude>org.apache.ignite:ignite-appserver-test</exclude>
                 <exclude>org.apache.ignite:ignite-websphere-test</exclude>
+                <exclude>org.apache.ignite:ignite-cassandra</exclude>
             </excludes>
             <sources>
                 <includeModuleDirectory>true</includeModuleDirectory>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ca28001f/assembly/dependencies-fabric.xml
----------------------------------------------------------------------
diff --git a/assembly/dependencies-fabric.xml b/assembly/dependencies-fabric.xml
index ff4075a..63e6ac8 100644
--- a/assembly/dependencies-fabric.xml
+++ b/assembly/dependencies-fabric.xml
@@ -135,6 +135,7 @@
                 <exclude>org.apache.ignite:ignite-geospatial</exclude>
                 <exclude>org.apache.ignite:ignite-appserver-test</exclude>
                 <exclude>org.apache.ignite:ignite-websphere-test</exclude>
+                <exclude>org.apache.ignite:ignite-cassandra</exclude>
             </excludes>
             <sources>
                 <includeModuleDirectory>true</includeModuleDirectory>


[44/52] ignite git commit: IGNITE-4305 marshalling fix

Posted by vo...@apache.org.
IGNITE-4305 marshalling fix

(cherry picked from commit f906dd6)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d0e0eaac
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d0e0eaac
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d0e0eaac

Branch: refs/heads/master
Commit: d0e0eaacce25d76199c20db2ad6ada615160fb4f
Parents: 44bb361
Author: Konstantin Dudkov <kd...@ya.ru>
Authored: Mon Nov 28 12:59:02 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Nov 29 11:49:24 2016 +0300

----------------------------------------------------------------------
 .../dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java       | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d0e0eaac/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
index f18a13c..6582063 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
@@ -203,7 +203,8 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
             addDepInfo = true;
 
         if (entryProcessor != null && entryProcessorBytes == null) {
-            prepareObject(entryProcessor, cctx);
+            if (addDepInfo)
+                prepareObject(entryProcessor, cctx);
 
             entryProcessorBytes = CU.marshal(cctx, entryProcessor);
         }


[15/52] ignite git commit: IGNITE-4288: ODBC: Fixed DML interoperability. This closes #1272.

Posted by vo...@apache.org.
IGNITE-4288: ODBC: Fixed DML interoperability. This closes #1272.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/4a758d1a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4a758d1a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4a758d1a

Branch: refs/heads/master
Commit: 4a758d1a6285f9441af7c894e2a31e0df895cb03
Parents: d2b915e
Author: isapego <ig...@gmail.com>
Authored: Thu Nov 24 15:49:08 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Nov 24 15:49:08 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/odbc/OdbcNioListener.java     |  2 +-
 .../internal/processors/odbc/OdbcRequestHandler.java  | 14 +++++++-------
 .../ignite/internal/processors/odbc/OdbcUtils.java    |  4 ++--
 .../cpp/odbc/include/ignite/odbc/common_types.h       |  1 +
 .../platforms/cpp/odbc/include/ignite/odbc/message.h  |  9 +--------
 .../cpp/odbc/include/ignite/odbc/statement.h          |  4 ++--
 modules/platforms/cpp/odbc/src/connection.cpp         |  4 +++-
 modules/platforms/cpp/odbc/src/statement.cpp          |  4 ++--
 modules/platforms/cpp/odbc/src/utility.cpp            |  6 ------
 9 files changed, 19 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4a758d1a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
index e7baaff..378e5f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
@@ -137,7 +137,7 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
         catch (Exception e) {
             log.error("Failed to process ODBC request [id=" + reqId + ", err=" + e + ']');
 
-            ses.send(parser.encode(new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage())));
+            ses.send(parser.encode(new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString())));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a758d1a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index 588fc66..32fb9db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
@@ -164,7 +164,7 @@ public class OdbcRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to perform handshake [reqId=" + reqId + ", req=" + req + ']', e);
 
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage());
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
         }
     }
 
@@ -220,7 +220,7 @@ public class OdbcRequestHandler {
 
             U.error(log, "Failed to execute SQL query [reqId=" + reqId + ", req=" + req + ']', e);
 
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage());
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
         }
     }
 
@@ -255,7 +255,7 @@ public class OdbcRequestHandler {
 
             U.error(log, "Failed to close SQL query [reqId=" + reqId + ", req=" + req.queryId() + ']', e);
 
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage());
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
         }
     }
 
@@ -295,7 +295,7 @@ public class OdbcRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to fetch SQL query result [reqId=" + reqId + ", req=" + req + ']', e);
 
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage());
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
         }
     }
 
@@ -352,7 +352,7 @@ public class OdbcRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get columns metadata [reqId=" + reqId + ", req=" + req + ']', e);
 
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage());
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
         }
     }
 
@@ -397,7 +397,7 @@ public class OdbcRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get tables metadata [reqId=" + reqId + ", req=" + req + ']', e);
 
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage());
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
         }
     }
 
@@ -429,7 +429,7 @@ public class OdbcRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get params metadata [reqId=" + reqId + ", req=" + req + ']', e);
 
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage());
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a758d1a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
index 300b079..3903562 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
@@ -28,7 +28,7 @@ public class OdbcUtils {
      * @return String surrounded with quotation marks.
      */
     public static String addQuotationMarksIfNeeded(String str) {
-        if (!str.startsWith("\"") && !str.isEmpty())
+        if (str != null && !str.isEmpty() && !(str.startsWith("\"") && str.endsWith("\"")))
             return "\"" + str + "\"";
 
         return str;
@@ -41,7 +41,7 @@ public class OdbcUtils {
      * @return String without leading and trailing quotation marks.
      */
     public static String removeQuotationMarksIfNeeded(String str) {
-        if (str.startsWith("\"") && str.endsWith("\""))
+        if (str != null && str.startsWith("\"") && str.endsWith("\""))
             return str.substring(1, str.length() - 1);
 
         return str;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a758d1a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
index a61be53..517fe4e 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
@@ -30,6 +30,7 @@ namespace ignite
     namespace odbc
     {
         typedef SQLLEN SqlLen;
+        typedef SQLULEN SqlUlen;
 
         /**
          * SQL result.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a758d1a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
index 5c6a933..a2bbd99 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
@@ -461,16 +461,9 @@ namespace ignite
                 status = reader.ReadInt8();
 
                 if (status == RESPONSE_STATUS_SUCCESS)
-                {
                     ReadOnSuccess(reader);
-                }
                 else
-                {
-                    int32_t errorLen = reader.ReadString(0, 0);
-                    error.resize(errorLen);
-
-                    reader.ReadString(&error[0], static_cast<int32_t>(error.size()));
-                }
+                    utility::ReadString(reader, error);;
             }
             
             /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a758d1a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
index 8adcc60..db56660 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
@@ -357,7 +357,7 @@ namespace ignite
              * @param nullable - Nullability flag.
              */
             void DescribeParam(int16_t paramNum, int16_t* dataType,
-                size_t* paramSize, int16_t* decimalDigits, int16_t* nullable);
+                SqlUlen* paramSize, int16_t* decimalDigits, int16_t* nullable);
 
         private:
             IGNITE_NO_COPY_ASSIGNMENT(Statement);
@@ -583,7 +583,7 @@ namespace ignite
              * @return Operation result.
              */
             SqlResult InternalDescribeParam(int16_t paramNum, int16_t* dataType,
-                size_t* paramSize, int16_t* decimalDigits, int16_t* nullable);
+                SqlUlen* paramSize, int16_t* decimalDigits, int16_t* nullable);
 
             /**
              * Make request to data source to update parameters metadata.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a758d1a/modules/platforms/cpp/odbc/src/connection.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp
index e8db376..38f8cba 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -180,7 +180,9 @@ namespace ignite
             if (!connected)
                 IGNITE_ERROR_1(IgniteError::IGNITE_ERR_ILLEGAL_STATE, "Connection is not established");
 
-            common::FixedSizeArray<int8_t> msg(len + sizeof(OdbcProtocolHeader));
+            int32_t newLen = static_cast<int32_t>(len + sizeof(OdbcProtocolHeader));
+
+            common::FixedSizeArray<int8_t> msg(newLen);
 
             OdbcProtocolHeader *hdr = reinterpret_cast<OdbcProtocolHeader*>(msg.GetData());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a758d1a/modules/platforms/cpp/odbc/src/statement.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp
index a0b38d7..02c6dd9 100644
--- a/modules/platforms/cpp/odbc/src/statement.cpp
+++ b/modules/platforms/cpp/odbc/src/statement.cpp
@@ -821,14 +821,14 @@ namespace ignite
         }
 
         void Statement::DescribeParam(int16_t paramNum, int16_t* dataType,
-            size_t* paramSize, int16_t* decimalDigits, int16_t* nullable)
+            SqlUlen* paramSize, int16_t* decimalDigits, int16_t* nullable)
         {
             IGNITE_ODBC_API_CALL(InternalDescribeParam(paramNum,
                 dataType, paramSize, decimalDigits, nullable));
         }
 
         SqlResult Statement::InternalDescribeParam(int16_t paramNum, int16_t* dataType,
-            size_t* paramSize, int16_t* decimalDigits, int16_t* nullable)
+            SqlUlen* paramSize, int16_t* decimalDigits, int16_t* nullable)
         {
             query::Query *qry = currentQuery.get();
             if (!qry)

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a758d1a/modules/platforms/cpp/odbc/src/utility.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/utility.cpp b/modules/platforms/cpp/odbc/src/utility.cpp
index 9763144..22191eb 100644
--- a/modules/platforms/cpp/odbc/src/utility.cpp
+++ b/modules/platforms/cpp/odbc/src/utility.cpp
@@ -63,13 +63,7 @@ namespace ignite
                 reader.ReadString(&str[0], static_cast<int32_t>(str.size()));
             }
             else
-            {
                 str.clear();
-
-                char dummy;
-
-                reader.ReadString(&dummy, sizeof(dummy));
-            }
         }
 
         void WriteString(ignite::impl::binary::BinaryWriterImpl& writer, const std::string & str)


[27/52] ignite git commit: IGNITE-4279: Fixed JdbcResultSetSelfTest. This closes #1284.

Posted by vo...@apache.org.
IGNITE-4279: Fixed JdbcResultSetSelfTest. This closes #1284.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7b8d5308
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7b8d5308
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7b8d5308

Branch: refs/heads/master
Commit: 7b8d530858b997aeedc2f0281bbb707d9d8b86ef
Parents: 5b2eacf
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Nov 25 14:22:17 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 14:22:17 2016 +0300

----------------------------------------------------------------------
 .../internal/jdbc2/JdbcResultSetSelfTest.java   | 18 ++++++++
 .../ignite/internal/jdbc2/JdbcResultSet.java    | 47 +++++++++++++++++---
 .../ignite/internal/jdbc2/JdbcStatement.java    |  4 +-
 3 files changed, 62 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7b8d5308/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java
index 131ed74..1b5223e 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java
@@ -596,6 +596,24 @@ public class JdbcResultSetSelfTest extends GridCommonAbstractTest {
         stmt.setFetchSize(0);
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNewQueryTaskFetchSize() throws Exception {
+        stmt.setFetchSize(1);
+
+        boolean res = stmt.execute("select * from TestObject where id > 0");
+
+        assertTrue(res);
+
+        ResultSet rs = stmt.getResultSet();
+
+        assertTrue(rs.next());
+        assertTrue(rs.next());
+        assertTrue(rs.next());
+
+        stmt.setFetchSize(0);
+    }
 
     /**
      * @throws Exception If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b8d5308/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
index c1a5f4c..b53521e 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
@@ -88,10 +88,11 @@ public class JdbcResultSet implements ResultSet {
     /** Fetch size. */
     private int fetchSize;
 
+    /** Which query task to use under the hood - {@link JdbcQueryTaskV2} if {@code true}, {@link JdbcQueryTask} otherwise. */
+    private final boolean useNewQryTask;
+
     /**
-     * Creates new result set with predefined fields.
-     * Result set created with this constructor will
-     * never execute remote tasks.
+     * Creates new result set.
      *
      * @param uuid Query UUID.
      * @param stmt Statement.
@@ -102,6 +103,39 @@ public class JdbcResultSet implements ResultSet {
      */
     JdbcResultSet(@Nullable UUID uuid, JdbcStatement stmt, List<String> tbls, List<String> cols,
         List<String> types, Collection<List<?>> fields, boolean finished) {
+        this(uuid, stmt, tbls, cols, types, fields, finished, false);
+    }
+
+    /**
+     * Creates new result set that will be based on {@link JdbcQueryTaskV2}. This method is intended for use inside
+     *     {@link JdbcStatement} only.
+     *
+     * @param uuid Query UUID.
+     * @param stmt Statement.
+     * @param tbls Table names.
+     * @param cols Column names.
+     * @param types Types.
+     * @param fields Fields.
+     */
+    static JdbcResultSet resultSetForQueryTaskV2(@Nullable UUID uuid, JdbcStatement stmt, List<String> tbls,
+            List<String> cols, List<String> types, Collection<List<?>> fields, boolean finished) {
+        return new JdbcResultSet(uuid, stmt, tbls, cols, types, fields, finished, true);
+    }
+
+    /**
+     * Creates new result set.
+     *
+     * @param uuid Query UUID.
+     * @param stmt Statement.
+     * @param tbls Table names.
+     * @param cols Column names.
+     * @param types Types.
+     * @param fields Fields.
+     * @param useNewQryTask Which query task to use under the hood - {@link JdbcQueryTaskV2} if {@code true},
+     *     {@link JdbcQueryTask} otherwise.
+     */
+    private JdbcResultSet(@Nullable UUID uuid, JdbcStatement stmt, List<String> tbls, List<String> cols,
+        List<String> types, Collection<List<?>> fields, boolean finished, boolean useNewQryTask) {
         assert stmt != null;
         assert tbls != null;
         assert cols != null;
@@ -116,6 +150,8 @@ public class JdbcResultSet implements ResultSet {
         this.finished = finished;
 
         this.it = fields.iterator();
+
+        this.useNewQryTask = useNewQryTask;
     }
 
     /** {@inheritDoc} */
@@ -147,7 +183,7 @@ public class JdbcResultSet implements ResultSet {
 
             boolean loc = nodeId == null;
 
-            if (conn.isDmlSupported()) {
+            if (useNewQryTask) {
                 // Connections from new clients send queries with new tasks, so we have to continue in the same manner
                 JdbcQueryTaskV2 qryTask = new JdbcQueryTaskV2(loc ? ignite : null, conn.cacheName(), null, true, loc, null,
                     fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins());
@@ -209,8 +245,9 @@ public class JdbcResultSet implements ResultSet {
      * If this result set is associated with locally executed query then query cursor will also closed.
      */
     void closeInternal() throws SQLException  {
-        if (((JdbcConnection)stmt.getConnection()).nodeId() == null && uuid != null)
+        if (((JdbcConnection)stmt.getConnection()).nodeId() == null && uuid != null) {
             JdbcQueryTask.remove(uuid);
+        }
 
         closed = true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b8d5308/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
index dbb2390..5621058 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
@@ -346,8 +346,8 @@ public class JdbcStatement implements Statement {
                 loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask);
 
             if (res.isQuery()) {
-                JdbcResultSet rs = new JdbcResultSet(uuid, this, res.getTbls(), res.getCols(), res.getTypes(),
-                    res.getRows(), res.isFinished());
+                JdbcResultSet rs = JdbcResultSet.resultSetForQueryTaskV2(uuid, this, res.getTbls(), res.getCols(),
+                    res.getTypes(), res.getRows(), res.isFinished());
 
                 rs.setFetchSize(fetchSize);
 


[12/52] ignite git commit: IGNITE-3910: ODBC: Implemented SQLDescribeParam. This closes #1269.

Posted by vo...@apache.org.
IGNITE-3910: ODBC: Implemented SQLDescribeParam. This closes #1269.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/4d243251
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4d243251
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4d243251

Branch: refs/heads/master
Commit: 4d243251006bc8d1de85d7b6a481976899755a82
Parents: 83363d8
Author: isapego <ig...@gmail.com>
Authored: Wed Nov 23 18:11:35 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Nov 23 18:11:35 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcMessageParser.java      |  16 ++
 .../odbc/OdbcQueryGetParamsMetaRequest.java     |  60 ++++++
 .../odbc/OdbcQueryGetParamsMetaResult.java      |  40 ++++
 .../internal/processors/odbc/OdbcRequest.java   |   3 +
 .../processors/odbc/OdbcRequestHandler.java     | 128 +++++++++++--
 .../query-example/src/query_example.cpp         |   8 +-
 .../platforms/cpp/odbc/include/ignite/odbc.h    |  14 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |   2 +
 .../cpp/odbc/include/ignite/odbc/message.h      | 129 ++++++++++++-
 .../odbc/include/ignite/odbc/query/data_query.h |  12 +-
 .../cpp/odbc/include/ignite/odbc/query/query.h  |  44 ++++-
 .../cpp/odbc/include/ignite/odbc/statement.h    |  35 ++++
 .../cpp/odbc/include/ignite/odbc/type_traits.h  |   2 +-
 .../cpp/odbc/include/ignite/odbc/utility.h      |  11 +-
 .../platforms/cpp/odbc/src/app/parameter.cpp    |   3 +-
 modules/platforms/cpp/odbc/src/entry_points.cpp |  32 ++--
 modules/platforms/cpp/odbc/src/odbc.cpp         |  21 +++
 .../odbc/src/query/column_metadata_query.cpp    |   2 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |   2 +-
 .../cpp/odbc/src/query/foreign_keys_query.cpp   |   2 +-
 .../cpp/odbc/src/query/primary_keys_query.cpp   |   2 +-
 .../odbc/src/query/special_columns_query.cpp    |   2 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |   2 +-
 .../cpp/odbc/src/query/type_info_query.cpp      |   2 +-
 modules/platforms/cpp/odbc/src/statement.cpp    | 187 +++++++++++++++++--
 modules/platforms/cpp/odbc/src/utility.cpp      |  14 ++
 26 files changed, 697 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
index 19aace8..7b863d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
@@ -154,6 +154,15 @@ public class OdbcMessageParser {
                 break;
             }
 
+            case OdbcRequest.GET_PARAMS_META: {
+                String cacheName = reader.readString();
+                String sqlQuery = reader.readString();
+
+                res = new OdbcQueryGetParamsMetaRequest(cacheName, sqlQuery);
+
+                break;
+            }
+
             default:
                 throw new IgniteException("Unknown ODBC command: [cmd=" + cmd + ']');
         }
@@ -287,6 +296,13 @@ public class OdbcMessageParser {
             for (OdbcTableMeta tableMeta : tablesMeta)
                 tableMeta.writeBinary(writer);
         }
+        else if (res0 instanceof OdbcQueryGetParamsMetaResult) {
+            OdbcQueryGetParamsMetaResult res = (OdbcQueryGetParamsMetaResult) res0;
+
+            byte[] typeIds = res.typeIds();
+
+            writer.writeObjectDetached(typeIds);
+        }
         else
             assert false : "Should not reach here.";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetParamsMetaRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetParamsMetaRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetParamsMetaRequest.java
new file mode 100644
index 0000000..c372c0b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetParamsMetaRequest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.odbc;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * ODBC query get params meta request.
+ */
+public class OdbcQueryGetParamsMetaRequest extends OdbcRequest {
+    /** Cache. */
+    private final String cacheName;
+
+    /** Query. */
+    private final String query;
+
+    /**
+     * @param query SQL Query.
+     */
+    public OdbcQueryGetParamsMetaRequest(String cacheName, String query) {
+        super(GET_PARAMS_META);
+
+        this.cacheName = cacheName;
+        this.query = query;
+    }
+
+    /**
+     * @return SQL Query.
+     */
+    public String query() {
+        return query;
+    }
+
+    /**
+     * @return Cache name.
+     */
+    public String cacheName() {
+        return cacheName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(OdbcQueryGetParamsMetaRequest.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetParamsMetaResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetParamsMetaResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetParamsMetaResult.java
new file mode 100644
index 0000000..616c82d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetParamsMetaResult.java
@@ -0,0 +1,40 @@
+/*
+ * 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.odbc;
+
+/**
+ * ODBC query get params meta result.
+ */
+public class OdbcQueryGetParamsMetaResult {
+    /** List of parameter type IDs. */
+    private final byte[] typeIds;
+
+    /**
+     * @param typeIds List of parameter type IDs.
+     */
+    public OdbcQueryGetParamsMetaResult(byte[] typeIds) {
+        this.typeIds = typeIds;
+    }
+
+    /**
+     * @return List of parameter type IDs.
+     */
+    public byte[] typeIds() {
+        return typeIds;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java
index ebecb60..f056fed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java
@@ -39,6 +39,9 @@ public class OdbcRequest {
     /** Get columns meta query. */
     public static final int GET_TABLES_META = 6;
 
+    /** Get parameters meta. */
+    public static final int GET_PARAMS_META = 7;
+
     /** Command. */
     private final int cmd;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index cf2615d..588fc66 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
@@ -18,10 +18,12 @@
 package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeUtils;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
@@ -32,6 +34,9 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteProductVersion;
 
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.Types;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
@@ -114,6 +119,9 @@ public class OdbcRequestHandler {
 
                 case GET_TABLES_META:
                     return getTablesMeta(reqId, (OdbcQueryGetTablesMetaRequest)req);
+
+                case GET_PARAMS_META:
+                    return getParamsMeta(reqId, (OdbcQueryGetParamsMetaRequest)req);
             }
 
             return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Unsupported ODBC request: " + req);
@@ -131,26 +139,33 @@ public class OdbcRequestHandler {
      * @return Response.
      */
     private OdbcResponse performHandshake(long reqId, OdbcHandshakeRequest req) {
-        OdbcProtocolVersion version = req.version();
+        try {
+            OdbcProtocolVersion version = req.version();
 
-        if (version.isUnknown()) {
-            IgniteProductVersion ver = ctx.grid().version();
+            if (version.isUnknown()) {
+                IgniteProductVersion ver = ctx.grid().version();
 
-            String verStr = Byte.toString(ver.major()) + '.' + ver.minor() + '.' + ver.maintenance();
+                String verStr = Byte.toString(ver.major()) + '.' + ver.minor() + '.' + ver.maintenance();
 
-            OdbcHandshakeResult res = new OdbcHandshakeResult(false, OdbcProtocolVersion.current().since(), verStr);
+                OdbcHandshakeResult res = new OdbcHandshakeResult(false, OdbcProtocolVersion.current().since(), verStr);
 
-            return new OdbcResponse(res);
-        }
+                return new OdbcResponse(res);
+            }
+
+            OdbcHandshakeResult res = new OdbcHandshakeResult(true, null, null);
 
-        OdbcHandshakeResult res = new OdbcHandshakeResult(true, null, null);
+            if (version.isDistributedJoinsSupported()) {
+                distributedJoins = req.distributedJoins();
+                enforceJoinOrder = req.enforceJoinOrder();
+            }
 
-        if (version.isDistributedJoinsSupported()) {
-            distributedJoins = req.distributedJoins();
-            enforceJoinOrder = req.enforceJoinOrder();
+            return new OdbcResponse(res);
         }
+        catch (Exception e) {
+            U.error(log, "Failed to perform handshake [reqId=" + reqId + ", req=" + req + ']', e);
 
-        return new OdbcResponse(res);
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage());
+        }
     }
 
     /**
@@ -387,6 +402,95 @@ public class OdbcRequestHandler {
     }
 
     /**
+     * {@link OdbcQueryGetParamsMetaRequest} command handler.
+     *
+     * @param reqId Request ID.
+     * @param req Get params metadata request.
+     * @return Response.
+     */
+    private OdbcResponse getParamsMeta(long reqId, OdbcQueryGetParamsMetaRequest req) {
+        try {
+            PreparedStatement stmt = ctx.query().prepareNativeStatement(req.cacheName(), req.query());
+
+            ParameterMetaData pmd = stmt.getParameterMetaData();
+
+            byte[] typeIds = new byte[pmd.getParameterCount()];
+
+            for (int i = 1; i <= pmd.getParameterCount(); ++i) {
+                int sqlType = pmd.getParameterType(i);
+
+                typeIds[i - 1] = sqlTypeToBinary(sqlType);
+            }
+
+            OdbcQueryGetParamsMetaResult res = new OdbcQueryGetParamsMetaResult(typeIds);
+
+            return new OdbcResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get params metadata [reqId=" + reqId + ", req=" + req + ']', e);
+
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage());
+        }
+    }
+
+    /**
+     * Convert {@link java.sql.Types} to binary type constant (See {@link GridBinaryMarshaller} constants).
+     *
+     * @param sqlType SQL type.
+     * @return Binary type.
+     */
+    private static byte sqlTypeToBinary(int sqlType) {
+        switch (sqlType) {
+            case Types.BIGINT:
+                return GridBinaryMarshaller.LONG;
+
+            case Types.BOOLEAN:
+                return GridBinaryMarshaller.BOOLEAN;
+
+            case Types.DATE:
+                return GridBinaryMarshaller.DATE;
+
+            case Types.DOUBLE:
+                return GridBinaryMarshaller.DOUBLE;
+
+            case Types.FLOAT:
+            case Types.REAL:
+                return GridBinaryMarshaller.FLOAT;
+
+            case Types.NUMERIC:
+            case Types.DECIMAL:
+                return GridBinaryMarshaller.DECIMAL;
+
+            case Types.INTEGER:
+                return GridBinaryMarshaller.INT;
+
+            case Types.SMALLINT:
+                return GridBinaryMarshaller.SHORT;
+
+            case Types.TIME:
+            case Types.TIMESTAMP:
+                return GridBinaryMarshaller.TIMESTAMP;
+
+            case Types.TINYINT:
+                return GridBinaryMarshaller.BYTE;
+
+            case Types.CHAR:
+            case Types.VARCHAR:
+            case Types.LONGNVARCHAR:
+                return GridBinaryMarshaller.STRING;
+
+            case Types.NULL:
+                return GridBinaryMarshaller.NULL;
+
+            case Types.BINARY:
+            case Types.VARBINARY:
+            case Types.LONGVARBINARY:
+            default:
+                return GridBinaryMarshaller.BYTE_ARR;
+        }
+    }
+
+    /**
      * Convert metadata in collection from {@link GridQueryFieldMetadata} to
      * {@link OdbcColumnMeta}.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/examples/query-example/src/query_example.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/query-example/src/query_example.cpp b/modules/platforms/cpp/examples/query-example/src/query_example.cpp
index 9bf3e52..8b8d008 100644
--- a/modules/platforms/cpp/examples/query-example/src/query_example.cpp
+++ b/modules/platforms/cpp/examples/query-example/src/query_example.cpp
@@ -416,6 +416,12 @@ int main()
         // Populate cache.
         Initialize();
 
+        std::cout << std::endl;
+        std::cout << ">>> Ready" << std::endl;
+        std::cout << std::endl;
+
+        std::cin.get();
+
         // Example for SCAN-based query based on a predicate.
         DoScanQuery();
 
@@ -456,4 +462,4 @@ int main()
     std::cin.get();
 
     return 0;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/include/ignite/odbc.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc.h b/modules/platforms/cpp/odbc/include/ignite/odbc.h
index 345cdb8..639a02a 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc.h
@@ -34,11 +34,6 @@
 
 namespace ignite
 {
-    BOOL ConfigDSN(HWND     hwndParent,
-                   WORD     req,
-                   LPCSTR   driver,
-                   LPCSTR   attributes);
-
     SQLRETURN SQLGetInfo(SQLHDBC        conn,
                          SQLUSMALLINT   infoType,
                          SQLPOINTER     infoValue,
@@ -265,6 +260,13 @@ namespace ignite
                        SQLSMALLINT  msgBufLen,
                        SQLSMALLINT* msgResLen);
 
+    SQLRETURN SQLDescribeParam(SQLHSTMT     stmt,
+                               SQLUSMALLINT paramNum,
+                               SQLSMALLINT* dataType,
+                               SQLULEN*     paramSize,
+                               SQLSMALLINT* decimalDigits,
+                               SQLSMALLINT* nullable);
+
 } // namespace ignite
 
-#endif //_IGNITE_ODBC
\ No newline at end of file
+#endif //_IGNITE_ODBC

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
index 354cf08..a61be53 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
@@ -23,6 +23,8 @@
 
 #include <ignite/odbc/system/odbc_constants.h>
 
+#include <ignite/odbc/system/odbc_constants.h>
+
 namespace ignite
 {
     namespace odbc

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
index 03fa627..5c6a933 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
@@ -46,7 +46,9 @@ namespace ignite
 
             REQUEST_TYPE_GET_COLUMNS_METADATA = 5,
 
-            REQUEST_TYPE_GET_TABLES_METADATA = 6
+            REQUEST_TYPE_GET_TABLES_METADATA = 6,
+
+            REQUEST_TYPE_GET_PARAMS_METADATA = 7
         };
 
         enum ResponseStatus
@@ -125,7 +127,9 @@ namespace ignite
              */
             QueryExecuteRequest(const std::string& cache, const std::string& sql,
                 const app::ParameterBindingMap& params) :
-                cache(cache), sql(sql), params(params)
+                cache(cache),
+                sql(sql),
+                params(params)
             {
                 // No-op.
             }
@@ -151,9 +155,20 @@ namespace ignite
                 writer.WriteInt32(static_cast<int32_t>(params.size()));
 
                 app::ParameterBindingMap::const_iterator i;
+                uint16_t prev = 0;
+
+                for (i = params.begin(); i != params.end(); ++i) {
+                    uint16_t current = i->first;
+
+                    while ((current - prev) > 1) {
+                        writer.WriteNull();
+                        ++prev;
+                    }
 
-                for (i = params.begin(); i != params.end(); ++i)
                     i->second.Write(writer);
+
+                    prev = current;
+                }
             }
 
         private:
@@ -220,7 +235,8 @@ namespace ignite
              * @param pageSize Required page size.
              */
             QueryFetchRequest(int64_t queryId, int32_t pageSize) :
-                queryId(queryId), pageSize(pageSize)
+                queryId(queryId),
+                pageSize(pageSize)
             {
                 // No-op.
             }
@@ -266,7 +282,9 @@ namespace ignite
              * @param column Column name.
              */
             QueryGetColumnsMetaRequest(const std::string& schema, const std::string& table, const std::string& column) :
-                schema(schema), table(table), column(column)
+                schema(schema),
+                table(table),
+                column(column)
             {
                 // No-op.
             }
@@ -319,7 +337,10 @@ namespace ignite
              */
             QueryGetTablesMetaRequest(const std::string& catalog, const std::string& schema,
                                       const std::string& table, const std::string& tableTypes) :
-                catalog(catalog), schema(schema), table(table), tableTypes(tableTypes)
+                catalog(catalog),
+                schema(schema),
+                table(table),
+                tableTypes(tableTypes)
             {
                 // No-op.
             }
@@ -361,6 +382,55 @@ namespace ignite
         };
 
         /**
+         * Get parameter metadata request.
+         */
+        class QueryGetParamsMetaRequest
+        {
+        public:
+            /**
+             * Constructor.
+             *
+             * @param catalog Catalog search pattern.
+             * @param schema Schema search pattern.
+             * @param table Table search pattern.
+             * @param tableTypes Table types search pattern.
+             */
+            QueryGetParamsMetaRequest(const std::string& cacheName, const std::string& sqlQuery) :
+                cacheName(cacheName),
+                sqlQuery(sqlQuery)
+            {
+                // No-op.
+            }
+
+            /**
+             * Destructor.
+             */
+            ~QueryGetParamsMetaRequest()
+            {
+                // No-op.
+            }
+
+            /**
+             * Write request using provided writer.
+             * @param writer Writer.
+             */
+            void Write(ignite::impl::binary::BinaryWriterImpl& writer) const
+            {
+                writer.WriteInt8(REQUEST_TYPE_GET_PARAMS_METADATA);
+
+                utility::WriteString(writer, cacheName);
+                utility::WriteString(writer, sqlQuery);
+            }
+
+        private:
+            /** Cache name. */
+            std::string cacheName;
+
+            /** SQL query. */
+            std::string sqlQuery;
+        };
+
+        /**
          * Query close response.
          */
         class Response
@@ -760,7 +830,52 @@ namespace ignite
             /** Columns metadata. */
             meta::TableMetaVector meta;
         };
+
+        /**
+         * Get params metadata response.
+         */
+        class QueryGetParamsMetaResponse : public Response
+        {
+        public:
+            /**
+             * Constructor.
+             */
+            QueryGetParamsMetaResponse()
+            {
+                // No-op.
+            }
+
+            /**
+             * Destructor.
+             */
+            ~QueryGetParamsMetaResponse()
+            {
+                // No-op.
+            }
+
+            /**
+             * Get parameter type IDs.
+             * @return Type IDs.
+             */
+            const std::vector<int8_t>& GetTypeIds() const
+            {
+                return typeIds;
+            }
+
+        private:
+            /**
+             * Read response using provided reader.
+             * @param reader Reader.
+             */
+            virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader)
+            {
+                utility::ReadByteArray(reader, typeIds);
+            }
+
+            /** Columns metadata. */
+            std::vector<int8_t> typeIds;
+        };
     }
 }
 
-#endif //_IGNITE_ODBC_MESSAGE
\ No newline at end of file
+#endif //_IGNITE_ODBC_MESSAGE

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h b/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h
index 3db3b46..68bb877 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h
@@ -105,6 +105,16 @@ namespace ignite
                  */
                 virtual int64_t AffectedRows() const;
 
+                /**
+                 * Get SQL query string.
+                 *
+                 * @return SQL query string.
+                 */
+                const std::string& GetSql() const
+                {
+                    return sql;
+                }
+
             private:
                 IGNITE_NO_COPY_ASSIGNMENT(DataQuery);
 
@@ -149,4 +159,4 @@ namespace ignite
     }
 }
 
-#endif //_IGNITE_ODBC_QUERY_DATA_QUERY
\ No newline at end of file
+#endif //_IGNITE_ODBC_QUERY_DATA_QUERY

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h b/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h
index 1b3512d..40be1ed 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h
@@ -39,6 +39,30 @@ namespace ignite
             class Query
             {
             public:
+                /** Query type. */
+                enum Type {
+                    /** Column metadata query type. */
+                    COLUMN_METADATA,
+
+                    /** Data query type. */
+                    DATA,
+
+                    /** Foreign keys query type. */
+                    FOREIGN_KEYS,
+
+                    /** Primary keys query type. */
+                    PRIMARY_KEYS,
+
+                    /** Special columns query type. */
+                    SPECIAL_COLUMNS,
+
+                    /** Table metadata query type. */
+                    TABLE_METADATA,
+
+                    /** Type info query type. */
+                    TYPE_INFO
+                };
+
                 /**
                  * Destructor.
                  */
@@ -99,21 +123,35 @@ namespace ignite
                  */
                 virtual int64_t AffectedRows() const = 0;
 
+                /**
+                 * Get query type.
+                 *
+                 * @return Query type.
+                 */
+                Type GetType() const
+                {
+                    return type;
+                }
+
             protected:
                 /**
                  * Constructor.
                  */
-                Query(diagnostic::Diagnosable& diag) :
-                    diag(diag)
+                Query(diagnostic::Diagnosable& diag, Type type) :
+                    diag(diag),
+                    type(type)
                 {
                     // No-op.
                 }
 
                 /** Diagnostics collector. */
                 diagnostic::Diagnosable& diag;
+
+                /** Query type. */
+                Type type;
             };
         }
     }
 }
 
-#endif //_IGNITE_ODBC_QUERY_QUERY
\ No newline at end of file
+#endif //_IGNITE_ODBC_QUERY_QUERY

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
index 981ade1..8adcc60 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
@@ -347,6 +347,18 @@ namespace ignite
              */
             void PutData(void* data, SqlLen len);
 
+            /**
+             * Get type info of the parameter of the prepared statement.
+             *
+             * @param paramNum - Parameter index.
+             * @param dataType - Data type.
+             * @param paramSize - Size of the parameter.
+             * @param decimalDigits - Decimal digits.
+             * @param nullable - Nullability flag.
+             */
+            void DescribeParam(int16_t paramNum, int16_t* dataType,
+                size_t* paramSize, int16_t* decimalDigits, int16_t* nullable);
+
         private:
             IGNITE_NO_COPY_ASSIGNMENT(Statement);
 
@@ -547,6 +559,7 @@ namespace ignite
              * Select next parameter data for which is required.
              *
              * @param paramPtr Pointer to param id stored here.
+             * @return Operation result.
              */
             SqlResult InternalSelectParam(void** paramPtr);
 
@@ -555,10 +568,29 @@ namespace ignite
              *
              * @param data Data.
              * @param len Data length.
+             * @return Operation result.
              */
             SqlResult InternalPutData(void* data, SqlLen len);
 
             /**
+             * Get type info of the parameter of the prepared statement.
+             *
+             * @param paramNum - Parameter index.
+             * @param dataType - Data type.
+             * @param paramSize - Size of the parameter.
+             * @param decimalDigits - Decimal digits.
+             * @param nullable - Nullability flag.
+             * @return Operation result.
+             */
+            SqlResult InternalDescribeParam(int16_t paramNum, int16_t* dataType,
+                size_t* paramSize, int16_t* decimalDigits, int16_t* nullable);
+
+            /**
+             * Make request to data source to update parameters metadata.
+             */
+            SqlResult UpdateParamsMeta();
+
+            /**
              * Constructor.
              * Called by friend classes.
              *
@@ -575,6 +607,9 @@ namespace ignite
             /** Parameter bindings. */
             app::ParameterBindingMap paramBindings;
 
+            /** Parameter meta. */
+            std::vector<int8_t> paramTypes;
+
             /** Underlying query. */
             std::auto_ptr<query::Query> currentQuery;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/include/ignite/odbc/type_traits.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/type_traits.h b/modules/platforms/cpp/odbc/include/ignite/odbc/type_traits.h
index a85cfeb..a57abdd 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/type_traits.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/type_traits.h
@@ -313,4 +313,4 @@ namespace ignite
     }
 }
 
-#endif //_IGNITE_ODBC_TYPE_TRAITS
\ No newline at end of file
+#endif //_IGNITE_ODBC_TYPE_TRAITS

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h b/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
index 34627c1..37daa50 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
@@ -43,7 +43,7 @@ void logInit(const char*);
 #   define LOG_MSG(fmt, ...)                                        \
     do {                                                            \
         logInit(ODBC_LOG_PATH);                                     \
-        fprintf(log_file, "%s: " fmt, __FUNCTION__, __VA_ARGS__);   \
+        fprintf(log_file, "%s: " fmt, __FUNCTION__, ##__VA_ARGS__);   \
         fflush(log_file);                                           \
     } while (false)
 
@@ -137,6 +137,13 @@ namespace ignite
         size_t CopyStringToBuffer(const std::string& str, char* buf, size_t buflen);
 
         /**
+         * Read array from reader.
+         * @param reader Reader.
+         * @param res Resulting vector.
+         */
+        void ReadByteArray(impl::binary::BinaryReaderImpl& reader, std::vector<int8_t>& res);
+
+        /**
          * Read string from reader.
          * @param reader Reader.
          * @param str String.
@@ -177,4 +184,4 @@ namespace ignite
     }
 }
 
-#endif //_IGNITE_ODBC_UTILITY
\ No newline at end of file
+#endif //_IGNITE_ODBC_UTILITY

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/app/parameter.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/app/parameter.cpp b/modules/platforms/cpp/odbc/src/app/parameter.cpp
index d70ef6a..937ef58 100644
--- a/modules/platforms/cpp/odbc/src/app/parameter.cpp
+++ b/modules/platforms/cpp/odbc/src/app/parameter.cpp
@@ -52,7 +52,7 @@ namespace ignite
                 // No-op.
             }
 
-            Parameter::Parameter(const Parameter & other) :
+            Parameter::Parameter(const Parameter& other) :
                 buffer(other.buffer),
                 sqlType(other.sqlType),
                 columnSize(other.columnSize),
@@ -266,4 +266,3 @@ namespace ignite
         }
     }
 }
-

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/entry_points.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/entry_points.cpp b/modules/platforms/cpp/odbc/src/entry_points.cpp
index a85b3cf..850fa10 100644
--- a/modules/platforms/cpp/odbc/src/entry_points.cpp
+++ b/modules/platforms/cpp/odbc/src/entry_points.cpp
@@ -394,13 +394,11 @@ SQLRETURN SQL_API SQLSpecialColumns(SQLHSTMT    stmt,
                                     SQLUSMALLINT scope,
                                     SQLUSMALLINT nullable)
 {
-    return ignite::SQLSpecialColumns(stmt, idType, catalogName,
-        catalogNameLen, schemaName, schemaNameLen, tableName,
-        tableNameLen, scope, nullable);
+    return ignite::SQLSpecialColumns(stmt, idType, catalogName, catalogNameLen, schemaName,
+        schemaNameLen, tableName, tableNameLen, scope, nullable);
 }
 
-SQLRETURN SQL_API SQLParamData(SQLHSTMT    stmt,
-                               SQLPOINTER* value)
+SQLRETURN SQL_API SQLParamData(SQLHSTMT stmt, SQLPOINTER* value)
 {
     return ignite::SQLParamData(stmt, value);
 }
@@ -412,6 +410,17 @@ SQLRETURN SQL_API SQLPutData(SQLHSTMT     stmt,
     return ignite::SQLPutData(stmt, data, strLengthOrIndicator);
 }
 
+SQLRETURN SQL_API SQLDescribeParam(SQLHSTMT     stmt,
+                                   SQLUSMALLINT paramNum,
+                                   SQLSMALLINT* dataType,
+                                   SQLULEN*     paramSize,
+                                   SQLSMALLINT* decimalDigits,
+                                   SQLSMALLINT* nullable)
+{
+    return ignite::SQLDescribeParam(stmt, paramNum, dataType,
+     paramSize, decimalDigits, nullable);
+}
+
 SQLRETURN SQL_API SQLError(SQLHENV      env,
                            SQLHDBC      conn,
                            SQLHSTMT     stmt,
@@ -663,19 +672,6 @@ SQLRETURN SQL_API SQLColumnPrivileges(SQLHSTMT      stmt,
     return SQL_SUCCESS;
 }
 
-/*
-SQLRETURN SQL_API SQLDescribeParam(SQLHSTMT     stmt,
-                                   SQLUSMALLINT paramNum,
-                                   SQLSMALLINT* dataType,
-                                   SQLULEN*     paramSize,
-                                   SQLSMALLINT* decimalDigits,
-                                   SQLSMALLINT* nullable)
-{
-    LOG_MSG("SQLDescribeParam called\n");
-    return SQL_SUCCESS;
-}
-*/
-
 SQLRETURN SQL_API SQLParamOptions(SQLHSTMT  stmt,
                                   SQLULEN   paramSetSize,
                                   SQLULEN*  paramsProcessed)

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/odbc.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp
index 8ecc0f8..0b18a11 100644
--- a/modules/platforms/cpp/odbc/src/odbc.cpp
+++ b/modules/platforms/cpp/odbc/src/odbc.cpp
@@ -1257,6 +1257,27 @@ namespace ignite
         return statement->GetDiagnosticRecords().GetReturnCode();
     }
 
+    SQLRETURN SQLDescribeParam(SQLHSTMT     stmt,
+                               SQLUSMALLINT paramNum,
+                               SQLSMALLINT* dataType,
+                               SQLULEN*     paramSize,
+                               SQLSMALLINT* decimalDigits,
+                               SQLSMALLINT* nullable)
+    {
+        using namespace ignite::odbc;
+
+        LOG_MSG("SQLDescribeParam called\n");
+
+        Statement *statement = reinterpret_cast<Statement*>(stmt);
+
+        if (!statement)
+            return SQL_INVALID_HANDLE;
+
+        statement->DescribeParam(paramNum, dataType, paramSize, decimalDigits, nullable);
+
+        return statement->GetDiagnosticRecords().GetReturnCode();
+    }
+
     SQLRETURN SQLError(SQLHENV      env,
                        SQLHDBC      conn,
                        SQLHSTMT     stmt,

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp b/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
index 5e764a2..1cd2a9f 100644
--- a/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
@@ -73,7 +73,7 @@ namespace ignite
             ColumnMetadataQuery::ColumnMetadataQuery(diagnostic::Diagnosable& diag, 
                 Connection& connection, const std::string& schema,
                 const std::string& table, const std::string& column) :
-                Query(diag),
+                Query(diag, COLUMN_METADATA),
                 connection(connection),
                 schema(schema),
                 table(table),

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/query/data_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp
index 8f76153..5b1b758 100644
--- a/modules/platforms/cpp/odbc/src/query/data_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp
@@ -28,7 +28,7 @@ namespace ignite
             DataQuery::DataQuery(diagnostic::Diagnosable& diag,
                 Connection& connection, const std::string& sql,
                 const app::ParameterBindingMap& params) :
-                Query(diag),
+                Query(diag, DATA),
                 connection(connection),
                 sql(sql),
                 params(params)

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/query/foreign_keys_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/foreign_keys_query.cpp b/modules/platforms/cpp/odbc/src/query/foreign_keys_query.cpp
index 4fefa22..5813767 100644
--- a/modules/platforms/cpp/odbc/src/query/foreign_keys_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/foreign_keys_query.cpp
@@ -32,7 +32,7 @@ namespace ignite
                 const std::string& primaryCatalog, const std::string& primarySchema,
                 const std::string& primaryTable, const std::string& foreignCatalog,
                 const std::string& foreignSchema, const std::string& foreignTable) :
-                Query(diag),
+                Query(diag, FOREIGN_KEYS),
                 connection(connection),
                 primaryCatalog(primaryCatalog),
                 primarySchema(primarySchema),

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/query/primary_keys_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/primary_keys_query.cpp b/modules/platforms/cpp/odbc/src/query/primary_keys_query.cpp
index 11ff4fa..a24670d 100644
--- a/modules/platforms/cpp/odbc/src/query/primary_keys_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/primary_keys_query.cpp
@@ -55,7 +55,7 @@ namespace ignite
             PrimaryKeysQuery::PrimaryKeysQuery(diagnostic::Diagnosable& diag,
                 Connection& connection, const std::string& catalog,
                 const std::string& schema, const std::string& table) :
-                Query(diag),
+                Query(diag, PRIMARY_KEYS),
                 connection(connection),
                 catalog(catalog),
                 schema(schema),

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/query/special_columns_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/special_columns_query.cpp b/modules/platforms/cpp/odbc/src/query/special_columns_query.cpp
index ad8aea6..f507c52 100644
--- a/modules/platforms/cpp/odbc/src/query/special_columns_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/special_columns_query.cpp
@@ -29,7 +29,7 @@ namespace ignite
             SpecialColumnsQuery::SpecialColumnsQuery(diagnostic::Diagnosable& diag,
                 int16_t type, const std::string& catalog, const std::string& schema,
                 const std::string& table, int16_t scope, int16_t nullable) :
-                Query(diag),
+                Query(diag, SPECIAL_COLUMNS),
                 type(type),
                 catalog(catalog),
                 schema(schema),

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp b/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
index 53aebe4..3cddd1b 100644
--- a/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
@@ -52,7 +52,7 @@ namespace ignite
             TableMetadataQuery::TableMetadataQuery(diagnostic::Diagnosable& diag,
                 Connection& connection, const std::string& catalog,const std::string& schema,
                 const std::string& table, const std::string& tableType) :
-                Query(diag),
+                Query(diag, TABLE_METADATA),
                 connection(connection),
                 catalog(catalog),
                 schema(schema),

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/query/type_info_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/type_info_query.cpp b/modules/platforms/cpp/odbc/src/query/type_info_query.cpp
index 96bed3a..2de2a7c 100644
--- a/modules/platforms/cpp/odbc/src/query/type_info_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/type_info_query.cpp
@@ -116,7 +116,7 @@ namespace ignite
         namespace query
         {
             TypeInfoQuery::TypeInfoQuery(diagnostic::Diagnosable& diag, int16_t sqlType) :
-                Query(diag),
+                Query(diag, TYPE_INFO),
                 columnsMeta(),
                 executed(false),
                 types(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/statement.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp
index 49a9acf..a0b38d7 100644
--- a/modules/platforms/cpp/odbc/src/statement.cpp
+++ b/modules/platforms/cpp/odbc/src/statement.cpp
@@ -320,11 +320,13 @@ namespace ignite
             IGNITE_ODBC_API_CALL(InternalGetColumnData(columnIdx, buffer));
         }
 
-        SqlResult Statement::InternalGetColumnData(uint16_t columnIdx, app::ApplicationDataBuffer& buffer)
+        SqlResult Statement::InternalGetColumnData(uint16_t columnIdx,
+            app::ApplicationDataBuffer& buffer)
         {
             if (!currentQuery.get())
             {
-                AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE, "Cursor is not in the open state.");
+                AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE,
+                    "Cursor is not in the open state.");
 
                 return SQL_RESULT_ERROR;
             }
@@ -346,6 +348,9 @@ namespace ignite
 
             currentQuery.reset(new query::DataQuery(*this, connection, query, paramBindings));
 
+            // Resetting parameters types as we are changing the query.
+            paramTypes.clear();
+
             return SQL_RESULT_SUCCESS;
         }
 
@@ -413,7 +418,8 @@ namespace ignite
             if (cache.empty())
                 cache = connection.GetCache();
 
-            currentQuery.reset(new query::ColumnMetadataQuery(*this, connection, cache, table, column));
+            currentQuery.reset(new query::ColumnMetadataQuery(*this,
+                connection, cache, table, column));
 
             return currentQuery->Execute();
         }
@@ -421,7 +427,8 @@ namespace ignite
         void Statement::ExecuteGetTablesMetaQuery(const std::string& catalog,
             const std::string& schema, const std::string& table, const std::string& tableType)
         {
-            IGNITE_ODBC_API_CALL(InternalExecuteGetTablesMetaQuery(catalog, schema, table, tableType));
+            IGNITE_ODBC_API_CALL(InternalExecuteGetTablesMetaQuery(
+                catalog, schema, table, tableType));
         }
 
         SqlResult Statement::InternalExecuteGetTablesMetaQuery(const std::string& catalog,
@@ -430,7 +437,8 @@ namespace ignite
             if (currentQuery.get())
                 currentQuery->Close();
 
-            currentQuery.reset(new query::TableMetadataQuery(*this, connection, catalog, schema, table, tableType));
+            currentQuery.reset(new query::TableMetadataQuery(*this,
+                connection, catalog, schema, table, tableType));
 
             return currentQuery->Execute();
         }
@@ -452,25 +460,26 @@ namespace ignite
             if (currentQuery.get())
                 currentQuery->Close();
 
-            currentQuery.reset(new query::ForeignKeysQuery(*this, connection, primaryCatalog, primarySchema,
-                primaryTable, foreignCatalog, foreignSchema, foreignTable));
+            currentQuery.reset(new query::ForeignKeysQuery(*this, connection, primaryCatalog,
+                primarySchema, primaryTable, foreignCatalog, foreignSchema, foreignTable));
 
             return currentQuery->Execute();
         }
 
-        void Statement::ExecuteGetPrimaryKeysQuery(const std::string& catalog, const std::string& schema,
-            const std::string& table)
+        void Statement::ExecuteGetPrimaryKeysQuery(const std::string& catalog,
+            const std::string& schema, const std::string& table)
         {
             IGNITE_ODBC_API_CALL(InternalExecuteGetPrimaryKeysQuery(catalog, schema, table));
         }
 
-        SqlResult Statement::InternalExecuteGetPrimaryKeysQuery(const std::string& catalog, const std::string& schema,
-            const std::string& table)
+        SqlResult Statement::InternalExecuteGetPrimaryKeysQuery(const std::string& catalog,
+            const std::string& schema, const std::string& table)
         {
             if (currentQuery.get())
                 currentQuery->Close();
 
-            currentQuery.reset(new query::PrimaryKeysQuery(*this, connection, catalog, schema, table));
+            currentQuery.reset(new query::PrimaryKeysQuery(*this,
+                connection, catalog, schema, table));
 
             return currentQuery->Execute();
         }
@@ -513,7 +522,8 @@ namespace ignite
         {
             if (!type_traits::IsSqlTypeSupported(sqlType))
             {
-                AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, "Data type is not supported.");
+                AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED,
+                    "Data type is not supported.");
 
                 return SQL_RESULT_ERROR;
             }
@@ -535,7 +545,7 @@ namespace ignite
         {
             if (!currentQuery.get())
                 return SQL_RESULT_SUCCESS;
-
+            
             SqlResult result = currentQuery->Close();
 
             if (result == SQL_RESULT_SUCCESS)
@@ -556,7 +566,8 @@ namespace ignite
 
             if (!currentQuery.get())
             {
-                AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE, "Cursor is not in the open state.");
+                AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE,
+                    "Cursor is not in the open state.");
 
                 return SQL_RESULT_ERROR;
             }
@@ -625,7 +636,8 @@ namespace ignite
 
             if (colIdx > meta->size() + 1 || colIdx < 1)
             {
-                AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Column index is out of range.", 0, colIdx);
+                AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR,
+                    "Column index is out of range.", 0, colIdx);
 
                 return SQL_RESULT_ERROR;
             }
@@ -654,7 +666,8 @@ namespace ignite
 
             if (!found)
             {
-                AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, "Unknown attribute.");
+                AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED,
+                    "Unknown attribute.");
 
                 return SQL_RESULT_ERROR;
             }
@@ -714,7 +727,8 @@ namespace ignite
         {
             if (!paramPtr)
             {
-                AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Invalid parameter: ValuePtrPtr is null.");
+                AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR,
+                    "Invalid parameter: ValuePtrPtr is null.");
 
                 return SQL_RESULT_ERROR;
             }
@@ -805,6 +819,143 @@ namespace ignite
 
             return SQL_RESULT_SUCCESS;
         }
+
+        void Statement::DescribeParam(int16_t paramNum, int16_t* dataType,
+            size_t* paramSize, int16_t* decimalDigits, int16_t* nullable)
+        {
+            IGNITE_ODBC_API_CALL(InternalDescribeParam(paramNum,
+                dataType, paramSize, decimalDigits, nullable));
+        }
+
+        SqlResult Statement::InternalDescribeParam(int16_t paramNum, int16_t* dataType,
+            size_t* paramSize, int16_t* decimalDigits, int16_t* nullable)
+        {
+            query::Query *qry = currentQuery.get();
+            if (!qry)
+            {
+                AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query is not prepared.");
+
+                return SQL_RESULT_ERROR;
+            }
+
+            if (qry->GetType() != query::Query::DATA)
+            {
+                AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query is not SQL data query.");
+
+                return SQL_RESULT_ERROR;
+            }
+
+            int8_t type = 0;
+
+            if (paramNum > 0 && static_cast<size_t>(paramNum) <= paramTypes.size())
+                type = paramTypes[paramNum - 1];
+
+            LOG_MSG("Type: %d\n", type);
+
+            if (!type)
+            {
+                SqlResult res = UpdateParamsMeta();
+
+                if (res != SQL_RESULT_SUCCESS)
+                    return res;
+
+                if (paramNum < 1 || static_cast<size_t>(paramNum) > paramTypes.size())
+                    type = impl::binary::IGNITE_HDR_NULL;
+                else
+                    type = paramTypes[paramNum - 1];
+            }
+
+            if (dataType)
+                *dataType = type_traits::BinaryToSqlType(type);
+
+            if (paramSize)
+                *paramSize = type_traits::BinaryTypeColumnSize(type);
+
+            if (decimalDigits)
+                *decimalDigits = type_traits::BinaryTypeDecimalDigits(type);
+
+            if (nullable)
+                *nullable = type_traits::BinaryTypeNullability(type);
+
+            return SQL_RESULT_SUCCESS;
+        }
+
+        SqlResult Statement::UpdateParamsMeta()
+        {
+            query::Query *qry0 = currentQuery.get();
+
+            assert(qry0 != 0);
+            assert(qry0->GetType() == query::Query::DATA);
+
+            query::DataQuery* qry = static_cast<query::DataQuery*>(qry0);
+
+            const std::string& cacheName = connection.GetCache();
+            const std::string& sql = qry->GetSql();
+
+            QueryGetParamsMetaRequest req(cacheName, sql);
+            QueryGetParamsMetaResponse rsp;
+
+            try
+            {
+                connection.SyncMessage(req, rsp);
+            }
+            catch (const IgniteError& err)
+            {
+                AddStatusRecord(SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT, err.GetText());
+
+                return SQL_RESULT_ERROR;
+            }
+
+            if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
+            {
+                LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+
+                AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
+
+                return SQL_RESULT_ERROR;
+            }
+
+            paramTypes = rsp.GetTypeIds();
+
+            for (size_t i = 0; i < paramTypes.size(); ++i)
+                LOG_MSG("[%zu] Parameter type: %u\n", i, paramTypes[i]);
+
+            return SQL_RESULT_SUCCESS;
+        }
     }
 }
 
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d243251/modules/platforms/cpp/odbc/src/utility.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/utility.cpp b/modules/platforms/cpp/odbc/src/utility.cpp
index d4a48af..9763144 100644
--- a/modules/platforms/cpp/odbc/src/utility.cpp
+++ b/modules/platforms/cpp/odbc/src/utility.cpp
@@ -136,6 +136,20 @@ namespace ignite
 
             return res;
         }
+
+        void ReadByteArray(impl::binary::BinaryReaderImpl& reader, std::vector<int8_t>& res)
+        {
+            int32_t len = reader.ReadInt8Array(0, 0);
+
+            if (len > 0)
+            {
+                res.resize(len);
+
+                reader.ReadInt8Array(&res[0], static_cast<int32_t>(res.size()));
+            }
+            else
+                res.clear();
+        }
     }
 }
 


[05/52] ignite git commit: IGNITE-2294: Implemented DML.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySerialiedFieldComparatorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySerialiedFieldComparatorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySerialiedFieldComparatorSelfTest.java
new file mode 100644
index 0000000..c3d1951
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySerialiedFieldComparatorSelfTest.java
@@ -0,0 +1,568 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Unit tests for serialized field comparer.
+ */
+public class BinarySerialiedFieldComparatorSelfTest extends GridCommonAbstractTest {
+    /** Type counter. */
+    private static final AtomicInteger TYPE_CTR = new AtomicInteger();
+
+    /** Single field name. */
+    private static final String FIELD_SINGLE = "single";
+
+    /** Pointers to release. */
+    private final Set<Long> ptrs = new ConcurrentHashSet<>();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        TYPE_CTR.incrementAndGet();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        for (Long ptr : ptrs)
+            GridUnsafe.freeMemory(ptr);
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        return cfg;
+    }
+
+    /**
+     * Test byte fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testByte() throws Exception {
+        checkTwoValues((byte)1, (byte)2);
+    }
+
+    /**
+     * Test boolean fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testBoolean() throws Exception {
+        checkTwoValues(true, false);
+    }
+
+    /**
+     * Test short fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testShort() throws Exception {
+        checkTwoValues((short)1, (short)2);
+    }
+
+    /**
+     * Test char fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testChar() throws Exception {
+        checkTwoValues('a', 'b');
+    }
+
+    /**
+     * Test int fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testInt() throws Exception {
+        checkTwoValues(1, 2);
+    }
+
+    /**
+     * Test long fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLong() throws Exception {
+        checkTwoValues(1L, 2L);
+    }
+
+    /**
+     * Test float fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFloat() throws Exception {
+        checkTwoValues(1.0f, 2.0f);
+    }
+
+    /**
+     * Test double fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDouble() throws Exception {
+        checkTwoValues(1.0d, 2.0d);
+    }
+
+    /**
+     * Test string fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testString() throws Exception {
+        checkTwoValues("str1", "str2");
+    }
+
+    /**
+     * Test date fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDate() throws Exception {
+        long time = System.currentTimeMillis();
+
+        checkTwoValues(new Date(time), new Date(time + 100));
+    }
+
+    /**
+     * Test date fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testTimestamp() throws Exception {
+        long time = System.currentTimeMillis();
+
+        checkTwoValues(new Timestamp(time), new Timestamp(time + 100));
+    }
+
+    /**
+     * Test UUID fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testUuid() throws Exception {
+        checkTwoValues(UUID.randomUUID(), UUID.randomUUID());
+    }
+
+    /**
+     * Test decimal fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDecimal() throws Exception {
+        checkTwoValues(new BigDecimal("12.3E+7"), new BigDecimal("12.4E+7"));
+        checkTwoValues(new BigDecimal("12.3E+7"), new BigDecimal("12.3E+8"));
+    }
+
+    /**
+     * Test object fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testInnerObject() throws Exception {
+        checkTwoValues(new InnerClass(1), new InnerClass(2));
+    }
+
+    /**
+     * Test byte array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testByteArray() throws Exception {
+        checkTwoValues(new byte[] { 1, 2 }, new byte[] { 1, 3 });
+        checkTwoValues(new byte[] { 1, 2 }, new byte[] { 1 });
+        checkTwoValues(new byte[] { 1, 2 }, new byte[] { 3 });
+        checkTwoValues(new byte[] { 1, 2 }, new byte[] { 1, 2, 3 });
+    }
+
+    /**
+     * Test boolean array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testBooleanArray() throws Exception {
+        checkTwoValues(new boolean[] { true, false }, new boolean[] { false, true });
+        checkTwoValues(new boolean[] { true, false }, new boolean[] { true });
+        checkTwoValues(new boolean[] { true, false }, new boolean[] { false });
+        checkTwoValues(new boolean[] { true, false }, new boolean[] { true, false, true });
+    }
+
+    /**
+     * Test short array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testShortArray() throws Exception {
+        checkTwoValues(new short[] { 1, 2 }, new short[] { 1, 3 });
+        checkTwoValues(new short[] { 1, 2 }, new short[] { 1 });
+        checkTwoValues(new short[] { 1, 2 }, new short[] { 3 });
+        checkTwoValues(new short[] { 1, 2 }, new short[] { 1, 2, 3 });
+    }
+
+    /**
+     * Test char array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCharArray() throws Exception {
+        checkTwoValues(new char[] { 1, 2 }, new char[] { 1, 3 });
+        checkTwoValues(new char[] { 1, 2 }, new char[] { 1 });
+        checkTwoValues(new char[] { 1, 2 }, new char[] { 3 });
+        checkTwoValues(new char[] { 1, 2 }, new char[] { 1, 2, 3 });
+    }
+
+    /**
+     * Test int array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testIntArray() throws Exception {
+        checkTwoValues(new int[] { 1, 2 }, new int[] { 1, 3 });
+        checkTwoValues(new int[] { 1, 2 }, new int[] { 1 });
+        checkTwoValues(new int[] { 1, 2 }, new int[] { 3 });
+        checkTwoValues(new int[] { 1, 2 }, new int[] { 1, 2, 3 });
+    }
+
+    /**
+     * Test long array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLongArray() throws Exception {
+        checkTwoValues(new long[] { 1, 2 }, new long[] { 1, 3 });
+        checkTwoValues(new long[] { 1, 2 }, new long[] { 1 });
+        checkTwoValues(new long[] { 1, 2 }, new long[] { 3 });
+        checkTwoValues(new long[] { 1, 2 }, new long[] { 1, 2, 3 });
+    }
+
+    /**
+     * Test float array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFloatArray() throws Exception {
+        checkTwoValues(new float[] { 1.0f, 2.0f }, new float[] { 1.0f, 3.0f });
+        checkTwoValues(new float[] { 1.0f, 2.0f }, new float[] { 1.0f });
+        checkTwoValues(new float[] { 1.0f, 2.0f }, new float[] { 3.0f });
+        checkTwoValues(new float[] { 1.0f, 2.0f }, new float[] { 1.0f, 2.0f, 3.0f });
+    }
+
+    /**
+     * Test double array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDoubleArray() throws Exception {
+        checkTwoValues(new double[] { 1.0d, 2.0d }, new double[] { 1.0d, 3.0d });
+        checkTwoValues(new double[] { 1.0d, 2.0d }, new double[] { 1.0d });
+        checkTwoValues(new double[] { 1.0d, 2.0d }, new double[] { 3.0d });
+        checkTwoValues(new double[] { 1.0d, 2.0d }, new double[] { 1.0d, 2.0d, 3.0d });
+    }
+
+    /**
+     * Test string array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testStringArray() throws Exception {
+        checkTwoValues(new String[] { "a", "b" }, new String[] { "a", "c" });
+        checkTwoValues(new String[] { "a", "b" }, new String[] { "a" });
+        checkTwoValues(new String[] { "a", "b" }, new String[] { "c" });
+        checkTwoValues(new String[] { "a", "b" }, new String[] { "a", "b", "c" });
+    }
+
+    /**
+     * Test date array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDateArray() throws Exception {
+        long curTime = System.currentTimeMillis();
+
+        Date v1 = new Date(curTime);
+        Date v2 = new Date(curTime + 1000);
+        Date v3 = new Date(curTime + 2000);
+
+        checkTwoValues(new Date[] { v1, v2 }, new Date[] { v1, v3 });
+        checkTwoValues(new Date[] { v1, v2 }, new Date[] { v1 });
+        checkTwoValues(new Date[] { v1, v2 }, new Date[] { v3 });
+        checkTwoValues(new Date[] { v1, v2 }, new Date[] { v1, v2, v3 });
+    }
+
+    /**
+     * Test timestamp array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testTimestampArray() throws Exception {
+        long curTime = System.currentTimeMillis();
+
+        Timestamp v1 = new Timestamp(curTime);
+        Timestamp v2 = new Timestamp(curTime + 1000);
+        Timestamp v3 = new Timestamp(curTime + 2000);
+
+        checkTwoValues(new Timestamp[] { v1, v2 }, new Timestamp[] { v1, v3 });
+        checkTwoValues(new Timestamp[] { v1, v2 }, new Timestamp[] { v1 });
+        checkTwoValues(new Timestamp[] { v1, v2 }, new Timestamp[] { v3 });
+        checkTwoValues(new Timestamp[] { v1, v2 }, new Timestamp[] { v1, v2, v3 });
+    }
+
+    /**
+     * Test UUID array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testUuidArray() throws Exception {
+        UUID v1 = UUID.randomUUID();
+        UUID v2 = UUID.randomUUID();
+        UUID v3 = UUID.randomUUID();
+
+        checkTwoValues(new UUID[] { v1, v2 }, new UUID[] { v1, v3 });
+        checkTwoValues(new UUID[] { v1, v2 }, new UUID[] { v1 });
+        checkTwoValues(new UUID[] { v1, v2 }, new UUID[] { v3 });
+        checkTwoValues(new UUID[] { v1, v2 }, new UUID[] { v1, v2, v3 });
+    }
+
+    /**
+     * Test decimal array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDecimalArray() throws Exception {
+        BigDecimal v1 = new BigDecimal("12.3E+7");
+        BigDecimal v2 = new BigDecimal("12.4E+7");
+        BigDecimal v3 = new BigDecimal("12.5E+7");
+
+        checkTwoValues(new BigDecimal[] { v1, v2 }, new BigDecimal[] { v1, v3 });
+        checkTwoValues(new BigDecimal[] { v1, v2 }, new BigDecimal[] { v1 });
+        checkTwoValues(new BigDecimal[] { v1, v2 }, new BigDecimal[] { v3 });
+        checkTwoValues(new BigDecimal[] { v1, v2 }, new BigDecimal[] { v1, v2, v3 });
+
+        v2 = new BigDecimal("12.3E+8");
+        v3 = new BigDecimal("12.3E+9");
+
+        checkTwoValues(new BigDecimal[] { v1, v2 }, new BigDecimal[] { v1, v3 });
+        checkTwoValues(new BigDecimal[] { v1, v2 }, new BigDecimal[] { v1 });
+        checkTwoValues(new BigDecimal[] { v1, v2 }, new BigDecimal[] { v3 });
+        checkTwoValues(new BigDecimal[] { v1, v2 }, new BigDecimal[] { v1, v2, v3 });
+    }
+
+    /**
+     * Test object array fields.
+     *
+     * @throws Exception If failed.
+     */
+    public void testInnerObjectArray() throws Exception {
+        InnerClass v1 = new InnerClass(1);
+        InnerClass v2 = new InnerClass(2);
+        InnerClass v3 = new InnerClass(3);
+
+        checkTwoValues(new InnerClass[] { v1, v2 }, new InnerClass[] { v1, v3 });
+        checkTwoValues(new InnerClass[] { v1, v2 }, new InnerClass[] { v1 });
+        checkTwoValues(new InnerClass[] { v1, v2 }, new InnerClass[] { v3 });
+        checkTwoValues(new InnerClass[] { v1, v2 }, new InnerClass[] { v1, v2, v3 });
+    }
+
+    /**
+     * Check two different not-null values.
+     *
+     * @throws Exception If failed.
+     */
+    public void checkTwoValues(Object val1, Object val2) throws Exception {
+        checkTwoValues(val1, val2, false, false);
+        checkTwoValues(val1, val2, false, true);
+        checkTwoValues(val1, val2, true, false);
+        checkTwoValues(val1, val2, true, true);
+    }
+
+    /**
+     * Check two different not-null values.
+     *
+     * @param val1 Value 1.
+     * @param val2 Value 2.
+     * @param offheap1 Offheap flag 1.
+     * @param offheap2 Offheap flag 2.
+     * @throws Exception If failed.
+     */
+    public void checkTwoValues(Object val1, Object val2, boolean offheap1, boolean offheap2) throws Exception {
+        assertNotNull(val1);
+        assertNotNull(val2);
+
+        compareSingle(convert(buildSingle(val1), offheap1), convert(buildSingle(val1), offheap2), true);
+        compareSingle(convert(buildSingle(val1), offheap1), convert(buildSingle(val2), offheap2), false);
+        compareSingle(convert(buildSingle(val1), offheap1), convert(buildSingle(null), offheap2), false);
+        compareSingle(convert(buildSingle(val1), offheap1), convert(buildEmpty(), offheap2), false);
+
+        compareSingle(convert(buildSingle(val2), offheap1), convert(buildSingle(val1), offheap2), false);
+        compareSingle(convert(buildSingle(val2), offheap1), convert(buildSingle(val2), offheap2), true);
+        compareSingle(convert(buildSingle(val2), offheap1), convert(buildSingle(null), offheap2), false);
+        compareSingle(convert(buildSingle(val2), offheap1), convert(buildEmpty(), offheap2), false);
+
+        compareSingle(convert(buildSingle(null), offheap1), convert(buildSingle(val1), offheap2), false);
+        compareSingle(convert(buildSingle(null), offheap1), convert(buildSingle(val2), offheap2), false);
+        compareSingle(convert(buildSingle(null), offheap1), convert(buildSingle(null), offheap2), true);
+        compareSingle(convert(buildSingle(null), offheap1), convert(buildEmpty(), offheap2), true);
+
+        compareSingle(convert(buildEmpty(), offheap1), convert(buildSingle(val1), offheap2), false);
+        compareSingle(convert(buildEmpty(), offheap1), convert(buildSingle(val2), offheap2), false);
+        compareSingle(convert(buildEmpty(), offheap1), convert(buildSingle(null), offheap2), true);
+        compareSingle(convert(buildEmpty(), offheap1), convert(buildEmpty(), offheap2), true);
+    }
+
+    /**
+     * Compare single field.
+     *
+     * @param first First object.
+     * @param second Second object.
+     * @param expRes Expected result.
+     */
+    private void compareSingle(BinaryObjectExImpl first, BinaryObjectExImpl second, boolean expRes) {
+        BinarySerializedFieldComparator firstComp = first.createFieldComparator();
+        BinarySerializedFieldComparator secondComp = second.createFieldComparator();
+
+        // Compare expected result.
+        firstComp.findField(singleFieldOrder(first));
+        secondComp.findField(singleFieldOrder(second));
+
+        assertEquals(expRes, BinarySerializedFieldComparator.equals(firstComp, secondComp));
+    }
+
+    /**
+     * Get single field order.
+     *
+     * @param obj Object.
+     * @return Order.
+     */
+    private int singleFieldOrder(BinaryObjectExImpl obj) {
+        return obj.hasField(FIELD_SINGLE) ? 0 : BinarySchema.ORDER_NOT_FOUND;
+    }
+
+    /**
+     * Convert binary object to it's final state.
+     *
+     * @param obj Object.
+     * @param offheap Offheap flag.
+     * @return Result.
+     */
+    private BinaryObjectExImpl convert(BinaryObjectExImpl obj, boolean offheap) {
+        if (offheap) {
+            byte[] arr = obj.array();
+
+            long ptr = GridUnsafe.allocateMemory(arr.length);
+
+            ptrs.add(ptr);
+
+            GridUnsafe.copyMemory(arr, GridUnsafe.BYTE_ARR_OFF, null, ptr, arr.length);
+
+            obj = new BinaryObjectOffheapImpl(obj.context(), ptr, 0, obj.array().length);
+        }
+
+        return obj;
+    }
+
+    /**
+     * Build object with a single field.
+     *
+     * @param val Value.
+     * @return Result.
+     */
+    private BinaryObjectImpl buildSingle(Object val) {
+        return build(FIELD_SINGLE, val);
+    }
+
+    /**
+     * Build empty object.
+     *
+     * @return Empty object.
+     */
+    private BinaryObjectImpl buildEmpty() {
+        return build();
+    }
+
+    /**
+     * Build object.
+     *
+     * @param parts Parts.
+     * @return Result.
+     */
+    private BinaryObjectImpl build(Object... parts) {
+        String typeName = "Type" + TYPE_CTR.get();
+
+        BinaryObjectBuilder builder = grid().binary().builder(typeName);
+
+        if (!F.isEmpty(parts)) {
+            for (int i = 0; i < parts.length; )
+                builder.setField((String)parts[i++], parts[i++]);
+        }
+
+        return (BinaryObjectImpl) builder.build();
+    }
+
+    /**
+     * Inner class.
+     */
+    @SuppressWarnings("unused")
+    private static class InnerClass {
+        /** Value. */
+        private int val;
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        public InnerClass(int val) {
+            this.val = val;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
index 150c245..2a177ff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
@@ -25,6 +25,7 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
@@ -36,17 +37,22 @@ import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteBinary;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryArrayIdentityResolver;
 import org.apache.ignite.binary.BinaryNameMapper;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryIdentityResolver;
 import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.binary.BinaryWriter;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
@@ -55,9 +61,12 @@ import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.BinaryObjectImpl;
 import org.apache.ignite.internal.binary.BinaryObjectOffheapImpl;
+import org.apache.ignite.binary.BinaryFieldIdentityResolver;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
+import org.apache.ignite.internal.processors.cache.MapCacheStoreStrategy;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -103,13 +112,79 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
 
         cfg.setDiscoverySpi(disco);
 
+        CacheConfiguration cacheCfg = createCacheConfig();
+
+        cacheCfg.setCacheStoreFactory(singletonFactory(new TestStore()));
+
+        CacheConfiguration binKeysCacheCfg = createCacheConfig();
+
+        binKeysCacheCfg.setCacheStoreFactory(singletonFactory(new MapCacheStoreStrategy.MapCacheStore()));
+        binKeysCacheCfg.setStoreKeepBinary(true);
+        binKeysCacheCfg.setName("BinKeysCache");
+
+        cfg.setCacheConfiguration(cacheCfg, binKeysCacheCfg);
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        List<BinaryTypeConfiguration> binTypes = new ArrayList<>();
+
+        binTypes.add(new BinaryTypeConfiguration() {{
+            setTypeName("ArrayHashedKey");
+
+            setIdentityResolver(new BinaryArrayIdentityResolver());
+        }});
+
+        binTypes.add(new BinaryTypeConfiguration() {{
+            setTypeName("FieldsHashedKey");
+
+            BinaryFieldIdentityResolver id = new BinaryFieldIdentityResolver();
+            id.setFieldNames("fld1", "fld3");
+
+            setIdentityResolver(id);
+        }});
+
+        binTypes.add(new BinaryTypeConfiguration() {{
+            setTypeName("CustomHashedKey");
+
+            setIdentityResolver(new IdentityResolver());
+        }});
+
+        binTypes.add(new BinaryTypeConfiguration() {{
+            setTypeName(ComplexBinaryFieldsListHashedKey.class.getName());
+
+            BinaryFieldIdentityResolver id = new BinaryFieldIdentityResolver();
+
+            id.setFieldNames("secondField", "thirdField");
+
+            setIdentityResolver(id);
+        }});
+
+        BinaryConfiguration binCfg = new BinaryConfiguration();
+        binCfg.setTypeConfigurations(binTypes);
+
+        cfg.setBinaryConfiguration(binCfg);
+
+        CacheKeyConfiguration arrayHashCfg = new CacheKeyConfiguration("ArrayHashedKey", "fld1");
+        CacheKeyConfiguration fieldsHashCfg = new CacheKeyConfiguration("FieldsHashedKey", "fld1");
+        CacheKeyConfiguration customHashCfg = new CacheKeyConfiguration("CustomHashedKey", "fld1");
+
+        cfg.setCacheKeyConfiguration(arrayHashCfg, fieldsHashCfg, customHashCfg);
+
+        GridCacheBinaryObjectsAbstractSelfTest.cfg = cfg;
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache configuration with basic settings.
+     */
+    @SuppressWarnings("unchecked")
+    private CacheConfiguration createCacheConfig() {
         CacheConfiguration cacheCfg = new CacheConfiguration();
 
         cacheCfg.setCacheMode(cacheMode());
         cacheCfg.setAtomicityMode(atomicityMode());
         cacheCfg.setNearConfiguration(nearConfiguration());
         cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-        cacheCfg.setCacheStoreFactory(singletonFactory(new TestStore()));
         cacheCfg.setReadThrough(true);
         cacheCfg.setWriteThrough(true);
         cacheCfg.setLoadPreviousValue(true);
@@ -120,13 +195,7 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
             cacheCfg.setOffHeapMaxMemory(0);
         }
 
-        cfg.setCacheConfiguration(cacheCfg);
-
-        cfg.setMarshaller(new BinaryMarshaller());
-
-        this.cfg = cfg;
-
-        return cfg;
+        return cacheCfg;
     }
 
     /**
@@ -924,6 +993,125 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
     }
 
     /**
+     *
+     */
+    @SuppressWarnings("unchecked")
+    public void testCrossFormatObjectsIdentity() {
+        IgniteCache c = binKeysCache();
+
+        c.put(new ComplexBinaryFieldsListHashedKey(), "zzz");
+
+        // Now let's build an identical key for get
+        BinaryObjectBuilder bldr = grid(0).binary().builder(ComplexBinaryFieldsListHashedKey.class.getName());
+
+        bldr.setField("firstField", 365);
+        bldr.setField("secondField", "value");
+        bldr.setField("thirdField", 0x1020304050607080L);
+
+        BinaryObject binKey = bldr.build();
+
+        assertEquals("zzz", c.get(binKey));
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("unchecked")
+    public void testPutWithArrayHashing() {
+        IgniteCache c = binKeysCache();
+
+        {
+            BinaryObjectBuilder bldr = grid(0).binary().builder("ArrayHashedKey");
+
+            BinaryObject binKey = bldr.setField("fld1", 5).setField("fld2", 1).setField("fld3", "abc").build();
+
+            c.put(binKey, "zzz");
+        }
+
+        // Now let's build an identical key for get.
+        {
+            BinaryObjectBuilder bldr = grid(0).binary().builder("ArrayHashedKey");
+
+            BinaryObject binKey = bldr.setField("fld1", 5).setField("fld2", 1).setField("fld3", "abc").build();
+
+            assertEquals("zzz", c.get(binKey));
+        }
+
+        // Now let's build not identical key for get.
+        {
+            BinaryObjectBuilder bldr = grid(0).binary().builder("ArrayHashedKey");
+
+            BinaryObject binKey = bldr.setField("fld1", 5).setField("fld2", 100).setField("fld3", "abc").build();
+
+            assertNull(c.get(binKey));
+        }
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("unchecked")
+    public void testPutWithFieldsHashing() {
+        IgniteCache c = binKeysCache();
+
+        {
+            BinaryObjectBuilder bldr = grid(0).binary().builder("FieldsHashedKey");
+
+            bldr.setField("fld1", 5);
+            bldr.setField("fld2", 1);
+            bldr.setField("fld3", "abc");
+
+            BinaryObject binKey = bldr.build();
+
+            c.put(binKey, "zzz");
+        }
+
+        // Now let's build an identical key for get
+        {
+            BinaryObjectBuilder bldr = grid(0).binary().builder("FieldsHashedKey");
+
+            bldr.setField("fld1", 5);
+            bldr.setField("fld2", 100); // This one does not participate in hashing
+            bldr.setField("fld3", "abc");
+
+            BinaryObject binKey = bldr.build();
+
+            assertEquals("zzz", c.get(binKey));
+        }
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("unchecked")
+    public void testPutWithCustomHashing() {
+        IgniteCache c = binKeysCache();
+
+        {
+            BinaryObjectBuilder bldr = grid(0).binary().builder("CustomHashedKey");
+
+            bldr.setField("fld1", 5);
+            bldr.setField("fld2", "abc");
+
+            BinaryObject binKey = bldr.build();
+
+            c.put(binKey, "zzz");
+        }
+
+        // Now let's build an identical key for get
+        {
+            BinaryObjectBuilder bldr = grid(0).binary().builder("CustomHashedKey");
+
+            bldr.setField("fld1", 5);
+            bldr.setField("fld2", "xxx");
+
+            BinaryObject binKey = bldr.build();
+
+            assertEquals("zzz", c.get(binKey));
+        }
+    }
+
+    /**
      * @throws Exception if failed.
      */
     public void testKeepBinaryTxOverwrite() throws Exception {
@@ -1034,6 +1222,13 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
     }
 
     /**
+     * @return Cache tuned to utilize classless binary objects as keys.
+     */
+    private <K, V> IgniteCache<K, V> binKeysCache() {
+        return ignite(0).cache("BinKeysCache").withKeepBinary();
+    }
+
+    /**
      * @param key Key.
      * @throws Exception If failed.
      */
@@ -1221,4 +1416,53 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
             // No-op.
         }
     }
+
+    /**
+     *
+     */
+    private final static class IdentityResolver implements BinaryIdentityResolver {
+        /** {@inheritDoc} */
+        @Override public int hashCode(BinaryObject builder) {
+            return (Integer) builder.field("fld1") * 31 / 5;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(BinaryObject o1, BinaryObject o2) {
+            return o1 == o2 || (o1 != null && o2 != null && F.eq(o1.field("fld1"), o2.field("fld1")));
+
+        }
+    }
+
+    /**
+     * Key to test puts and gets with
+     */
+    @SuppressWarnings({"ConstantConditions", "unused"})
+    private final static class ComplexBinaryFieldsListHashedKey {
+        /** */
+        private final Integer firstField = 1;
+
+        /** */
+        private final String secondField = "value";
+
+        /** */
+        private final Long thirdField = 0x1020304050607080L;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            ComplexBinaryFieldsListHashedKey that = (ComplexBinaryFieldsListHashedKey) o;
+
+            return secondField.equals(that.secondField) &&
+                thirdField.equals(that.thirdField);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = secondField.hashCode();
+            res = 31 * res + thirdField.hashCode();
+            return res;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
index c1d9974..3496dbf 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
@@ -18,19 +18,23 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.binary.BinaryArrayIdentityResolverSelfTest;
 import org.apache.ignite.internal.binary.BinaryBasicIdMapperSelfTest;
 import org.apache.ignite.internal.binary.BinaryBasicNameMapperSelfTest;
 import org.apache.ignite.internal.binary.BinaryConfigurationConsistencySelfTest;
 import org.apache.ignite.internal.binary.BinaryEnumsSelfTest;
+import org.apache.ignite.internal.binary.BinaryFieldIdentityResolverSelfTest;
 import org.apache.ignite.internal.binary.BinaryFieldsHeapSelfTest;
 import org.apache.ignite.internal.binary.BinaryFieldsOffheapSelfTest;
 import org.apache.ignite.internal.binary.BinaryFooterOffsetsHeapSelfTest;
 import org.apache.ignite.internal.binary.BinaryFooterOffsetsOffheapSelfTest;
+import org.apache.ignite.internal.binary.BinaryIdentityResolverConfigurationSelfTest;
 import org.apache.ignite.internal.binary.BinaryMarshallerSelfTest;
 import org.apache.ignite.internal.binary.BinaryObjectBuilderAdditionalSelfTest;
 import org.apache.ignite.internal.binary.BinaryObjectBuilderDefaultMappersSelfTest;
 import org.apache.ignite.internal.binary.BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest;
 import org.apache.ignite.internal.binary.BinaryObjectToStringSelfTest;
+import org.apache.ignite.internal.binary.BinarySerialiedFieldComparatorSelfTest;
 import org.apache.ignite.internal.binary.BinarySimpleNameTestPropertySelfTest;
 import org.apache.ignite.internal.binary.BinaryTreeSelfTest;
 import org.apache.ignite.internal.binary.GridBinaryAffinityKeySelfTest;
@@ -89,6 +93,12 @@ public class IgniteBinaryObjectsTestSuite extends TestSuite {
 
         suite.addTestSuite(BinaryTreeSelfTest.class);
         suite.addTestSuite(BinaryMarshallerSelfTest.class);
+
+        suite.addTestSuite(BinarySerialiedFieldComparatorSelfTest.class);
+        suite.addTestSuite(BinaryArrayIdentityResolverSelfTest.class);
+        suite.addTestSuite(BinaryFieldIdentityResolverSelfTest.class);
+        suite.addTestSuite(BinaryIdentityResolverConfigurationSelfTest.class);
+
         suite.addTestSuite(BinaryConfigurationConsistencySelfTest.class);
         suite.addTestSuite(GridBinaryMarshallerCtxDisabledSelfTest.class);
         suite.addTestSuite(BinaryObjectBuilderDefaultMappersSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
new file mode 100644
index 0000000..8dcba2f
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
@@ -0,0 +1,253 @@
+/*
+ * 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.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Two step map-reduce style query.
+ */
+public class GridCacheTwoStepQuery {
+    /** */
+    public static final int DFLT_PAGE_SIZE = 1000;
+
+    /** */
+    @GridToStringInclude
+    private List<GridCacheSqlQuery> mapQrys = new ArrayList<>();
+
+    /** */
+    @GridToStringInclude
+    private GridCacheSqlQuery rdc;
+
+    /** */
+    private int pageSize = DFLT_PAGE_SIZE;
+
+    /** */
+    private boolean explain;
+
+    /** */
+    private Collection<String> spaces;
+
+    /** */
+    private Set<String> schemas;
+
+    /** */
+    private Set<String> tbls;
+
+    /** */
+    private boolean distributedJoins;
+
+    /** */
+    private boolean skipMergeTbl;
+
+    /** */
+    private List<Integer> caches;
+
+    /** */
+    private List<Integer> extraCaches;
+
+    /**
+     * @param schemas Schema names in query.
+     * @param tbls Tables in query.
+     */
+    public GridCacheTwoStepQuery(Set<String> schemas, Set<String> tbls) {
+        this.schemas = schemas;
+        this.tbls = tbls;
+    }
+
+    /**
+     * Specify if distributed joins are enabled for this query.
+     *
+     * @param distributedJoins Distributed joins enabled.
+     */
+    public void distributedJoins(boolean distributedJoins) {
+        this.distributedJoins = distributedJoins;
+    }
+
+    /**
+     * Check if distributed joins are enabled for this query.
+     *
+     * @return {@code true} If distributed joind enabled.
+     */
+    public boolean distributedJoins() {
+        return distributedJoins;
+    }
+
+
+    /**
+     * @return {@code True} if reduce query can skip merge table creation and get data directly from merge index.
+     */
+    public boolean skipMergeTable() {
+        return skipMergeTbl;
+    }
+
+    /**
+     * @param skipMergeTbl Skip merge table.
+     */
+    public void skipMergeTable(boolean skipMergeTbl) {
+        this.skipMergeTbl = skipMergeTbl;
+    }
+
+    /**
+     * @return If this is explain query.
+     */
+    public boolean explain() {
+        return explain;
+    }
+
+    /**
+     * @param explain If this is explain query.
+     */
+    public void explain(boolean explain) {
+        this.explain = explain;
+    }
+
+    /**
+     * @param pageSize Page size.
+     */
+    public void pageSize(int pageSize) {
+        this.pageSize = pageSize;
+    }
+
+    /**
+     * @return Page size.
+     */
+    public int pageSize() {
+        return pageSize;
+    }
+
+    /**
+     * @param qry SQL Query.
+     * @return {@code this}.
+     */
+    public GridCacheTwoStepQuery addMapQuery(GridCacheSqlQuery qry) {
+        mapQrys.add(qry);
+
+        return this;
+    }
+
+    /**
+     * @return Reduce query.
+     */
+    public GridCacheSqlQuery reduceQuery() {
+        return rdc;
+    }
+
+    /**
+     * @param rdc Reduce query.
+     */
+    public void reduceQuery(GridCacheSqlQuery rdc) {
+        this.rdc = rdc;
+    }
+
+    /**
+     * @return Map queries.
+     */
+    public List<GridCacheSqlQuery> mapQueries() {
+        return mapQrys;
+    }
+
+    /**
+     * @return Caches.
+     */
+    public List<Integer> caches() {
+        return caches;
+    }
+
+    /**
+     * @param caches Caches.
+     */
+    public void caches(List<Integer> caches) {
+        this.caches = caches;
+    }
+
+    /**
+     * @return Caches.
+     */
+    public List<Integer> extraCaches() {
+        return extraCaches;
+    }
+
+    /**
+     * @param extraCaches Caches.
+     */
+    public void extraCaches(List<Integer> extraCaches) {
+        this.extraCaches = extraCaches;
+    }
+
+    /**
+     * @return Spaces.
+     */
+    public Collection<String> spaces() {
+        return spaces;
+    }
+
+    /**
+     * @param spaces Spaces.
+     */
+    public void spaces(Collection<String> spaces) {
+        this.spaces = spaces;
+    }
+
+    /**
+     * @return Schemas.
+     */
+    public Set<String> schemas() {
+        return schemas;
+    }
+
+    /**
+     * @param args New arguments to copy with.
+     * @return Copy.
+     */
+    public GridCacheTwoStepQuery copy(Object[] args) {
+        assert !explain;
+
+        GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(schemas, tbls);
+
+        cp.caches = caches;
+        cp.extraCaches = extraCaches;
+        cp.spaces = spaces;
+        cp.rdc = rdc.copy(args);
+        cp.skipMergeTbl = skipMergeTbl;
+        cp.pageSize = pageSize;
+        cp.distributedJoins = distributedJoins;
+
+        for (int i = 0; i < mapQrys.size(); i++)
+            cp.mapQrys.add(mapQrys.get(i).copy(args));
+
+        return cp;
+    }
+
+    /**
+     * @return Tables.
+     */
+    public Set<String> tables() {
+        return tbls;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridCacheTwoStepQuery.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
new file mode 100644
index 0000000..7634965
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -0,0 +1,1027 @@
+/*
+ * 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.h2;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.EntryProcessorResult;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryArrayIdentityResolver;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.processors.cache.CacheOperationContext;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheUtils;
+import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
+import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
+import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
+import org.apache.ignite.internal.processors.query.h2.dml.KeyValueSupplier;
+import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
+import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
+import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
+import org.apache.ignite.internal.util.lang.IgniteSingletonIterator;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.h2.command.Prepared;
+import org.h2.jdbc.JdbcPreparedStatement;
+import org.h2.table.Column;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentHashMap8;
+
+import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.createJdbcSqlException;
+
+/**
+ *
+ */
+public class DmlStatementsProcessor {
+    /** Default number of attempts to re-run DELETE and UPDATE queries in case of concurrent modifications of values. */
+    private final static int DFLT_DML_RERUN_ATTEMPTS = 4;
+
+    /** Indexing. */
+    private final IgniteH2Indexing indexing;
+
+    /** Set of binary type ids for which warning about missing identity in configuration has been printed. */
+    private final static Set<Integer> WARNED_TYPES =
+        Collections.newSetFromMap(new ConcurrentHashMap8<Integer, Boolean>());
+
+    /** Default size for update plan cache. */
+    private static final int PLAN_CACHE_SIZE = 1024;
+
+    /** Update plans cache. */
+    private final ConcurrentMap<String, ConcurrentMap<String, UpdatePlan>> planCache = new ConcurrentHashMap<>();
+
+    /** Dummy metadata for update result. */
+    private final static List<GridQueryFieldMetadata> UPDATE_RESULT_META = Collections.<GridQueryFieldMetadata>
+        singletonList(new IgniteH2Indexing.SqlFieldMetadata(null, null, "UPDATED", Long.class.getName()));
+
+    /**
+     * @param indexing indexing.
+     */
+    DmlStatementsProcessor(IgniteH2Indexing indexing) {
+        this.indexing = indexing;
+    }
+
+    /**
+     * Execute DML statement, possibly with few re-attempts in case of concurrent data modifications.
+     *
+     * @param spaceName Space name.
+     * @param stmt JDBC statement.
+     * @param fieldsQry Original query.
+     * @param loc Query locality flag.
+     * @param filters Space name and key filter.
+     * @param cancel Cancel.
+     * @return Update result (modified items count and failed keys).
+     * @throws IgniteCheckedException if failed.
+     */
+    private long updateSqlFields(String spaceName, PreparedStatement stmt, SqlFieldsQuery fieldsQry,
+        boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
+        Object[] errKeys = null;
+
+        long items = 0;
+
+        UpdatePlan plan = getPlanForStatement(spaceName, stmt, null);
+
+        for (int i = 0; i < DFLT_DML_RERUN_ATTEMPTS; i++) {
+            UpdateResult r = executeUpdateStatement(plan.tbl.rowDescriptor().context(), stmt, fieldsQry, loc, filters,
+                cancel, errKeys);
+
+            if (F.isEmpty(r.errKeys))
+                return r.cnt + items;
+            else {
+                items += r.cnt;
+                errKeys = r.errKeys;
+            }
+        }
+
+        throw new IgniteSQLException("Failed to update or delete some keys: " + Arrays.deepToString(errKeys),
+            IgniteQueryErrorCode.CONCURRENT_UPDATE);
+    }
+
+    /**
+     * @param spaceName Space name.
+     * @param stmt Prepared statement.
+     * @param fieldsQry Initial query.
+     * @param cancel Query cancel.
+     * @return Update result wrapped into {@link GridQueryFieldsResult}
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings("unchecked")
+    QueryCursorImpl<List<?>> updateSqlFieldsTwoStep(String spaceName, PreparedStatement stmt,
+        SqlFieldsQuery fieldsQry, GridQueryCancel cancel) throws IgniteCheckedException {
+        long res = updateSqlFields(spaceName, stmt, fieldsQry, false, null, cancel);
+
+        return cursorForUpdateResult(res);
+    }
+
+    /**
+     * Execute DML statement on local cache.
+     * @param spaceName Space name.
+     * @param stmt Prepared statement.
+     * @param filters Space name and key filter.
+     * @param cancel Query cancel.
+     * @return Update result wrapped into {@link GridQueryFieldsResult}
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings("unchecked")
+    GridQueryFieldsResult updateLocalSqlFields(String spaceName, PreparedStatement stmt,
+        SqlFieldsQuery fieldsQry, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
+        long res = updateSqlFields(spaceName, stmt, fieldsQry, true, filters, cancel);
+
+        return new GridQueryFieldsResultAdapter(UPDATE_RESULT_META,
+            new IgniteSingletonIterator(Collections.singletonList(res)));
+    }
+
+    /**
+     * Actually perform SQL DML operation locally.
+     * @param cctx Cache context.
+     * @param prepStmt Prepared statement for DML query.
+     * @param filters Space name and key filter.
+     * @param failedKeys Keys to restrict UPDATE and DELETE operations with. Null or empty array means no restriction.
+     * @return Pair [number of successfully processed items; keys that have failed to be processed]
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    private UpdateResult executeUpdateStatement(final GridCacheContext cctx, PreparedStatement prepStmt,
+        SqlFieldsQuery fieldsQry, boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel, Object[] failedKeys)
+        throws IgniteCheckedException {
+        Integer errKeysPos = null;
+
+        if (!F.isEmpty(failedKeys))
+            errKeysPos = F.isEmpty(fieldsQry.getArgs()) ? 1 : fieldsQry.getArgs().length + 1;
+
+        UpdatePlan plan = getPlanForStatement(cctx.name(), prepStmt, errKeysPos);
+
+        Object[] params = fieldsQry.getArgs();
+
+        if (plan.fastUpdateArgs != null) {
+            assert F.isEmpty(failedKeys) && errKeysPos == null;
+
+            return new UpdateResult(doSingleUpdate(plan, params), X.EMPTY_OBJECT_ARRAY);
+        }
+
+        assert !F.isEmpty(plan.selectQry);
+
+        QueryCursorImpl<List<?>> cur;
+
+        // Do a two-step query only if locality flag is not set AND if plan's SELECT corresponds to an actual
+        // subquery and not some dummy stuff like "select 1, 2, 3;"
+        if (!loc && !plan.isLocSubqry) {
+            SqlFieldsQuery newFieldsQry = new SqlFieldsQuery(plan.selectQry, fieldsQry.isCollocated())
+                .setArgs(params)
+                .setDistributedJoins(fieldsQry.isDistributedJoins())
+                .setEnforceJoinOrder(fieldsQry.isEnforceJoinOrder())
+                .setLocal(fieldsQry.isLocal())
+                .setPageSize(fieldsQry.getPageSize())
+                .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
+
+            cur = (QueryCursorImpl<List<?>>) indexing.queryTwoStep(cctx, newFieldsQry, cancel);
+        }
+        else {
+            final GridQueryFieldsResult res = indexing.queryLocalSqlFields(cctx.name(), plan.selectQry, F.asList(params),
+                filters, fieldsQry.isEnforceJoinOrder(), fieldsQry.getTimeout(), cancel);
+
+            cur = new QueryCursorImpl<>(new Iterable<List<?>>() {
+                @Override public Iterator<List<?>> iterator() {
+                    try {
+                        return new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepBinary());
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new IgniteException(e);
+                    }
+                }
+            }, cancel);
+
+            cur.fieldsMeta(res.metaData());
+        }
+
+        int pageSize = loc ? 0 : fieldsQry.getPageSize();
+
+        switch (plan.mode) {
+            case MERGE:
+                return new UpdateResult(doMerge(plan, cur, pageSize), X.EMPTY_OBJECT_ARRAY);
+
+            case INSERT:
+                return new UpdateResult(doInsert(plan, cur, pageSize), X.EMPTY_OBJECT_ARRAY);
+
+            case UPDATE:
+                return doUpdate(plan, cur, pageSize);
+
+            case DELETE:
+                return doDelete(cctx, cur, pageSize);
+
+            default:
+                throw new IgniteSQLException("Unexpected DML operation [mode=" + plan.mode + ']',
+                    IgniteQueryErrorCode.UNEXPECTED_OPERATION);
+        }
+    }
+
+    /**
+     * Generate SELECT statements to retrieve data for modifications from and find fast UPDATE or DELETE args,
+     * if available.
+     * @param spaceName Space name.
+     * @param prepStmt JDBC statement.
+     * @return Update plan.
+     */
+    @SuppressWarnings({"unchecked", "ConstantConditions"})
+    private UpdatePlan getPlanForStatement(String spaceName, PreparedStatement prepStmt,
+        @Nullable Integer errKeysPos) throws IgniteCheckedException {
+        Prepared p = GridSqlQueryParser.prepared((JdbcPreparedStatement) prepStmt);
+
+        spaceName = F.isEmpty(spaceName) ? "default" : spaceName;
+
+        ConcurrentMap<String, UpdatePlan> spacePlans = planCache.get(spaceName);
+
+        if (spacePlans == null) {
+            spacePlans = new GridBoundedConcurrentLinkedHashMap<>(PLAN_CACHE_SIZE);
+
+            spacePlans = U.firstNotNull(planCache.putIfAbsent(spaceName, spacePlans), spacePlans);
+        }
+
+        // getSQL returns field value, so it's fast
+        // Don't look for re-runs in cache, we don't cache them
+        UpdatePlan res = (errKeysPos == null ? spacePlans.get(p.getSQL()) : null);
+
+        if (res != null)
+            return res;
+
+        res = UpdatePlanBuilder.planForStatement(p, errKeysPos);
+
+        // Don't cache re-runs
+        if (errKeysPos == null)
+            return U.firstNotNull(spacePlans.putIfAbsent(p.getSQL(), res), res);
+        else
+            return res;
+    }
+
+    /**
+     * Perform single cache operation based on given args.
+     * @param params Query parameters.
+     * @return 1 if an item was affected, 0 otherwise.
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings({"unchecked", "ConstantConditions"})
+    private static long doSingleUpdate(UpdatePlan plan, Object[] params) throws IgniteCheckedException {
+        GridCacheContext cctx = plan.tbl.rowDescriptor().context();
+
+        FastUpdateArguments singleUpdate = plan.fastUpdateArgs;
+
+        assert singleUpdate != null;
+
+        int res;
+
+        Object key = singleUpdate.key.apply(params);
+        Object val = singleUpdate.val.apply(params);
+        Object newVal = singleUpdate.newVal.apply(params);
+
+        if (newVal != null) { // Single item UPDATE
+            if (val == null) // No _val bound in source query
+                res = cctx.cache().replace(key, newVal) ? 1 : 0;
+            else
+                res = cctx.cache().replace(key, val, newVal) ? 1 : 0;
+        }
+        else { // Single item DELETE
+            if (val == null) // No _val bound in source query
+                res = cctx.cache().remove(key) ? 1 : 0;
+            else
+                res = cctx.cache().remove(key, val) ? 1 : 0;
+        }
+
+        return res;
+    }
+
+    /**
+     * Perform DELETE operation on top of results of SELECT.
+     * @param cctx Cache context.
+     * @param cursor SELECT results.
+     * @param pageSize Batch size for streaming, anything <= 0 for single page operations.
+     * @return Results of DELETE (number of items affected AND keys that failed to be updated).
+     */
+    @SuppressWarnings({"unchecked", "ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
+    private UpdateResult doDelete(GridCacheContext cctx, QueryCursorImpl<List<?>> cursor, int pageSize)
+        throws IgniteCheckedException {
+        // With DELETE, we have only two columns - key and value.
+        long res = 0;
+
+        CacheOperationContext opCtx = cctx.operationContextPerCall();
+
+        // Force keepBinary for operation context to avoid binary deserialization inside entry processor
+        if (cctx.binaryMarshaller()) {
+            CacheOperationContext newOpCtx = null;
+
+            if (opCtx == null)
+                // Mimics behavior of GridCacheAdapter#keepBinary and GridCacheProxyImpl#keepBinary
+                newOpCtx = new CacheOperationContext(false, null, true, null, false, null);
+            else if (!opCtx.isKeepBinary())
+                newOpCtx = opCtx.keepBinary();
+
+            if (newOpCtx != null)
+                cctx.operationContextPerCall(newOpCtx);
+        }
+
+        // Keys that failed to DELETE due to concurrent updates.
+        List<Object> failedKeys = new ArrayList<>();
+
+        SQLException resEx = null;
+
+        try {
+            Iterator<List<?>> it = cursor.iterator();
+            Map<Object, EntryProcessor<Object, Object, Boolean>> rows = new LinkedHashMap<>();
+
+            while (it.hasNext()) {
+                List<?> e = it.next();
+                if (e.size() != 2) {
+                    U.warn(indexing.getLogger(), "Invalid row size on DELETE - expected 2, got " + e.size());
+                    continue;
+                }
+
+                rows.put(e.get(0), new ModifyingEntryProcessor(e.get(1), RMV));
+
+                if ((pageSize > 0 && rows.size() == pageSize) || (!it.hasNext())) {
+                    PageProcessingResult pageRes = processPage(cctx, rows);
+
+                    res += pageRes.cnt;
+
+                    failedKeys.addAll(F.asList(pageRes.errKeys));
+
+                    if (pageRes.ex != null) {
+                        if (resEx == null)
+                            resEx = pageRes.ex;
+                        else
+                            resEx.setNextException(pageRes.ex);
+                    }
+
+                    if (it.hasNext())
+                        rows.clear(); // No need to clear after the last batch.
+                }
+            }
+
+            if (resEx != null) {
+                if (!F.isEmpty(failedKeys)) {
+                    // Don't go for a re-run if processing of some keys yielded exceptions and report keys that
+                    // had been modified concurrently right away.
+                    String msg = "Failed to DELETE some keys because they had been modified concurrently " +
+                        "[keys=" + failedKeys + ']';
+
+                    SQLException conEx = createJdbcSqlException(msg, IgniteQueryErrorCode.CONCURRENT_UPDATE);
+
+                    conEx.setNextException(resEx);
+
+                    resEx = conEx;
+                }
+
+                throw new IgniteSQLException(resEx);
+            }
+        }
+        finally {
+            cctx.operationContextPerCall(opCtx);
+        }
+
+        return new UpdateResult(res, failedKeys.toArray());
+    }
+
+    /**
+     * Perform UPDATE operation on top of results of SELECT.
+     * @param cursor SELECT results.
+     * @param pageSize Batch size for streaming, anything <= 0 for single page operations.
+     * @return Pair [cursor corresponding to results of UPDATE (contains number of items affected); keys whose values
+     *     had been modified concurrently (arguments for a re-run)].
+     */
+    @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
+    private UpdateResult doUpdate(UpdatePlan plan, QueryCursorImpl<List<?>> cursor, int pageSize)
+        throws IgniteCheckedException {
+        GridH2RowDescriptor desc = plan.tbl.rowDescriptor();
+
+        GridCacheContext cctx = desc.context();
+
+        boolean bin = cctx.binaryMarshaller();
+
+        String[] updatedColNames = plan.colNames;
+
+        int valColIdx = plan.valColIdx;
+
+        boolean hasNewVal = (valColIdx != -1);
+
+        // Statement updates distinct properties if it does not have _val in updated columns list
+        // or if its list of updated columns includes only _val, i.e. is single element.
+        boolean hasProps = !hasNewVal || updatedColNames.length > 1;
+
+        long res = 0;
+
+        CacheOperationContext opCtx = cctx.operationContextPerCall();
+
+        // Force keepBinary for operation context to avoid binary deserialization inside entry processor
+        if (cctx.binaryMarshaller()) {
+            CacheOperationContext newOpCtx = null;
+
+            if (opCtx == null)
+                // Mimics behavior of GridCacheAdapter#keepBinary and GridCacheProxyImpl#keepBinary
+                newOpCtx = new CacheOperationContext(false, null, true, null, false, null);
+            else if (!opCtx.isKeepBinary())
+                newOpCtx = opCtx.keepBinary();
+
+            if (newOpCtx != null)
+                cctx.operationContextPerCall(newOpCtx);
+        }
+
+        Map<Object, EntryProcessor<Object, Object, Boolean>> rows = new LinkedHashMap<>();
+
+        // Keys that failed to UPDATE due to concurrent updates.
+        List<Object> failedKeys = new ArrayList<>();
+
+        SQLException resEx = null;
+
+        try {
+            Iterator<List<?>> it = cursor.iterator();
+
+            while (it.hasNext()) {
+                List<?> e = it.next();
+                Object key = e.get(0);
+                Object val = (hasNewVal ? e.get(valColIdx) : e.get(1));
+
+                Object newVal;
+
+                Map<String, Object> newColVals = new HashMap<>();
+
+                for (int i = 0; i < plan.colNames.length; i++) {
+                    if (hasNewVal && i == valColIdx - 2)
+                        continue;
+
+                    newColVals.put(plan.colNames[i], e.get(i + 2));
+                }
+
+                newVal = plan.valSupplier.apply(e);
+
+                if (bin && !(val instanceof BinaryObject))
+                    val = cctx.grid().binary().toBinary(val);
+
+                // Skip key and value - that's why we start off with 2nd column
+                for (int i = 0; i < plan.tbl.getColumns().length - 2; i++) {
+                    Column c = plan.tbl.getColumn(i + 2);
+
+                    boolean hasNewColVal = newColVals.containsKey(c.getName());
+
+                    // Binary objects get old field values from the Builder, so we can skip what we're not updating
+                    if (bin && !hasNewColVal)
+                        continue;
+
+                    Object colVal = hasNewColVal ? newColVals.get(c.getName()) : desc.columnValue(key, val, i);
+
+                    desc.setColumnValue(key, newVal, colVal, i);
+                }
+
+                if (bin && hasProps) {
+                    assert newVal instanceof BinaryObjectBuilder;
+
+                    newVal = ((BinaryObjectBuilder) newVal).build();
+                }
+
+                Object srcVal = e.get(1);
+
+                if (bin && !(srcVal instanceof BinaryObject))
+                    srcVal = cctx.grid().binary().toBinary(srcVal);
+
+                rows.put(key, new ModifyingEntryProcessor(srcVal, new EntryValueUpdater(newVal)));
+
+                if ((pageSize > 0 && rows.size() == pageSize) || (!it.hasNext())) {
+                    PageProcessingResult pageRes = processPage(cctx, rows);
+
+                    res += pageRes.cnt;
+
+                    failedKeys.addAll(F.asList(pageRes.errKeys));
+
+                    if (pageRes.ex != null) {
+                        if (resEx == null)
+                            resEx = pageRes.ex;
+                        else
+                            resEx.setNextException(pageRes.ex);
+                    }
+
+                    if (it.hasNext())
+                        rows.clear(); // No need to clear after the last batch.
+                }
+            }
+
+            if (resEx != null) {
+                if (!F.isEmpty(failedKeys)) {
+                    // Don't go for a re-run if processing of some keys yielded exceptions and report keys that
+                    // had been modified concurrently right away.
+                    String msg = "Failed to UPDATE some keys because they had been modified concurrently " +
+                        "[keys=" + failedKeys + ']';
+
+                    SQLException dupEx = createJdbcSqlException(msg, IgniteQueryErrorCode.CONCURRENT_UPDATE);
+
+                    dupEx.setNextException(resEx);
+
+                    resEx = dupEx;
+                }
+
+                throw new IgniteSQLException(resEx);
+            }
+        }
+        finally {
+            cctx.operationContextPerCall(opCtx);
+        }
+
+        return new UpdateResult(res, failedKeys.toArray());
+    }
+
+    /**
+     * Process errors of entry processor - split the keys into duplicated/concurrently modified and those whose
+     * processing yielded an exception.
+     *
+     * @param res Result of {@link GridCacheAdapter#invokeAll)}
+     * @return pair [array of duplicated/concurrently modified keys, SQL exception for erroneous keys] (exception is
+     * null if all keys are duplicates/concurrently modified ones).
+     */
+    private static PageProcessingErrorResult splitErrors(Map<Object, EntryProcessorResult<Boolean>> res) {
+        Set<Object> errKeys = new LinkedHashSet<>(res.keySet());
+
+        SQLException currSqlEx = null;
+
+        SQLException firstSqlEx = null;
+
+        int errors = 0;
+
+        // Let's form a chain of SQL exceptions
+        for (Map.Entry<Object, EntryProcessorResult<Boolean>> e : res.entrySet()) {
+            try {
+                e.getValue().get();
+            }
+            catch (EntryProcessorException ex) {
+                SQLException next = createJdbcSqlException("Failed to process key '" + e.getKey() + '\'',
+                    IgniteQueryErrorCode.ENTRY_PROCESSING);
+
+                next.initCause(ex);
+
+                if (currSqlEx != null)
+                    currSqlEx.setNextException(next);
+                else
+                    firstSqlEx = next;
+
+                currSqlEx = next;
+
+                errKeys.remove(e.getKey());
+
+                errors++;
+            }
+        }
+
+        return new PageProcessingErrorResult(errKeys.toArray(), firstSqlEx, errors);
+    }
+
+    /**
+     * Execute MERGE statement plan.
+     * @param cursor Cursor to take inserted data from.
+     * @param pageSize Batch size to stream data from {@code cursor}, anything <= 0 for single page operations.
+     * @return Number of items affected.
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings("unchecked")
+    private long doMerge(UpdatePlan plan, QueryCursorImpl<List<?>> cursor, int pageSize) throws IgniteCheckedException {
+        GridH2RowDescriptor desc = plan.tbl.rowDescriptor();
+
+        GridCacheContext cctx = desc.context();
+
+        // If we have just one item to put, just do so
+        if (plan.rowsNum == 1) {
+            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.keySupplier,
+                plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc.type());
+
+            cctx.cache().put(t.getKey(), t.getValue());
+            return 1;
+        }
+        else {
+            int resCnt = 0;
+            Map<Object, Object> rows = new LinkedHashMap<>();
+
+            for (Iterator<List<?>> it = cursor.iterator(); it.hasNext();) {
+                List<?> row = it.next();
+
+                IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.keySupplier, plan.valSupplier,
+                    plan.keyColIdx, plan.valColIdx, desc.type());
+
+                rows.put(t.getKey(), t.getValue());
+
+                if ((pageSize > 0 && rows.size() == pageSize) || !it.hasNext()) {
+                    cctx.cache().putAll(rows);
+                    resCnt += rows.size();
+
+                    if (it.hasNext())
+                        rows.clear();
+                }
+            }
+
+            return resCnt;
+        }
+    }
+
+    /**
+     * Execute INSERT statement plan.
+     * @param cursor Cursor to take inserted data from.
+     * @param pageSize Batch size for streaming, anything <= 0 for single page operations.
+     * @return Number of items affected.
+     * @throws IgniteCheckedException if failed, particularly in case of duplicate keys.
+     */
+    @SuppressWarnings({"unchecked", "ConstantConditions"})
+    private long doInsert(UpdatePlan plan, QueryCursorImpl<List<?>> cursor, int pageSize) throws IgniteCheckedException {
+        GridH2RowDescriptor desc = plan.tbl.rowDescriptor();
+
+        GridCacheContext cctx = desc.context();
+
+        // If we have just one item to put, just do so
+        if (plan.rowsNum == 1) {
+            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.keySupplier,
+                plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc.type());
+
+            if (cctx.cache().putIfAbsent(t.getKey(), t.getValue()))
+                return 1;
+            else
+                throw new IgniteSQLException("Duplicate key during INSERT [key=" + t.getKey() + ']',
+                    IgniteQueryErrorCode.DUPLICATE_KEY);
+        }
+        else {
+            CacheOperationContext opCtx = cctx.operationContextPerCall();
+
+            // Force keepBinary for operation context to avoid binary deserialization inside entry processor
+            if (cctx.binaryMarshaller()) {
+                CacheOperationContext newOpCtx = null;
+
+                if (opCtx == null)
+                    // Mimics behavior of GridCacheAdapter#keepBinary and GridCacheProxyImpl#keepBinary
+                    newOpCtx = new CacheOperationContext(false, null, true, null, false, null);
+                else if (!opCtx.isKeepBinary())
+                    newOpCtx = opCtx.keepBinary();
+
+                if (newOpCtx != null)
+                    cctx.operationContextPerCall(newOpCtx);
+            }
+
+            Map<Object, EntryProcessor<Object, Object, Boolean>> rows = plan.isLocSubqry ?
+                new LinkedHashMap<Object, EntryProcessor<Object, Object, Boolean>>(plan.rowsNum) :
+                new LinkedHashMap<Object, EntryProcessor<Object, Object, Boolean>>();
+
+            // Keys that failed to INSERT due to duplication.
+            List<Object> duplicateKeys = new ArrayList<>();
+
+            int resCnt = 0;
+
+            SQLException resEx = null;
+
+            try {
+                Iterator<List<?>> it = cursor.iterator();
+
+                while (it.hasNext()) {
+                    List<?> row = it.next();
+
+                    final IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.keySupplier,
+                        plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc.type());
+
+                    rows.put(t.getKey(), new InsertEntryProcessor(t.getValue()));
+
+                    if (!it.hasNext() || (pageSize > 0 && rows.size() == pageSize)) {
+                        PageProcessingResult pageRes = processPage(cctx, rows);
+
+                        resCnt += pageRes.cnt;
+
+                        duplicateKeys.addAll(F.asList(pageRes.errKeys));
+
+                        if (pageRes.ex != null) {
+                            if (resEx == null)
+                                resEx = pageRes.ex;
+                            else
+                                resEx.setNextException(pageRes.ex);
+                        }
+
+                        rows.clear();
+                    }
+                }
+
+                if (!F.isEmpty(duplicateKeys)) {
+                    String msg = "Failed to INSERT some keys because they are already in cache " +
+                        "[keys=" + duplicateKeys + ']';
+
+                    SQLException dupEx = new SQLException(msg, null, IgniteQueryErrorCode.DUPLICATE_KEY);
+
+                    if (resEx == null)
+                        resEx = dupEx;
+                    else
+                        resEx.setNextException(dupEx);
+                }
+
+                if (resEx != null)
+                    throw new IgniteSQLException(resEx);
+
+                return resCnt;
+            }
+            finally {
+                cctx.operationContextPerCall(opCtx);
+            }
+        }
+    }
+
+    /**
+     * Execute given entry processors and collect errors, if any.
+     * @param cctx Cache context.
+     * @param rows Rows to process.
+     * @return Triple [number of rows actually changed; keys that failed to update (duplicates or concurrently
+     *     updated ones); chain of exceptions for all keys whose processing resulted in error, or null for no errors].
+     * @throws IgniteCheckedException
+     */
+    @SuppressWarnings({"unchecked", "ConstantConditions"})
+    private static PageProcessingResult processPage(GridCacheContext cctx,
+        Map<Object, EntryProcessor<Object, Object, Boolean>> rows) throws IgniteCheckedException {
+        Map<Object, EntryProcessorResult<Boolean>> res = cctx.cache().invokeAll(rows);
+
+        if (F.isEmpty(res))
+            return new PageProcessingResult(rows.size(), null, null);
+
+        PageProcessingErrorResult splitRes = splitErrors(res);
+
+        int keysCnt = splitRes.errKeys.length;
+
+        return new PageProcessingResult(rows.size() - keysCnt - splitRes.cnt, splitRes.errKeys, splitRes.ex);
+    }
+
+    /**
+     * Convert row presented as an array of Objects into key-value pair to be inserted to cache.
+     *
+     * @param cctx Cache context.
+     * @param row Row to process.
+     * @param cols Query cols.
+     * @param keySupplier Key instantiation method.
+     * @param valSupplier Key instantiation method.
+     * @param keyColIdx Key column index, or {@code -1} if no key column is mentioned in {@code cols}.
+     * @param valColIdx Value column index, or {@code -1} if no value column is mentioned in {@code cols}.
+     * @param desc Table descriptor.
+     * @return Key-value pair.
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings({"unchecked", "ConstantConditions", "ResultOfMethodCallIgnored"})
+    private IgniteBiTuple<?, ?> rowToKeyValue(GridCacheContext cctx, Object[] row, String[] cols,
+        KeyValueSupplier keySupplier, KeyValueSupplier valSupplier, int keyColIdx, int valColIdx,
+        GridQueryTypeDescriptor desc) throws IgniteCheckedException {
+        Object key = keySupplier.apply(F.asList(row));
+        Object val = valSupplier.apply(F.asList(row));
+
+        if (key == null)
+            throw new IgniteSQLException("Key for INSERT or MERGE must not be null",  IgniteQueryErrorCode.NULL_KEY);
+
+        if (val == null)
+            throw new IgniteSQLException("Value for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_VALUE);
+
+        for (int i = 0; i < cols.length; i++) {
+            if (i == keyColIdx || i == valColIdx)
+                continue;
+
+            desc.setValue(cols[i], key, val, row[i]);
+        }
+
+        if (cctx.binaryMarshaller()) {
+            if (key instanceof BinaryObjectBuilder)
+                key = ((BinaryObjectBuilder) key).build();
+
+            if (val instanceof BinaryObjectBuilder)
+                val = ((BinaryObjectBuilder) val).build();
+
+            if (key instanceof BinaryObject)
+                key = updateHashCodeIfNeeded(cctx, (BinaryObject) key);
+
+            if (val instanceof BinaryObject)
+                val = updateHashCodeIfNeeded(cctx, (BinaryObject) val);
+        }
+
+        return new IgniteBiTuple<>(key, val);
+    }
+
+    /**
+     * Set hash code to binary object if it does not have one.
+     *
+     * @param cctx Cache context.
+     * @param binObj Binary object.
+     * @return Binary object with hash code set.
+     */
+    private BinaryObject updateHashCodeIfNeeded(GridCacheContext cctx, BinaryObject binObj) {
+        if (U.isHashCodeEmpty(binObj)) {
+            if (WARNED_TYPES.add(binObj.type().typeId()))
+                U.warn(indexing.getLogger(), "Binary object's type does not have identity resolver explicitly set, therefore " +
+                    "BinaryArrayIdentityResolver is used to generate hash codes for its instances, and therefore " +
+                    "hash code of this binary object will most likely not match that of its non serialized form. " +
+                    "For finer control over identity of this type, please update your BinaryConfiguration accordingly." +
+                    " [typeId=" + binObj.type().typeId() + ", typeName=" + binObj.type().typeName() + ']');
+
+            int hash = BinaryArrayIdentityResolver.instance().hashCode(binObj);
+
+            // Empty hash code means no identity set for the type, therefore, we can safely set hash code
+            // via this Builder as it won't be overwritten.
+            return cctx.grid().binary().builder(binObj)
+                .hashCode(hash)
+                .build();
+        }
+        else
+            return binObj;
+    }
+
+    /** */
+    private final static class InsertEntryProcessor implements EntryProcessor<Object, Object, Boolean> {
+        /** Value to set. */
+        private final Object val;
+
+        /** */
+        private InsertEntryProcessor(Object val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Boolean process(MutableEntry<Object, Object> entry, Object... arguments) throws EntryProcessorException {
+            if (entry.getValue() != null)
+                return false;
+
+            entry.setValue(val);
+            return null; // To leave out only erroneous keys - nulls are skipped on results' processing.
+        }
+    }
+
+    /**
+     * Entry processor invoked by UPDATE and DELETE operations.
+     */
+    private final static class ModifyingEntryProcessor implements EntryProcessor<Object, Object, Boolean> {
+        /** Value to expect. */
+        private final Object val;
+
+        /** Action to perform on entry. */
+        private final IgniteInClosure<MutableEntry<Object, Object>> entryModifier;
+
+        /** */
+        private ModifyingEntryProcessor(Object val, IgniteInClosure<MutableEntry<Object, Object>> entryModifier) {
+            this.val = val;
+            this.entryModifier = entryModifier;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Boolean process(MutableEntry<Object, Object> entry, Object... arguments) throws EntryProcessorException {
+            // Something happened to the cache while we were performing map-reduce.
+            if (!F.eq(entry.getValue(), val))
+                return false;
+
+            entryModifier.apply(entry);
+            return null; // To leave out only erroneous keys - nulls are skipped on results' processing.
+        }
+    }
+
+    /** */
+    private static IgniteInClosure<MutableEntry<Object, Object>> RMV = new IgniteInClosure<MutableEntry<Object, Object>>() {
+        /** {@inheritDoc} */
+        @Override public void apply(MutableEntry<Object, Object> e) {
+            e.remove();
+        }
+    };
+
+    /**
+     *
+     */
+    private static final class EntryValueUpdater implements IgniteInClosure<MutableEntry<Object, Object>> {
+        /** Value to set. */
+        private final Object val;
+
+        /** */
+        private EntryValueUpdater(Object val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(MutableEntry<Object, Object> e) {
+            e.setValue(val);
+        }
+    }
+
+    /**
+     * Wrap result of DML operation (number of items affected) to Iterable suitable to be wrapped by cursor.
+     *
+     * @param itemsCnt Update result to wrap.
+     * @return Resulting Iterable.
+     */
+    @SuppressWarnings("unchecked")
+    private static QueryCursorImpl<List<?>> cursorForUpdateResult(long itemsCnt) {
+        QueryCursorImpl<List<?>> res =
+            new QueryCursorImpl(Collections.singletonList(Collections.singletonList(itemsCnt)), null, false);
+
+        res.fieldsMeta(UPDATE_RESULT_META);
+
+        return res;
+    }
+
+    /** Update result - modifications count and keys to re-run query with, if needed. */
+    private final static class UpdateResult {
+        /** Number of processed items. */
+        final long cnt;
+
+        /** Keys that failed to be UPDATEd or DELETEd due to concurrent modification of values. */
+        @NotNull
+        final Object[] errKeys;
+
+        /** */
+        @SuppressWarnings("ConstantConditions")
+        private UpdateResult(long cnt, Object[] errKeys) {
+            this.cnt = cnt;
+            this.errKeys = U.firstNotNull(errKeys, X.EMPTY_OBJECT_ARRAY);
+        }
+    }
+
+    /** Result of processing an individual page with {@link IgniteCache#invokeAll} including error details, if any. */
+    private final static class PageProcessingResult {
+        /** Number of successfully processed items. */
+        final long cnt;
+
+        /** Keys that failed to be UPDATEd or DELETEd due to concurrent modification of values. */
+        @NotNull
+        final Object[] errKeys;
+
+        /** Chain of exceptions corresponding to failed keys. Null if no keys yielded an exception. */
+        final SQLException ex;
+
+        /** */
+        @SuppressWarnings("ConstantConditions")
+        private PageProcessingResult(long cnt, Object[] errKeys, SQLException ex) {
+            this.cnt = cnt;
+            this.errKeys = U.firstNotNull(errKeys, X.EMPTY_OBJECT_ARRAY);
+            this.ex = ex;
+        }
+    }
+
+    /** Result of splitting keys whose processing resulted into an exception from those skipped by
+     * logic of {@link EntryProcessor}s (most likely INSERT duplicates, or UPDATE/DELETE keys whose values
+     * had been modified concurrently), counting and collecting entry processor exceptions.
+     */
+    private final static class PageProcessingErrorResult {
+        /** Keys that failed to be processed by {@link EntryProcessor} (not due to an exception). */
+        @NotNull
+        final Object[] errKeys;
+
+        /** Number of entries whose processing resulted into an exception. */
+        final int cnt;
+
+        /** Chain of exceptions corresponding to failed keys. Null if no keys yielded an exception. */
+        final SQLException ex;
+
+        /** */
+        @SuppressWarnings("ConstantConditions")
+        private PageProcessingErrorResult(@NotNull Object[] errKeys, SQLException ex, int exCnt) {
+            errKeys = U.firstNotNull(errKeys, X.EMPTY_OBJECT_ARRAY);
+            // When exceptions count must be zero, exceptions chain must be not null, and vice versa.
+            assert exCnt == 0 ^ ex != null;
+
+            this.errKeys = errKeys;
+            this.cnt = exCnt;
+            this.ex = ex;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
index e0680d3..c8c26c4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
@@ -24,6 +24,7 @@ import java.util.NoSuchElementException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
 import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -136,7 +137,7 @@ public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAd
             return true;
         }
         catch (SQLException e) {
-            throw new IgniteException(e);
+            throw new IgniteSQLException(e);
         }
     }
 


[31/52] ignite git commit: ignite-4305 marshalling fix in GridNearAtomicSingleUpdateInvokeRequest

Posted by vo...@apache.org.
ignite-4305 marshalling fix in GridNearAtomicSingleUpdateInvokeRequest


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/30b442e8
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/30b442e8
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/30b442e8

Branch: refs/heads/master
Commit: 30b442e871471ef08fbecf2dfb8423aaf8ff8f78
Parents: 2fccde7
Author: Konstantin Dudkov <kd...@ya.ru>
Authored: Fri Nov 25 16:30:16 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Nov 25 16:30:16 2016 +0300

----------------------------------------------------------------------
 .../dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java     | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/30b442e8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
index 42b51d6..f18a13c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
@@ -202,8 +202,11 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
         if (!addDepInfo && ctx.deploymentEnabled())
             addDepInfo = true;
 
-        if (entryProcessor != null && entryProcessorBytes == null)
+        if (entryProcessor != null && entryProcessorBytes == null) {
+            prepareObject(entryProcessor, cctx);
+
             entryProcessorBytes = CU.marshal(cctx, entryProcessor);
+        }
 
         if (invokeArgsBytes == null)
             invokeArgsBytes = marshalInvokeArguments(invokeArgs, cctx);


[08/52] ignite git commit: IGNITE-2294: Implemented DML.

Posted by vo...@apache.org.
IGNITE-2294: Implemented DML.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/86d143bb
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/86d143bb
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/86d143bb

Branch: refs/heads/master
Commit: 86d143bb8bdbdccba9209762b1fd802d5dc40268
Parents: 93c3ccd
Author: Alexander Paschenko <al...@gmail.com>
Authored: Wed Nov 23 12:58:26 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Nov 23 12:58:27 2016 +0300

----------------------------------------------------------------------
 .../clients/src/test/config/jdbc-bin-config.xml |   54 +
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |  263 +++++
 .../JdbcAbstractUpdateStatementSelfTest.java    |   37 +
 ...BinaryMarshallerInsertStatementSelfTest.java |   37 +
 ...cBinaryMarshallerMergeStatementSelfTest.java |   37 +
 .../jdbc2/JdbcDeleteStatementSelfTest.java      |   49 +
 .../jdbc2/JdbcInsertStatementSelfTest.java      |  122 +++
 .../jdbc2/JdbcMergeStatementSelfTest.java       |   91 ++
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    5 +
 .../java/org/apache/ignite/IgniteCache.java     |    2 +
 .../binary/BinaryAbstractIdentityResolver.java  |   53 +
 .../binary/BinaryArrayIdentityResolver.java     |  224 ++++
 .../binary/BinaryFieldIdentityResolver.java     |  307 ++++++
 .../ignite/binary/BinaryIdentityResolver.java   |   42 +
 .../ignite/binary/BinaryTypeConfiguration.java  |   27 +-
 .../org/apache/ignite/cache/QueryEntity.java    |   27 +
 .../ignite/cache/query/SqlFieldsQuery.java      |    2 +-
 .../configuration/CacheConfiguration.java       |   19 +-
 .../internal/binary/BinaryClassDescriptor.java  |   15 +
 .../ignite/internal/binary/BinaryContext.java   |   76 +-
 .../ignite/internal/binary/BinaryFieldImpl.java |   10 +-
 .../internal/binary/BinaryObjectExImpl.java     |   90 +-
 .../internal/binary/BinaryObjectImpl.java       |   48 +-
 .../binary/BinaryObjectOffheapImpl.java         |   44 +-
 .../internal/binary/BinaryPrimitives.java       |   24 +
 .../binary/BinarySerializedFieldComparator.java |  343 ++++++
 .../ignite/internal/binary/BinaryUtils.java     |    2 +-
 .../internal/binary/BinaryWriterExImpl.java     |   47 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |    6 +
 .../streams/BinaryAbstractInputStream.java      |    5 +
 .../streams/BinaryAbstractOutputStream.java     |    5 +
 .../binary/streams/BinaryHeapInputStream.java   |    5 +
 .../binary/streams/BinaryHeapOutputStream.java  |    5 +
 .../streams/BinaryOffheapInputStream.java       |   10 +
 .../streams/BinaryOffheapOutputStream.java      |    4 +-
 .../internal/binary/streams/BinaryStream.java   |   12 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |   17 +
 .../internal/jdbc2/JdbcPreparedStatement.java   |   71 +-
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |    5 +-
 .../ignite/internal/jdbc2/JdbcQueryTaskV2.java  |  406 +++++++
 .../ignite/internal/jdbc2/JdbcResultSet.java    |   27 +
 .../internal/jdbc2/JdbcSqlFieldsQuery.java      |   49 +
 .../ignite/internal/jdbc2/JdbcStatement.java    |  213 +++-
 .../apache/ignite/internal/jdbc2/JdbcUtils.java |   25 +-
 .../processors/cache/QueryCursorImpl.java       |   31 +-
 .../cache/query/GridCacheTwoStepQuery.java      |  253 -----
 .../cache/query/IgniteQueryErrorCode.java       |   91 ++
 .../memory/PlatformInputStreamImpl.java         |   10 +
 .../memory/PlatformOutputStreamImpl.java        |   12 +
 .../processors/query/GridQueryIndexing.java     |   23 +-
 .../processors/query/GridQueryProcessor.java    |  578 ++++++++--
 .../processors/query/GridQueryProperty.java     |   20 +
 .../query/GridQueryTypeDescriptor.java          |   25 +
 .../processors/query/IgniteSQLException.java    |   89 ++
 .../util/lang/IgniteSingletonIterator.java      |   56 +
 .../BinaryArrayIdentityResolverSelfTest.java    |  300 +++++
 .../BinaryFieldIdentityResolverSelfTest.java    |  333 ++++++
 ...ryIdentityResolverConfigurationSelfTest.java |  138 +++
 .../BinarySerialiedFieldComparatorSelfTest.java |  568 ++++++++++
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  260 ++++-
 .../IgniteBinaryObjectsTestSuite.java           |   10 +
 .../cache/query/GridCacheTwoStepQuery.java      |  253 +++++
 .../query/h2/DmlStatementsProcessor.java        | 1027 ++++++++++++++++++
 .../query/h2/GridH2ResultSetIterator.java       |    3 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  124 ++-
 .../query/h2/dml/FastUpdateArgument.java        |   27 +
 .../query/h2/dml/FastUpdateArguments.java       |   53 +
 .../query/h2/dml/KeyValueSupplier.java          |   30 +
 .../processors/query/h2/dml/UpdateMode.java     |   36 +
 .../processors/query/h2/dml/UpdatePlan.java     |  121 +++
 .../query/h2/dml/UpdatePlanBuilder.java         |  502 +++++++++
 .../processors/query/h2/dml/package-info.java   |   22 +
 .../query/h2/opt/GridH2RowDescriptor.java       |   23 +
 .../query/h2/opt/GridH2TreeIndex.java           |    2 +-
 .../processors/query/h2/sql/DmlAstUtils.java    |  599 ++++++++++
 .../processors/query/h2/sql/GridSqlArray.java   |    8 +
 .../processors/query/h2/sql/GridSqlConst.java   |    6 +
 .../processors/query/h2/sql/GridSqlDelete.java  |   68 ++
 .../query/h2/sql/GridSqlFunction.java           |    5 +-
 .../processors/query/h2/sql/GridSqlInsert.java  |  149 +++
 .../processors/query/h2/sql/GridSqlKeyword.java |   46 +
 .../processors/query/h2/sql/GridSqlMerge.java   |  143 +++
 .../processors/query/h2/sql/GridSqlQuery.java   |   44 +-
 .../query/h2/sql/GridSqlQueryParser.java        |  323 +++++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |    6 +-
 .../processors/query/h2/sql/GridSqlSelect.java  |    3 +-
 .../query/h2/sql/GridSqlStatement.java          |   64 ++
 .../processors/query/h2/sql/GridSqlUpdate.java  |  105 ++
 .../h2/twostep/GridReduceQueryExecutor.java     |   28 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |  567 ++++++++++
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |  219 ++++
 .../IgniteCacheDeleteSqlQuerySelfTest.java      |   81 ++
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  203 ++++
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  153 +++
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |  150 +++
 .../IgniteCacheAtomicFieldsQuerySelfTest.java   |   21 -
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   40 +-
 .../query/h2/sql/GridQueryParsingTest.java      |  109 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    9 +
 .../config/benchmark-bin-identity.properties    |   94 ++
 .../config/benchmark-multicast.properties       |   15 +
 .../config/benchmark-sql-dml.properties         |   72 ++
 modules/yardstick/config/ignite-base-config.xml |   73 +-
 .../config/ignite-bin-multicast-config.xml      |   86 ++
 .../cache/IgniteBinaryIdentityBenchmark.java    |  108 ++
 .../cache/IgniteBinaryIdentityGetBenchmark.java |   34 +
 .../cache/IgniteBinaryIdentityPutBenchmark.java |   35 +
 .../IgniteFieldsBinaryIdentityGetBenchmark.java |   30 +
 .../IgniteFieldsBinaryIdentityPutBenchmark.java |   30 +
 .../IgniteLegacyBinaryIdentityGetBenchmark.java |   30 +
 .../IgniteLegacyBinaryIdentityPutBenchmark.java |   30 +
 ...IgnitePutIfAbsentIndexedValue1Benchmark.java |   45 +
 .../IgniteReplaceIndexedValue1Benchmark.java    |   79 ++
 .../cache/dml/IgniteSqlDeleteBenchmark.java     |   83 ++
 .../dml/IgniteSqlDeleteFilteredBenchmark.java   |   88 ++
 .../IgniteSqlInsertIndexedValue1Benchmark.java  |   48 +
 .../IgniteSqlInsertIndexedValue2Benchmark.java  |   48 +
 .../IgniteSqlInsertIndexedValue8Benchmark.java  |   48 +
 .../cache/dml/IgniteSqlMergeAllBenchmark.java   |   82 ++
 .../cache/dml/IgniteSqlMergeBenchmark.java      |   42 +
 .../IgniteSqlMergeIndexedValue1Benchmark.java   |   43 +
 .../IgniteSqlMergeIndexedValue2Benchmark.java   |   43 +
 .../IgniteSqlMergeIndexedValue8Benchmark.java   |   43 +
 .../cache/dml/IgniteSqlMergeQueryBenchmark.java |  116 ++
 .../cache/dml/IgniteSqlUpdateBenchmark.java     |   82 ++
 .../dml/IgniteSqlUpdateFilteredBenchmark.java   |   88 ++
 .../yardstick/cache/model/SampleValue.java      |    2 +
 127 files changed, 12020 insertions(+), 728 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/clients/src/test/config/jdbc-bin-config.xml
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/config/jdbc-bin-config.xml b/modules/clients/src/test/config/jdbc-bin-config.xml
new file mode 100644
index 0000000..69f85a1
--- /dev/null
+++ b/modules/clients/src/test/config/jdbc-bin-config.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    Ignite Spring configuration file.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <!-- JDBC driver should force true value -->
+        <property name="clientMode" value="false"/>
+
+        <property name="localHost" value="127.0.0.1"/>
+
+        <property name="marshaller">
+            <bean class="org.apache.ignite.internal.binary.BinaryMarshaller" />
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500..47549</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="peerClassLoadingEnabled" value="true"/>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
new file mode 100644
index 0000000..4a97aef
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
@@ -0,0 +1,263 @@
+/*
+ * 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.jdbc2;
+
+import java.io.Serializable;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Collections;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Statement test.
+ */
+public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** JDBC URL. */
+    private static final String BASE_URL = CFG_URL_PREFIX + "modules/clients/src/test/config/jdbc-config.xml";
+
+    /** JDBC URL for tests involving binary objects manipulation. */
+    static final String BASE_URL_BIN = CFG_URL_PREFIX + "modules/clients/src/test/config/jdbc-bin-config.xml";
+
+    /** SQL SELECT query for verification. */
+    private static final String SQL_SELECT = "select _key, id, firstName, lastName, age from Person";
+
+    /** Connection. */
+    protected Connection conn;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return getConfiguration0(gridName);
+    }
+
+    /**
+     * @param gridName Grid name.
+     * @return Grid configuration used for starting the grid.
+     * @throws Exception If failed.
+     */
+    private IgniteConfiguration getConfiguration0(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<?,?> cache = defaultCacheConfiguration();
+
+        cache.setCacheMode(PARTITIONED);
+        cache.setBackups(1);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setIndexedTypes(
+            String.class, Person.class
+        );
+
+        cfg.setCacheConfiguration(cache);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setConnectorConfiguration(new ConnectorConfiguration());
+
+        return cfg;
+    }
+
+    /**
+     * @param gridName Grid name.
+     * @return Grid configuration used for starting the grid ready for manipulating binary objects.
+     * @throws Exception If failed.
+     */
+    IgniteConfiguration getBinaryConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = getConfiguration0(gridName);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        CacheConfiguration ccfg = cfg.getCacheConfiguration()[0];
+
+        ccfg.getQueryEntities().clear();
+
+        QueryEntity e = new QueryEntity();
+
+        e.setKeyType(String.class.getName());
+        e.setValueType("Person");
+
+        e.addQueryField("id", Integer.class.getName(), null);
+        e.addQueryField("age", Integer.class.getName(), null);
+        e.addQueryField("firstName", String.class.getName(), null);
+        e.addQueryField("lastName", String.class.getName(), null);
+
+        ccfg.setQueryEntities(Collections.singletonList(e));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3);
+
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        conn = DriverManager.getConnection(getCfgUrl());
+    }
+
+    /**
+     * @return URL of XML configuration file.
+     */
+    protected String getCfgUrl() {
+        return BASE_URL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        try (Statement selStmt = conn.createStatement()) {
+            assert selStmt.execute(SQL_SELECT);
+
+            ResultSet rs = selStmt.getResultSet();
+
+            assert rs != null;
+
+            while (rs.next()) {
+                int id = rs.getInt("id");
+
+                switch (id) {
+                    case 1:
+                        assertEquals("p1", rs.getString("_key"));
+                        assertEquals("John", rs.getString("firstName"));
+                        assertEquals("White", rs.getString("lastName"));
+                        assertEquals(25, rs.getInt("age"));
+                        break;
+
+                    case 2:
+                        assertEquals("p2", rs.getString("_key"));
+                        assertEquals("Joe", rs.getString("firstName"));
+                        assertEquals("Black", rs.getString("lastName"));
+                        assertEquals(35, rs.getInt("age"));
+                        break;
+
+                    case 3:
+                        assertEquals("p3", rs.getString("_key"));
+                        assertEquals("Mike", rs.getString("firstName"));
+                        assertEquals("Green", rs.getString("lastName"));
+                        assertEquals(40, rs.getInt("age"));
+                        break;
+
+                    case 4:
+                        assertEquals("p4", rs.getString("_key"));
+                        assertEquals("Leah", rs.getString("firstName"));
+                        assertEquals("Grey", rs.getString("lastName"));
+                        assertEquals(22, rs.getInt("age"));
+                        break;
+
+                    default:
+                        assert false : "Invalid ID: " + id;
+                }
+            }
+        }
+
+        grid(0).cache(null).clear();
+
+        assertEquals(0, grid(0).cache(null).size(CachePeekMode.ALL));
+    }
+
+    /**
+     * Person.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    static class Person implements Serializable {
+        /** ID. */
+        @QuerySqlField
+        private final int id;
+
+        /** First name. */
+        @QuerySqlField
+        private final String firstName;
+
+        /** Last name. */
+        @QuerySqlField
+        private final String lastName;
+
+        /** Age. */
+        @QuerySqlField
+        private final int age;
+
+        /**
+         * @param id ID.
+         * @param firstName First name.
+         * @param lastName Last name.
+         * @param age Age.
+         */
+        Person(int id, String firstName, String lastName, int age) {
+            assert !F.isEmpty(firstName);
+            assert !F.isEmpty(lastName);
+            assert age > 0;
+
+            this.id = id;
+            this.firstName = firstName;
+            this.lastName = lastName;
+            this.age = age;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Person person = (Person) o;
+
+            if (id != person.id) return false;
+            if (age != person.age) return false;
+            if (firstName != null ? !firstName.equals(person.firstName) : person.firstName != null) return false;
+            return lastName != null ? lastName.equals(person.lastName) : person.lastName == null;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int result = id;
+            result = 31 * result + (firstName != null ? firstName.hashCode() : 0);
+            result = 31 * result + (lastName != null ? lastName.hashCode() : 0);
+            result = 31 * result + age;
+            return result;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java
new file mode 100644
index 0000000..a20b815
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.jdbc2;
+
+import java.sql.Statement;
+
+public abstract class JdbcAbstractUpdateStatementSelfTest extends JdbcAbstractDmlStatementSelfTest {
+    /** SQL query to populate cache. */
+    private static final String ITEMS_SQL = "insert into Person(_key, id, firstName, lastName, age) values " +
+        "('p1', 1, 'John', 'White', 25), " +
+        "('p2', 2, 'Joe', 'Black', 35), " +
+        "('p3', 3, 'Mike', 'Green', 40)";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+        jcache(0).clear();
+        try (Statement s = conn.createStatement()) {
+            s.executeUpdate(ITEMS_SQL);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBinaryMarshallerInsertStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBinaryMarshallerInsertStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBinaryMarshallerInsertStatementSelfTest.java
new file mode 100644
index 0000000..667e9f0
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBinaryMarshallerInsertStatementSelfTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.jdbc2;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ * JDBC test of INSERT statement w/binary marshaller - no nodes know about classes.
+ */
+public class JdbcBinaryMarshallerInsertStatementSelfTest extends JdbcInsertStatementSelfTest {
+    /** {@inheritDoc} */
+    @Override protected String getCfgUrl() {
+        return BASE_URL_BIN;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return getBinaryConfiguration(gridName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBinaryMarshallerMergeStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBinaryMarshallerMergeStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBinaryMarshallerMergeStatementSelfTest.java
new file mode 100644
index 0000000..93451e7
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBinaryMarshallerMergeStatementSelfTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.jdbc2;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ * JDBC test of MERGE statement w/binary marshaller - no nodes know about classes.
+ */
+public class JdbcBinaryMarshallerMergeStatementSelfTest extends JdbcMergeStatementSelfTest {
+    /** {@inheritDoc} */
+    @Override protected String getCfgUrl() {
+        return BASE_URL_BIN;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return getBinaryConfiguration(gridName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java
new file mode 100644
index 0000000..d55c979
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.jdbc2;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashSet;
+
+/**
+ *
+ */
+public class JdbcDeleteStatementSelfTest extends JdbcAbstractUpdateStatementSelfTest {
+    /**
+     *
+     */
+    public void testExecute() throws SQLException {
+        conn.createStatement().execute("delete from Person where cast(substring(_key, 2, 1) as int) % 2 = 0");
+
+        assertFalse(jcache(0).containsKey("p2"));
+        assertTrue(jcache(0).containsKeys(new HashSet<Object>(Arrays.asList("p1", "p3"))));
+    }
+
+    /**
+     *
+     */
+    public void testExecuteUpdate() throws SQLException {
+        int res =
+            conn.createStatement().executeUpdate("delete from Person where cast(substring(_key, 2, 1) as int) % 2 = 0");
+
+        assertEquals(1, res);
+        assertFalse(jcache(0).containsKey("p2"));
+        assertTrue(jcache(0).containsKeys(new HashSet<Object>(Arrays.asList("p1", "p3"))));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
new file mode 100644
index 0000000..7fc92de
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.jdbc2;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ * Statement test.
+ */
+public class JdbcInsertStatementSelfTest extends JdbcAbstractDmlStatementSelfTest {
+    /** SQL query. */
+    private static final String SQL = "insert into Person(_key, id, firstName, lastName, age) values " +
+        "('p1', 1, 'John', 'White', 25), " +
+        "('p2', 2, 'Joe', 'Black', 35), " +
+        "('p3', 3, 'Mike', 'Green', 40)";
+
+    /** SQL query. */
+    private static final String SQL_PREPARED = "insert into Person(_key, id, firstName, lastName, age) values " +
+        "(?, ?, ?, ?, ?), (?, ?, ?, ?, ?)";
+
+    /** Statement. */
+    private Statement stmt;
+
+    /** Prepared statement. */
+    private PreparedStatement prepStmt;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+        stmt = conn.createStatement();
+        prepStmt = conn.prepareStatement(SQL_PREPARED);
+
+        assertNotNull(stmt);
+        assertFalse(stmt.isClosed());
+
+        assertNotNull(prepStmt);
+        assertFalse(prepStmt.isClosed());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        if (stmt != null && !stmt.isClosed())
+            stmt.close();
+
+        if (prepStmt != null && !prepStmt.isClosed())
+            prepStmt.close();
+
+        conn.close();
+
+        assertTrue(prepStmt.isClosed());
+        assertTrue(stmt.isClosed());
+        assertTrue(conn.isClosed());
+    }
+
+    /**
+     * @throws SQLException If failed.
+     */
+    public void testExecuteUpdate() throws SQLException {
+        int res = stmt.executeUpdate(SQL);
+
+        assertEquals(3, res);
+    }
+
+    /**
+     * @throws SQLException If failed.
+     */
+    public void testExecute() throws SQLException {
+        boolean res = stmt.execute(SQL);
+
+        assertEquals(false, res);
+    }
+
+    /**
+     *
+     */
+    public void testDuplicateKeys() {
+        jcache(0).put("p2", new Person(2, "Joe", "Black", 35));
+
+        Throwable reason = GridTestUtils.assertThrows(log, new Callable<Object>() {
+            /** {@inheritDoc} */
+            @Override public Object call() throws Exception {
+                return stmt.execute(SQL);
+            }
+        }, SQLException.class, null);
+
+        assertNotNull(reason.getCause());
+
+        reason = reason.getCause().getCause();
+
+        assertNotNull(reason);
+
+        assertEquals(IgniteException.class, reason.getClass());
+
+        assertEquals("Failed to INSERT some keys because they are already in cache [keys=[p2]]", reason.getMessage());
+
+        assertEquals(3, jcache(0).withKeepBinary().getAll(new HashSet<>(Arrays.asList("p1", "p2", "p3"))).size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
new file mode 100644
index 0000000..ecf6032
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.jdbc2;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+/**
+ * MERGE statement test.
+ */
+public class JdbcMergeStatementSelfTest extends JdbcAbstractDmlStatementSelfTest {
+    /** SQL query. */
+    private static final String SQL = "merge into Person(_key, id, firstName, lastName, age) values " +
+        "('p1', 1, 'John', 'White', 25), " +
+        "('p2', 2, 'Joe', 'Black', 35), " +
+        "('p3', 3, 'Mike', 'Green', 40)";
+
+    /** SQL query. */
+    protected static final String SQL_PREPARED = "merge into Person(_key, id, firstName, lastName, age) values " +
+        "(?, ?, ?, ?, ?), (?, ?, ?, ?, ?)";
+
+    /** Statement. */
+    protected Statement stmt;
+
+    /** Prepared statement. */
+    protected PreparedStatement prepStmt;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+        stmt = conn.createStatement();
+        prepStmt = conn.prepareStatement(SQL_PREPARED);
+
+        assertNotNull(stmt);
+        assertFalse(stmt.isClosed());
+
+        assertNotNull(prepStmt);
+        assertFalse(prepStmt.isClosed());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        if (stmt != null && !stmt.isClosed())
+            stmt.close();
+
+        if (prepStmt != null && !prepStmt.isClosed())
+            prepStmt.close();
+
+        conn.close();
+
+        assertTrue(prepStmt.isClosed());
+        assertTrue(stmt.isClosed());
+        assertTrue(conn.isClosed());
+    }
+
+    /**
+     * @throws SQLException If failed.
+     */
+    public void testExecuteUpdate() throws SQLException {
+        int res = stmt.executeUpdate(SQL);
+
+        assertEquals(3, res);
+    }
+
+    /**
+     * @throws SQLException If failed.
+     */
+    public void testExecute() throws SQLException {
+        boolean res = stmt.execute(SQL);
+
+        assertEquals(false, res);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
----------------------------------------------------------------------
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 b1053b0..048643b 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
@@ -62,6 +62,11 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcEmptyCacheSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcLocalCachesSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcNoDefaultCacheTest.class));
+        suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcMergeStatementSelfTest.class));
+        suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcBinaryMarshallerMergeStatementSelfTest.class));
+        suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcInsertStatementSelfTest.class));
+        suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcBinaryMarshallerInsertStatementSelfTest.class));
+        suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDeleteStatementSelfTest.class));
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index 7eb6e91..d7bccf5 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -47,6 +47,7 @@ import org.apache.ignite.cache.query.QueryDetailMetrics;
 import org.apache.ignite.cache.query.QueryMetrics;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SpiQuery;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.cache.query.TextQuery;
 import org.apache.ignite.cache.store.CacheStore;
@@ -291,6 +292,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * @return Cursor.
      * @see ScanQuery
      * @see SqlQuery
+     * @see SqlFieldsQuery
      * @see TextQuery
      * @see SpiQuery
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/binary/BinaryAbstractIdentityResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryAbstractIdentityResolver.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryAbstractIdentityResolver.java
new file mode 100644
index 0000000..b3036e2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryAbstractIdentityResolver.java
@@ -0,0 +1,53 @@
+/*
+ * 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.binary;
+
+/**
+ * Abstract identity resolver with common routines.
+ */
+public abstract class BinaryAbstractIdentityResolver implements BinaryIdentityResolver {
+    /** {@inheritDoc} */
+    @Override public int hashCode(BinaryObject obj) {
+        if (obj == null)
+            throw new BinaryObjectException("Cannot calculate hash code because binary object is null.");
+
+        return hashCode0(obj);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(BinaryObject o1, BinaryObject o2) {
+        return o1 == o2 || (o1 != null && o2 != null && equals0(o1, o2));
+    }
+
+    /**
+     * Internal hash code routine.
+     *
+     * @param obj Object.
+     * @return Result.
+     */
+    protected abstract int hashCode0(BinaryObject obj);
+
+    /**
+     * Internal equals routine.
+     *
+     * @param o1 First object.
+     * @param o2 Second object.
+     * @return Result.
+     */
+    protected abstract boolean equals0(BinaryObject o1, BinaryObject o2);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/binary/BinaryArrayIdentityResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryArrayIdentityResolver.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryArrayIdentityResolver.java
new file mode 100644
index 0000000..2f04c02
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryArrayIdentityResolver.java
@@ -0,0 +1,224 @@
+/*
+ * 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.binary;
+
+import java.util.Arrays;
+
+import org.apache.ignite.internal.binary.BinaryEnumObjectImpl;
+import org.apache.ignite.internal.binary.BinaryObjectEx;
+import org.apache.ignite.internal.binary.BinaryObjectExImpl;
+import org.apache.ignite.internal.binary.BinaryPrimitives;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Identity resolver implementation which compares raw array content of the binary object.
+ * <p>
+ * Hash code is calculated in the same way as {@link Arrays#hashCode(byte[])} does.
+ */
+public class BinaryArrayIdentityResolver extends BinaryAbstractIdentityResolver {
+    /** Singleton instance */
+    private static final BinaryArrayIdentityResolver INSTANCE = new BinaryArrayIdentityResolver();
+
+    /**
+     * Get singleton instance.
+     *
+     * @return Singleton instance.
+     */
+    public static BinaryArrayIdentityResolver instance() {
+        return INSTANCE;
+    }
+
+    /**
+     * Default constructor.
+     */
+    public BinaryArrayIdentityResolver() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int hashCode0(BinaryObject obj) {
+        int hash = 1;
+
+        if (obj instanceof BinaryObjectExImpl) {
+            BinaryObjectExImpl ex = (BinaryObjectExImpl)obj;
+
+            int start = ex.dataStartOffset();
+            int end = ex.footerStartOffset();
+
+            if (ex.hasArray()) {
+                // Handle heap object.
+                byte[] data = ex.array();
+
+                for (int i = start; i < end; i++)
+                    hash = 31 * hash + data[i];
+            }
+            else {
+                // Handle offheap object.
+                long ptr = ex.offheapAddress();
+
+                for (int i = start; i < end; i++)
+                    hash = 31 * hash + BinaryPrimitives.readByte(ptr, i);
+            }
+        }
+        else if (obj instanceof BinaryEnumObjectImpl) {
+            int ord = obj.enumOrdinal();
+
+            // Construct hash as if it was an int serialized in little-endian form.
+            hash = 31 * hash + (ord & 0x000000FF);
+            hash = 31 * hash + (ord & 0x0000FF00);
+            hash = 31 * hash + (ord & 0x00FF0000);
+            hash = 31 * hash + (ord & 0xFF000000);
+        }
+        else
+            throw new BinaryObjectException("Array identity resolver cannot be used with provided BinaryObject " +
+                "implementation: " + obj.getClass().getName());
+
+        return hash;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean equals0(BinaryObject o1, BinaryObject o2) {
+        if (o1 instanceof BinaryObjectEx && o2 instanceof BinaryObjectEx) {
+            BinaryObjectEx ex1 = (BinaryObjectEx)o1;
+            BinaryObjectEx ex2 = (BinaryObjectEx)o2;
+
+            if (ex1.typeId() != ex2.typeId())
+                return false;
+
+            if (ex1 instanceof BinaryObjectExImpl) {
+                // Handle regular object.
+                assert ex2 instanceof BinaryObjectExImpl;
+
+                BinaryObjectExImpl exx1 = (BinaryObjectExImpl)ex1;
+                BinaryObjectExImpl exx2 = (BinaryObjectExImpl)ex2;
+
+                if (exx1.hasArray())
+                    return exx2.hasArray() ? equalsHeap(exx1, exx2) : equalsHeapOffheap(exx1, exx2);
+                else
+                    return exx2.hasArray() ? equalsHeapOffheap(exx2, exx1) : equalsOffheap(exx1, exx2);
+            }
+            else {
+                // Handle enums.
+                assert ex1 instanceof BinaryEnumObjectImpl;
+                assert ex2 instanceof BinaryEnumObjectImpl;
+
+                return ex1.enumOrdinal() == ex2.enumOrdinal();
+            }
+        }
+
+        BinaryObject o = o1 instanceof BinaryObjectEx ? o2 : o1;
+
+        throw new BinaryObjectException("Array identity resolver cannot be used with provided BinaryObject " +
+            "implementation: " + o.getClass().getName());
+    }
+
+    /**
+     * Compare two heap objects.
+     *
+     * @param o1 Object 1.
+     * @param o2 Object 2.
+     * @return Result.
+     */
+    private static boolean equalsHeap(BinaryObjectExImpl o1, BinaryObjectExImpl o2) {
+        byte[] arr1 = o1.array();
+        byte[] arr2 = o2.array();
+
+        assert arr1 != null && arr2 != null;
+
+        int i = o1.dataStartOffset();
+        int j = o2.dataStartOffset();
+
+        int end = o1.footerStartOffset();
+
+        // Check length.
+        if (end - i != o2.footerStartOffset() - j)
+            return false;
+
+        for (; i < end; i++, j++) {
+            if (arr1[i] != arr2[j])
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Compare heap and offheap objects.
+     *
+     * @param o1 Object 1 (heap).
+     * @param o2 Object 2 (offheap).
+     * @return Result.
+     */
+    private static boolean equalsHeapOffheap(BinaryObjectExImpl o1, BinaryObjectExImpl o2) {
+        byte[] arr1 = o1.array();
+        long ptr2 = o2.offheapAddress();
+
+        assert arr1 != null && ptr2 != 0;
+
+        int i = o1.dataStartOffset();
+        int j = o2.dataStartOffset();
+
+        int end = o1.footerStartOffset();
+
+        // Check length.
+        if (end - i != o2.footerStartOffset() - j)
+            return false;
+
+        for (; i < end; i++, j++) {
+            if (arr1[i] != BinaryPrimitives.readByte(ptr2, j))
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Compare two offheap objects.
+     *
+     * @param o1 Object 1.
+     * @param o2 Object 2.
+     * @return Result.
+     */
+    private static boolean equalsOffheap(BinaryObjectExImpl o1, BinaryObjectExImpl o2) {
+        long ptr1 = o1.offheapAddress();
+        long ptr2 = o2.offheapAddress();
+
+        assert ptr1 != 0 && ptr2 != 0;
+
+        int i = o1.dataStartOffset();
+        int j = o2.dataStartOffset();
+
+        int end = o1.footerStartOffset();
+
+        // Check length.
+        if (end - i != o2.footerStartOffset() - j)
+            return false;
+
+        for (; i < end; i++, j++) {
+            if (BinaryPrimitives.readByte(ptr1, i) != BinaryPrimitives.readByte(ptr2, j))
+                return false;
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(BinaryArrayIdentityResolver.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/binary/BinaryFieldIdentityResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryFieldIdentityResolver.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryFieldIdentityResolver.java
new file mode 100644
index 0000000..c4fc869
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryFieldIdentityResolver.java
@@ -0,0 +1,307 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.internal.binary.BinaryEnumObjectImpl;
+import org.apache.ignite.internal.binary.BinaryFieldImpl;
+import org.apache.ignite.internal.binary.BinaryObjectExImpl;
+import org.apache.ignite.internal.binary.BinarySerializedFieldComparator;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.util.HashMap;
+
+/**
+ * Identity resolver implementation which use the list of provided fields to calculate the hash code and to perform
+ * equality checks.
+ * <p>
+ * Standard polynomial function with multiplier {@code 31} is used to calculate hash code. For example, for three
+ * fields {@code [a, b, c]}it would be {@code hash = 31 * (31 * a + b) + c}. Order of fields is important.
+ */
+public class BinaryFieldIdentityResolver extends BinaryAbstractIdentityResolver {
+    /** Mutex for synchronization. */
+    private final Object mux = new Object();
+
+    /** Cached single accessor. */
+    private volatile FieldAccessor accessor;
+
+    /** Cached accessors used when multiple (typeId, schemaId) pairs are met. */
+    private volatile HashMap<Long, FieldAccessor> accessors;
+
+    /** Field names. */
+    private String[] fieldNames;
+
+    /**
+     * Default constructor.
+     */
+    public BinaryFieldIdentityResolver() {
+        // No-op.
+    }
+
+    /**
+     * Copy constructor.
+     *
+     * @param other Other instance.
+     */
+    public BinaryFieldIdentityResolver(BinaryFieldIdentityResolver other) {
+        fieldNames = other.fieldNames;
+    }
+
+    /**
+     * @return Fields list to hash/compare objects based upon.
+     */
+    public String[] getFieldNames() {
+        return fieldNames;
+    }
+
+    /**
+     * Set field names.
+     *
+     * @param fieldNames Field names.
+     * @return {@code this} for chaining.
+     */
+    public BinaryFieldIdentityResolver setFieldNames(String... fieldNames) {
+        this.fieldNames = fieldNames;
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode0(BinaryObject obj) {
+        if (obj instanceof BinaryObjectExImpl) {
+            BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
+
+            if (obj0.hasSchema()) {
+                // Handle optimized case.
+                FieldAccessor accessor = accessor(obj0, obj0.typeId(), obj0.schemaId());
+
+                assert accessor != null;
+
+                return accessor.hashCode(obj0);
+            }
+        }
+        else if (obj instanceof BinaryEnumObjectImpl)
+            throw new BinaryObjectException("Field identity resolver cannot be used with enums: " + obj);
+
+        // Handle regular case.
+        int hash = 0;
+
+        for (String fieldName : fieldNames) {
+            Object val = obj.field(fieldName);
+
+            hash = 31 * hash + (val != null ? val.hashCode() : 0);
+        }
+
+        return hash;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals0(BinaryObject o1, BinaryObject o2) {
+        if (o1 instanceof BinaryObjectExImpl && o2 instanceof BinaryObjectExImpl) {
+            BinaryObjectExImpl ex1 = (BinaryObjectExImpl) o1;
+            BinaryObjectExImpl ex2 = (BinaryObjectExImpl) o2;
+
+            int typeId = ex1.typeId();
+
+            if (typeId != ex2.typeId())
+                return false;
+
+            if (ex1.hasSchema() && ex2.hasSchema()) {
+                // Optimistic case: both objects have schemas.
+                int schemaId1 = ex1.schemaId();
+                int schemaId2 = ex2.schemaId();
+
+                FieldAccessor accessor1 = accessor(ex1, typeId, schemaId1);
+
+                FieldAccessor accessor2;
+
+                if (schemaId1 == schemaId2)
+                    accessor2 = accessor1;
+                else
+                    accessor2 = accessor(ex2, typeId, schemaId2);
+
+                // Even better case: compare fields without deserialization.
+                BinarySerializedFieldComparator comp1 = ex1.createFieldComparator();
+                BinarySerializedFieldComparator comp2 = ex2.createFieldComparator();
+
+                for (int i = 0; i < fieldNames.length; i++) {
+                    comp1.findField(accessor1.orders[i]);
+                    comp2.findField(accessor2.orders[i]);
+
+                    if (!BinarySerializedFieldComparator.equals(comp1, comp2))
+                        return false;
+                }
+
+                return true;
+            }
+            else
+                // Pessimistic case: object of unknown types, or without schemas. Have to read fields in usual way.
+                return equalsSlow(ex1, ex2);
+        }
+
+        if (o1 instanceof BinaryEnumObjectImpl)
+            throw new BinaryObjectException("Field identity resolver cannot be used with enums: " + o1);
+
+        if (o2 instanceof BinaryEnumObjectImpl)
+            throw new BinaryObjectException("Field identity resolver cannot be used with enums: " + o2);
+
+        return o1.type().typeId() == o2.type().typeId() && equalsSlow(o1, o2);
+    }
+
+    /**
+     * Slow-path equals routine: regular fields comparison.
+     *
+     * @param o1 Object 1.
+     * @param o2 Object 2.
+     * @return Result.
+     */
+    private boolean equalsSlow(BinaryObject o1, BinaryObject o2) {
+        for (String fieldName : fieldNames) {
+            Object val1 = o1.field(fieldName);
+            Object val2 = o2.field(fieldName);
+
+            if (!F.eq(val1, val2))
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Get fields accessor for the given object.
+     *
+     * @param obj Object.
+     * @param typId Type ID.
+     * @param schemaId Schema ID.
+     * @return Accessor.
+     */
+    private FieldAccessor accessor(BinaryObjectExImpl obj, int typId, int schemaId) {
+        // Try getting single accessor.
+        FieldAccessor res = accessor;
+
+        if (res != null && res.applicableTo(typId, schemaId))
+            return res;
+
+        // Try reading from map.
+        long key = ((long)typId << 32) + schemaId;
+
+        HashMap<Long, FieldAccessor> accessors0 = accessors;
+
+        if (accessors0 != null) {
+            res = accessors0.get(key);
+
+            if (res != null)
+                return res;
+        }
+
+        // Failed to get from cache, go to locking.
+        synchronized (mux) {
+            // Create accessor.
+            int[] orders = new int[fieldNames.length];
+
+            BinaryType type = obj.type();
+
+            for (int i = 0; i < fieldNames.length; i++) {
+                BinaryFieldImpl field = (BinaryFieldImpl)type.field(fieldNames[i]);
+
+                orders[i] = field.fieldOrder(obj);
+            }
+
+            res = new FieldAccessor(typId, schemaId, orders);
+
+            // Set accessor.
+            if (accessor == null)
+                accessor = res;
+            else {
+                if (accessors == null) {
+                    accessor = null;
+
+                    accessors0 = new HashMap<>();
+                }
+                else
+                    accessors0 = new HashMap<>(accessors);
+
+                accessors0.put(key, res);
+
+                accessors = accessors0;
+            }
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(BinaryFieldIdentityResolver.class, this);
+    }
+
+    /**
+     * Optimized fields accessor.
+     */
+    private static class FieldAccessor {
+        /** Type ID. */
+        private final int typeId;
+
+        /** Schema ID. */
+        private final int schemaId;
+
+        /** Field orders. */
+        private final int[] orders;
+
+        /**
+         * Constructor.
+         *
+         * @param typeId Type ID.
+         * @param schemaId Schema ID.
+         * @param orders Field orders.
+         */
+        private FieldAccessor(int typeId, int schemaId, int[] orders) {
+            this.typeId = typeId;
+            this.schemaId = schemaId;
+            this.orders = orders;
+        }
+
+        /**
+         * Check whether object is applicable to that hash code accessor.
+         * @param expTypeId Expected schema ID.
+         * @param expSchemaId Expected schema ID.
+         * @return {@code True} if matches.
+         */
+        private boolean applicableTo(int expTypeId, int expSchemaId) {
+            return typeId == expTypeId && schemaId == expSchemaId;
+        }
+
+        /**
+         * Calculate object hash code.
+         *
+         * @param obj Object.
+         * @return Hash code.
+         */
+        private int hashCode(BinaryObjectExImpl obj) {
+            int hash = 0;
+
+            for (int order : orders) {
+                Object val = obj.fieldByOrder(order);
+
+                hash = 31 * hash + (val != null ? val.hashCode() : 0);
+            }
+
+            return hash;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdentityResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdentityResolver.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdentityResolver.java
new file mode 100644
index 0000000..9796eca
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdentityResolver.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.binary;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface to compute hash codes for new binary objects and compare them for equality.
+ */
+public interface BinaryIdentityResolver {
+    /**
+     * Compute hash code for binary object.
+     *
+     * @param obj Binary object.
+     * @return Hash code value.
+     */
+    public int hashCode(BinaryObject obj);
+
+    /**
+     * Compare two binary objects for equality.
+     *
+     * @param o1 First object.
+     * @param o2 Second object.
+     * @return {@code True} if both objects are equal.
+     */
+    public boolean equals(@Nullable BinaryObject o1, @Nullable BinaryObject o2);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
index fea0af7..d95e0ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
@@ -21,6 +21,7 @@ import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Defines configuration properties for a specific binary type. Providing per-type
@@ -42,6 +43,9 @@ public class BinaryTypeConfiguration {
     /** Serializer. */
     private BinarySerializer serializer;
 
+    /** Identity. */
+    private BinaryIdentityResolver identityRslvr;
+
     /** Enum flag. */
     private boolean isEnum;
 
@@ -60,10 +64,11 @@ public class BinaryTypeConfiguration {
     public BinaryTypeConfiguration(BinaryTypeConfiguration other) {
         A.notNull(other, "other");
 
-        typeName = other.typeName;
+        identityRslvr = other.identityRslvr;
         idMapper = other.idMapper;
-        serializer = other.serializer;
         isEnum = other.isEnum;
+        serializer = other.serializer;
+        typeName = other.typeName;
     }
 
     /**
@@ -146,6 +151,24 @@ public class BinaryTypeConfiguration {
     }
 
     /**
+     * Gets identity resolver.
+     *
+     * @return Identity resolver.
+     */
+    @Nullable public BinaryIdentityResolver getIdentityResolver() {
+        return identityRslvr;
+    }
+
+    /**
+     * Sets identity resolver.
+     *
+     * @param identityRslvr Identity resolver.
+     */
+    public void setIdentityResolver(@Nullable BinaryIdentityResolver identityRslvr) {
+        this.identityRslvr = identityRslvr;
+    }
+
+    /**
      * Gets whether this is enum type.
      *
      * @return {@code True} if enum.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
index 9758cfc..fc3b921 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
@@ -20,8 +20,10 @@ package org.apache.ignite.cache;
 import java.io.Serializable;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.Map;
+import java.util.Set;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
 
@@ -42,6 +44,9 @@ public class QueryEntity implements Serializable {
     /** Fields available for query. A map from field name to type name. */
     private LinkedHashMap<String, String> fields = new LinkedHashMap<>();
 
+    /** Set of field names that belong to the key. */
+    private Set<String> keyFields = new HashSet<>();
+
     /** Aliases. */
     private Map<String, String> aliases = new HashMap<>();
 
@@ -123,6 +128,28 @@ public class QueryEntity implements Serializable {
     }
 
     /**
+     * Gets query fields for this query pair that belongs to the key. We need this for the cases when no key-value classes
+     * are present on cluster nodes, and we need to build/modify keys and values during SQL DML operations.
+     * Thus, setting this parameter in XML is not mandatory and should be based on particular use case.
+     *
+     * @return Set of names of key fields.
+     */
+    public Set<String> getKeyFields() {
+        return keyFields;
+    }
+
+    /**
+     * Gets query fields for this query pair that belongs to the key. We need this for the cases when no key-value classes
+     * are present on cluster nodes, and we need to build/modify keys and values during SQL DML operations.
+     * Thus, setting this parameter in XML is not mandatory and should be based on particular use case.
+     *
+     * @param keyFields Set of names of key fields.
+     */
+    public void setKeyFields(Set<String> keyFields) {
+        this.keyFields = keyFields;
+    }
+
+    /**
      * Gets a collection of index entities.
      *
      * @return Collection of index entities.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
index d3f85af..9b17e78 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
@@ -44,7 +44,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
  *
  * @see IgniteCache#query(Query)
  */
-public final class SqlFieldsQuery extends Query<List<?>> {
+public class SqlFieldsQuery extends Query<List<?>> {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index f9c114b..56fc5b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -31,6 +31,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.TreeSet;
 import javax.cache.Cache;
 import javax.cache.CacheException;
@@ -2198,7 +2199,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param desc Type descriptor.
      * @return Type metadata.
      */
-    static QueryEntity convert(TypeDescriptor desc) {
+    private static QueryEntity convert(TypeDescriptor desc) {
         QueryEntity entity = new QueryEntity();
 
         // Key and val types.
@@ -2208,6 +2209,8 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         for (ClassProperty prop : desc.props.values())
             entity.addQueryField(prop.fullName(), U.box(prop.type()).getName(), prop.alias());
 
+        entity.setKeyFields(desc.keyProperties);
+
         QueryIndex txtIdx = null;
 
         Collection<QueryIndex> idxs = new ArrayList<>();
@@ -2355,7 +2358,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
                     processAnnotation(key, sqlAnn, txtAnn, field.getType(), prop, type);
 
-                    type.addProperty(prop, true);
+                    type.addProperty(prop, key, true);
                 }
             }
 
@@ -2377,7 +2380,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
                     processAnnotation(key, sqlAnn, txtAnn, mtd.getReturnType(), prop, type);
 
-                    type.addProperty(prop, true);
+                    type.addProperty(prop, key, true);
                 }
             }
         }
@@ -2461,6 +2464,10 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
         /** */
         @GridToStringInclude
+        private final Set<String> keyProperties = new HashSet<>();
+
+        /** */
+        @GridToStringInclude
         private final Map<String, IndexDescriptor> indexes = new HashMap<>();
 
         /** */
@@ -2567,15 +2574,19 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
          * Adds property to the type descriptor.
          *
          * @param prop Property.
+         * @param key Property ownership flag (key or not).
          * @param failOnDuplicate Fail on duplicate flag.
          */
-        public void addProperty(ClassProperty prop, boolean failOnDuplicate) {
+        void addProperty(ClassProperty prop, boolean key, boolean failOnDuplicate) {
             String name = prop.fullName();
 
             if (props.put(name, prop) != null && failOnDuplicate)
                 throw new CacheException("Property with name '" + name + "' already exists.");
 
             fields.put(name, prop.type());
+
+            if (key)
+                keyProperties.add(name);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index b121337..afe7b37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -751,6 +751,8 @@ public class BinaryClassDescriptor {
                                 schemaReg.addSchema(newSchema.schemaId(), newSchema);
                             }
                         }
+
+                        postWriteHashCode(writer, obj);
                     }
                     finally {
                         writer.popSchema();
@@ -780,6 +782,7 @@ public class BinaryClassDescriptor {
                         writer.schemaId(stableSchema.schemaId());
 
                         postWrite(writer, obj);
+                        postWriteHashCode(writer, obj);
                     }
                     finally {
                         writer.popSchema();
@@ -888,6 +891,18 @@ public class BinaryClassDescriptor {
     }
 
     /**
+     * Post-write routine for hash code.
+     *
+     * @param writer Writer.
+     * @param obj Object.
+     */
+    private void postWriteHashCode(BinaryWriterExImpl writer, Object obj) {
+        // No need to call "postWriteHashCode" here because we do not care about hash code.
+        if (!(obj instanceof CacheObjectImpl))
+            writer.postWriteHashCode(registered ? null : cls.getName());
+    }
+
+    /**
      * @return Instance.
      * @throws BinaryObjectException In case of error.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index cc18318..f1f205d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -26,6 +26,7 @@ import org.apache.ignite.binary.BinaryIdMapper;
 import org.apache.ignite.binary.BinaryInvalidTypeException;
 import org.apache.ignite.binary.BinaryNameMapper;
 import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryIdentityResolver;
 import org.apache.ignite.binary.BinaryReflectiveSerializer;
 import org.apache.ignite.binary.BinarySerializer;
 import org.apache.ignite.binary.BinaryType;
@@ -135,7 +136,7 @@ public class BinaryContext {
     /** Set of system classes that should be marshalled with BinaryMarshaller. */
     private static final Set<String> BINARYLIZABLE_SYS_CLSS;
 
-    /** Binarylizable system classes set initialization. */
+    /* Binarylizable system classes set initialization. */
     static {
         Set<String> sysClss = new HashSet<>();
 
@@ -222,6 +223,9 @@ public class BinaryContext {
     /** Maps className to mapper */
     private final ConcurrentMap<String, BinaryInternalMapper> cls2Mappers = new ConcurrentHashMap8<>(0);
 
+    /** Affinity key field names. */
+    private final ConcurrentMap<Integer, BinaryIdentityResolver> identities = new ConcurrentHashMap8<>(0);
+
     /** */
     private BinaryMetadataHandler metaHnd;
 
@@ -433,39 +437,29 @@ public class BinaryContext {
                     throw new BinaryObjectException("Class name is required for binary type configuration.");
 
                 // Resolve mapper.
-                BinaryIdMapper idMapper = globalIdMapper;
-
-                if (typeCfg.getIdMapper() != null)
-                    idMapper = typeCfg.getIdMapper();
-
-                BinaryNameMapper nameMapper = globalNameMapper;
-
-                if (typeCfg.getNameMapper() != null)
-                    nameMapper = typeCfg.getNameMapper();
+                BinaryIdMapper idMapper = U.firstNotNull(typeCfg.getIdMapper(), globalIdMapper);
+                BinaryNameMapper nameMapper = U.firstNotNull(typeCfg.getNameMapper(), globalNameMapper);
+                BinarySerializer serializer = U.firstNotNull(typeCfg.getSerializer(), globalSerializer);
+                BinaryIdentityResolver identity = typeCfg.getIdentityResolver();
 
                 BinaryInternalMapper mapper = resolveMapper(nameMapper, idMapper);
 
-                // Resolve serializer.
-                BinarySerializer serializer = globalSerializer;
-
-                if (typeCfg.getSerializer() != null)
-                    serializer = typeCfg.getSerializer();
-
                 if (clsName.endsWith(".*")) {
                     String pkgName = clsName.substring(0, clsName.length() - 2);
 
                     for (String clsName0 : classesInPackage(pkgName))
-                        descs.add(clsName0, mapper, serializer, affFields.get(clsName0),
+                        descs.add(clsName0, mapper, serializer, identity, affFields.get(clsName0),
                             typeCfg.isEnum(), true);
                 }
                 else
-                    descs.add(clsName, mapper, serializer, affFields.get(clsName),
+                    descs.add(clsName, mapper, serializer, identity, affFields.get(clsName),
                         typeCfg.isEnum(), false);
             }
         }
 
         for (TypeDescriptor desc : descs.descriptors())
-            registerUserType(desc.clsName, desc.mapper, desc.serializer, desc.affKeyFieldName, desc.isEnum);
+            registerUserType(desc.clsName, desc.mapper, desc.serializer, desc.identity, desc.affKeyFieldName,
+                desc.isEnum);
 
         BinaryInternalMapper globalMapper = resolveMapper(globalNameMapper, globalIdMapper);
 
@@ -1086,6 +1080,7 @@ public class BinaryContext {
      * @param clsName Class name.
      * @param mapper ID mapper.
      * @param serializer Serializer.
+     * @param identity Type identity.
      * @param affKeyFieldName Affinity key field name.
      * @param isEnum If enum.
      * @throws BinaryObjectException In case of error.
@@ -1094,6 +1089,7 @@ public class BinaryContext {
     public void registerUserType(String clsName,
         BinaryInternalMapper mapper,
         @Nullable BinarySerializer serializer,
+        @Nullable BinaryIdentityResolver identity,
         @Nullable String affKeyFieldName,
         boolean isEnum)
         throws BinaryObjectException {
@@ -1114,14 +1110,19 @@ public class BinaryContext {
 
         //Workaround for IGNITE-1358
         if (predefinedTypes.get(id) != null)
-            throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+            throw duplicateTypeIdException(clsName, id);
 
         if (typeId2Mapper.put(id, mapper) != null)
-            throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+            throw duplicateTypeIdException(clsName, id);
+
+        if (identity != null) {
+            if (identities.put(id, identity) != null)
+                throw duplicateTypeIdException(clsName, id);
+        }
 
         if (affKeyFieldName != null) {
             if (affKeyFieldNames.put(id, affKeyFieldName) != null)
-                throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+                throw duplicateTypeIdException(clsName, id);
         }
 
         cls2Mappers.put(clsName, mapper);
@@ -1163,6 +1164,16 @@ public class BinaryContext {
     }
 
     /**
+     * Throw exception on class duplication.
+     *
+     * @param clsName Class name.
+     * @param id Type id.
+     */
+    private static BinaryObjectException duplicateTypeIdException(String clsName, int id) {
+        return new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+    }
+
+    /**
      * Check whether reflective serializer can be used for class.
      *
      * @param cls Class.
@@ -1208,6 +1219,14 @@ public class BinaryContext {
 
     /**
      * @param typeId Type ID.
+     * @return Type identity.
+     */
+    public BinaryIdentityResolver identity(int typeId) {
+        return identities.get(typeId);
+    }
+
+    /**
+     * @param typeId Type ID.
      * @param meta Meta data.
      * @throws BinaryObjectException In case of error.
      */
@@ -1315,6 +1334,7 @@ public class BinaryContext {
          * @param clsName Class name.
          * @param mapper Mapper.
          * @param serializer Serializer.
+         * @param identity Key hashing mode.
          * @param affKeyFieldName Affinity key field name.
          * @param isEnum Enum flag.
          * @param canOverride Whether this descriptor can be override.
@@ -1323,6 +1343,7 @@ public class BinaryContext {
         private void add(String clsName,
             BinaryInternalMapper mapper,
             BinarySerializer serializer,
+            BinaryIdentityResolver identity,
             String affKeyFieldName,
             boolean isEnum,
             boolean canOverride)
@@ -1330,6 +1351,7 @@ public class BinaryContext {
             TypeDescriptor desc = new TypeDescriptor(clsName,
                 mapper,
                 serializer,
+                identity,
                 affKeyFieldName,
                 isEnum,
                 canOverride);
@@ -1365,6 +1387,9 @@ public class BinaryContext {
         /** Serializer. */
         private BinarySerializer serializer;
 
+        /** Type identity. */
+        private BinaryIdentityResolver identity;
+
         /** Affinity key field name. */
         private String affKeyFieldName;
 
@@ -1376,19 +1401,21 @@ public class BinaryContext {
 
         /**
          * Constructor.
-         *
          * @param clsName Class name.
          * @param mapper ID mapper.
          * @param serializer Serializer.
+         * @param identity Key hashing mode.
          * @param affKeyFieldName Affinity key field name.
          * @param isEnum Enum type.
          * @param canOverride Whether this descriptor can be override.
          */
         private TypeDescriptor(String clsName, BinaryInternalMapper mapper,
-            BinarySerializer serializer, String affKeyFieldName, boolean isEnum, boolean canOverride) {
+            BinarySerializer serializer, BinaryIdentityResolver identity, String affKeyFieldName, boolean isEnum,
+            boolean canOverride) {
             this.clsName = clsName;
             this.mapper = mapper;
             this.serializer = serializer;
+            this.identity = identity;
             this.affKeyFieldName = affKeyFieldName;
             this.isEnum = isEnum;
             this.canOverride = canOverride;
@@ -1406,6 +1433,7 @@ public class BinaryContext {
             if (canOverride) {
                 mapper = other.mapper;
                 serializer = other.serializer;
+                identity = other.identity;
                 affKeyFieldName = other.affKeyFieldName;
                 isEnum = other.isEnum;
                 canOverride = other.canOverride;

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
index 78ed17a..59e79fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
@@ -50,7 +50,6 @@ public class BinaryFieldImpl implements BinaryField {
     public BinaryFieldImpl(int typeId, BinarySchemaRegistry schemas, String fieldName, int fieldId) {
         assert typeId != 0;
         assert schemas != null;
-        assert fieldName != null;
         assert fieldId != 0;
 
         this.typeId = typeId;
@@ -64,6 +63,13 @@ public class BinaryFieldImpl implements BinaryField {
         return fieldName;
     }
 
+    /**
+     * @return Field ID.
+     */
+    public int fieldId() {
+        return fieldId;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean exists(BinaryObject obj) {
         BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
@@ -87,7 +93,7 @@ public class BinaryFieldImpl implements BinaryField {
      * @param obj Object.
      * @return Field offset.
      */
-    private int fieldOrder(BinaryObjectExImpl obj) {
+    public int fieldOrder(BinaryObjectExImpl obj) {
         if (typeId != obj.typeId()) {
             throw new BinaryObjectException("Failed to get field because type ID of passed object differs" +
                 " from type ID this " + BinaryField.class.getSimpleName() + " belongs to [expected=" + typeId +


[36/52] ignite git commit: IGNITE-4244: INCR/DECR to operate on AtomicLong.

Posted by vo...@apache.org.
IGNITE-4244: INCR/DECR to operate on AtomicLong.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d1cc2925
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d1cc2925
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d1cc2925

Branch: refs/heads/master
Commit: d1cc292583b9358ffb7efab5f15ab84bcfb67e9e
Parents: 2ca709f
Author: shtykh_roman <rs...@yahoo.com>
Authored: Mon Nov 28 16:55:47 2016 +0900
Committer: shtykh_roman <rs...@yahoo.com>
Committed: Mon Nov 28 16:55:47 2016 +0900

----------------------------------------------------------------------
 .../tcp/redis/RedisProtocolSelfTest.java        | 101 ++++++++++++++++---
 .../string/GridRedisGetCommandHandler.java      |  27 ++++-
 .../string/GridRedisIncrDecrCommandHandler.java |  63 ++++++++++--
 .../string/GridRedisSetCommandHandler.java      |  24 ++++-
 .../tcp/redis/GridRedisNioListener.java         |   4 +-
 5 files changed, 193 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d1cc2925/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/RedisProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/RedisProtocolSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/RedisProtocolSelfTest.java
index c059205..a0066aa 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/RedisProtocolSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/RedisProtocolSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.rest.protocols.tcp.redis;
 
-import java.math.BigInteger;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -264,15 +263,20 @@ public class RedisProtocolSelfTest extends GridCommonAbstractTest {
             Assert.assertEquals(1, (long)jedis.incr("newKeyIncr"));
             Assert.assertEquals(-1, (long)jedis.decr("newKeyDecr"));
 
-            jcache().put("incrKey1", 1L);
+            Assert.assertEquals("1", jedis.get("newKeyIncr"));
+            Assert.assertEquals("-1", jedis.get("newKeyDecr"));
 
-            Assert.assertEquals(2L, (long)jedis.incr("incrKey1"));
+            Assert.assertEquals(1, (long)jedis.incr("incrKey1"));
 
-            jcache().put("decrKey1", 1L);
+            jedis.set("incrKey1", "10");
 
-            Assert.assertEquals(0L, (long)jedis.decr("decrKey1"));
+            Assert.assertEquals(11L, (long)jedis.incr("incrKey1"));
 
-            jcache().put("nonInt", "abc");
+            jedis.set("decrKey1", "10");
+
+            Assert.assertEquals(9L, (long)jedis.decr("decrKey1"));
+
+            jedis.set("nonInt", "abc");
 
             try {
                 jedis.incr("nonInt");
@@ -282,6 +286,7 @@ public class RedisProtocolSelfTest extends GridCommonAbstractTest {
             catch (JedisDataException e) {
                 assertTrue(e.getMessage().startsWith("ERR"));
             }
+
             try {
                 jedis.decr("nonInt");
 
@@ -291,9 +296,39 @@ public class RedisProtocolSelfTest extends GridCommonAbstractTest {
                 assertTrue(e.getMessage().startsWith("ERR"));
             }
 
-            jcache().put("outOfRange", new BigInteger("234293482390480948029348230948"));
+            jedis.set("outOfRangeIncr1", "9223372036854775808");
             try {
-                jedis.incr("outOfRange");
+                jedis.incr("outOfRangeIncr1");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
+
+            jedis.set("outOfRangeDecr1", "-9223372036854775809");
+            try {
+                jedis.decr("outOfRangeDecr1");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
+
+            jedis.set("outOfRangeInc2", String.valueOf(Long.MAX_VALUE));
+            try {
+                jedis.incr("outOfRangeInc2");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
+
+            jedis.set("outOfRangeDecr2", String.valueOf(Long.MIN_VALUE));
+            try {
+                jedis.decr("outOfRangeDecr2");
 
                 assert false : "Exception has to be thrown!";
             }
@@ -308,16 +343,54 @@ public class RedisProtocolSelfTest extends GridCommonAbstractTest {
      */
     public void testIncrDecrBy() throws Exception {
         try (Jedis jedis = pool.getResource()) {
-            Assert.assertEquals(2, (long)jedis.incrBy("newKeyIncr1", 2));
-            Assert.assertEquals(-2, (long)jedis.decrBy("newKeyDecr1", 2));
+            Assert.assertEquals(2, (long)jedis.incrBy("newKeyIncrBy", 2));
+            Assert.assertEquals(-2, (long)jedis.decrBy("newKeyDecrBy", 2));
+
+            jedis.set("incrDecrKeyBy", "1");
+
+            Assert.assertEquals(11L, (long)jedis.incrBy("incrDecrKeyBy", 10));
 
-            jcache().put("incrKey2", 1L);
+            Assert.assertEquals(9L, (long)jedis.decrBy("incrDecrKeyBy", 2));
 
-            Assert.assertEquals(3L, (long)jedis.incrBy("incrKey2", 2));
+            jedis.set("outOfRangeIncrBy", "1");
+            try {
+                jedis.incrBy("outOfRangeIncrBy", Long.MAX_VALUE);
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
 
-            jcache().put("decrKey2", 2L);
+            jedis.set("outOfRangeDecrBy", "-1");
+            try {
+                jedis.decrBy("outOfRangeDecrBy", Long.MIN_VALUE);
 
-            Assert.assertEquals(0L, (long)jedis.decrBy("decrKey2", 2));
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
+
+            jedis.set("outOfRangeIncBy2", String.valueOf(Long.MAX_VALUE));
+            try {
+                jedis.incrBy("outOfRangeIncBy2", Long.MAX_VALUE);
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
+
+            jedis.set("outOfRangeDecrBy2", String.valueOf(Long.MIN_VALUE));
+            try {
+                jedis.decrBy("outOfRangeDecrBy2", Long.MIN_VALUE);
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1cc2925/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
index a275522..aac4170 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
@@ -20,8 +20,10 @@ package org.apache.ignite.internal.processors.rest.handlers.redis.string;
 import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.List;
+import org.apache.ignite.IgniteAtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
 import org.apache.ignite.internal.processors.rest.GridRestResponse;
 import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
@@ -44,14 +46,21 @@ public class GridRedisGetCommandHandler extends GridRedisRestCommandHandler {
         GET
     );
 
+    /** Grid context. */
+    private final GridKernalContext ctx;
+
     /**
      * Constructor.
      *
      * @param log Logger.
      * @param hnd Handler.
+     * @param ctx Context.
      */
-    public GridRedisGetCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
+    public GridRedisGetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd,
+        GridKernalContext ctx) {
         super(log, hnd);
+
+        this.ctx = ctx;
     }
 
     /** {@inheritDoc} */
@@ -75,8 +84,20 @@ public class GridRedisGetCommandHandler extends GridRedisRestCommandHandler {
 
     /** {@inheritDoc} */
     @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
-        if (restRes.getResponse() == null)
-            return GridRedisProtocolParser.nil();
+        if (restRes.getResponse() == null) {
+            // check if an atomic long with the key exists (related to incr/decr).
+            IgniteAtomicLong l = ctx.grid().atomicLong(params.get(0), 0, false);
+
+            long val;
+            try {
+                val = l.get();
+            }
+            catch (Exception ignored) {
+                return GridRedisProtocolParser.nil();
+            }
+
+            return GridRedisProtocolParser.toBulkString(val);
+        }
 
         if (restRes.getResponse() instanceof String)
             return GridRedisProtocolParser.toBulkString(restRes.getResponse());

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1cc2925/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
index 4052ad5..2036d76 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
 import org.apache.ignite.internal.processors.rest.GridRestResponse;
 import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
 import org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException;
+import org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisTypeException;
 import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
 import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
 import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
@@ -37,6 +38,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.ATOMIC_DECREMENT;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.ATOMIC_INCREMENT;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_REMOVE;
 import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.DECR;
 import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.DECRBY;
 import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.INCR;
@@ -89,10 +91,38 @@ public class GridRedisIncrDecrCommandHandler extends GridRedisRestCommandHandler
         if (getResp.getResponse() == null)
             restReq.initial(0L);
         else {
-            if (getResp.getResponse() instanceof Long && (Long)getResp.getResponse() <= Long.MAX_VALUE)
-                restReq.initial((Long)getResp.getResponse());
+            if (getResp.getResponse() instanceof String) {
+                Long init;
+
+                try {
+                    init = Long.parseLong((String)getResp.getResponse());
+
+                    restReq.initial(init);
+                }
+                catch (Exception e) {
+                    U.error(log, "An initial value must be numeric and in range", e);
+
+                    throw new GridRedisGenericException("An initial value must be numeric and in range");
+                }
+
+                if ((init == Long.MAX_VALUE && (msg.command() == INCR || msg.command() == INCRBY))
+                    || (init == Long.MIN_VALUE && (msg.command() == DECR || msg.command() == DECRBY)))
+                    throw new GridRedisGenericException("Increment or decrement would overflow");
+            }
             else
-                throw new GridRedisGenericException("An initial value must be numeric and in range");
+                throw new GridRedisTypeException("Operation against a key holding the wrong kind of value");
+
+            // remove from cache.
+            GridRestCacheRequest rmReq = new GridRestCacheRequest();
+
+            rmReq.clientId(msg.clientId());
+            rmReq.key(msg.key());
+            rmReq.command(CACHE_REMOVE);
+
+            Object rmResp = hnd.handle(rmReq).getResponse();
+
+            if (rmResp == null || !(boolean)rmResp)
+                throw new GridRedisGenericException("Cannot incr/decr on the non-atomiclong key");
         }
 
         restReq.clientId(msg.clientId());
@@ -101,10 +131,16 @@ public class GridRedisIncrDecrCommandHandler extends GridRedisRestCommandHandler
 
         if (msg.messageSize() > 2) {
             try {
-                restReq.delta(Long.valueOf(msg.aux(DELTA_POS)));
+                Long delta = Long.valueOf(msg.aux(DELTA_POS));
+
+                // check if it can be safely added.
+                safeAdd(restReq.initial(), delta);
+
+                restReq.delta(delta);
             }
-            catch (NumberFormatException e) {
-                U.error(log, "Wrong increment delta", e);
+            catch (NumberFormatException | ArithmeticException e) {
+                U.error(log, "An increment value must be numeric and in range", e);
+
                 throw new GridRedisGenericException("An increment value must be numeric and in range");
             }
         }
@@ -139,4 +175,19 @@ public class GridRedisIncrDecrCommandHandler extends GridRedisRestCommandHandler
         else
             return GridRedisProtocolParser.toTypeError("Value is non-numeric or out of range");
     }
+
+    /**
+     * Safely add long values.
+     *
+     * @param left A long value.
+     * @param right A long value.
+     * @return An addition result or an exception is thrown when overflow occurs.
+     */
+    private static long safeAdd(long left, long right) {
+        if (right > 0 ? left > Long.MAX_VALUE - right
+            : left < Long.MIN_VALUE - right) {
+            throw new ArithmeticException("Long overflow");
+        }
+        return left + right;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1cc2925/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
index 956d4cb..2eea0e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
@@ -20,8 +20,11 @@ package org.apache.ignite.internal.processors.rest.handlers.redis.string;
 import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.List;
+import org.apache.ignite.IgniteAtomicLong;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
 import org.apache.ignite.internal.processors.rest.GridRestResponse;
 import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
@@ -52,14 +55,21 @@ public class GridRedisSetCommandHandler extends GridRedisRestCommandHandler {
     /** Value position in Redis message. */
     private static final int VAL_POS = 2;
 
+    /** Grid context. */
+    private final GridKernalContext ctx;
+
     /**
      * Constructor.
      *
      * @param log Logger.
      * @param hnd Handler.
+     * @param ctx Context.
      */
-    public GridRedisSetCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
+    public GridRedisSetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd,
+        GridKernalContext ctx) {
         super(log, hnd);
+
+        this.ctx = ctx;
     }
 
     /** {@inheritDoc} */
@@ -71,6 +81,18 @@ public class GridRedisSetCommandHandler extends GridRedisRestCommandHandler {
     @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
         assert msg != null;
 
+        // check if an atomic long with the key exists (related to incr/decr).
+        IgniteAtomicLong l = ctx.grid().atomicLong(msg.key(), 0, false);
+
+        if (l != null) {
+            try {
+                l.close();
+            }
+            catch (IgniteException e) {
+                U.warn(log, "Failed to remove atomic long for key [" + msg.key() + "]");
+            }
+        }
+
         GridRestCacheRequest restReq = new GridRestCacheRequest();
 
         restReq.clientId(msg.clientId());

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1cc2925/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/GridRedisNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/GridRedisNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/GridRedisNioListener.java
index 45c8061..1042bdb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/GridRedisNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/GridRedisNioListener.java
@@ -67,8 +67,8 @@ public class GridRedisNioListener extends GridNioServerListenerAdapter<GridRedis
         addCommandHandler(new GridRedisConnectionCommandHandler());
 
         // string commands.
-        addCommandHandler(new GridRedisGetCommandHandler(log, hnd));
-        addCommandHandler(new GridRedisSetCommandHandler(log, hnd));
+        addCommandHandler(new GridRedisGetCommandHandler(log, hnd, ctx));
+        addCommandHandler(new GridRedisSetCommandHandler(log, hnd, ctx));
         addCommandHandler(new GridRedisMSetCommandHandler(log, hnd));
         addCommandHandler(new GridRedisMGetCommandHandler(log, hnd));
         addCommandHandler(new GridRedisIncrDecrCommandHandler(log, hnd));


[19/52] ignite git commit: Fixed JavaDoc warnings in Redis REST classes.

Posted by vo...@apache.org.
Fixed JavaDoc warnings in Redis REST classes.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/deee3b9f
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/deee3b9f
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/deee3b9f

Branch: refs/heads/master
Commit: deee3b9f64406e084e65c1f7e48e14270e8ad688
Parents: fabb77d
Author: devozerov <vo...@gridgain.com>
Authored: Fri Nov 25 11:15:17 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 11:15:17 2016 +0300

----------------------------------------------------------------------
 .../rest/handlers/redis/key/GridRedisDelCommandHandler.java | 9 +++++++--
 .../handlers/redis/key/GridRedisExistsCommandHandler.java   | 9 +++++++--
 .../redis/server/GridRedisDbSizeCommandHandler.java         | 7 ++++++-
 .../redis/string/GridRedisAppendCommandHandler.java         | 9 +++++++--
 .../handlers/redis/string/GridRedisGetCommandHandler.java   | 9 +++++++--
 .../redis/string/GridRedisGetRangeCommandHandler.java       | 9 +++++++--
 .../redis/string/GridRedisGetSetCommandHandler.java         | 9 +++++++--
 .../redis/string/GridRedisIncrDecrCommandHandler.java       | 9 +++++++--
 .../handlers/redis/string/GridRedisMGetCommandHandler.java  | 9 +++++++--
 .../handlers/redis/string/GridRedisMSetCommandHandler.java  | 9 +++++++--
 .../handlers/redis/string/GridRedisSetCommandHandler.java   | 9 +++++++--
 .../redis/string/GridRedisSetRangeCommandHandler.java       | 9 +++++++--
 .../redis/string/GridRedisStrlenCommandHandler.java         | 9 +++++++--
 13 files changed, 90 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisDelCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisDelCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisDelCommandHandler.java
index dc9af19..0b2e79b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisDelCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisDelCommandHandler.java
@@ -46,8 +46,13 @@ public class GridRedisDelCommandHandler extends GridRedisRestCommandHandler {
         DEL
     );
 
-    /** {@inheritDoc} */
-    public GridRedisDelCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisDelCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisExistsCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisExistsCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisExistsCommandHandler.java
index 26b3871..c0248a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisExistsCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisExistsCommandHandler.java
@@ -46,8 +46,13 @@ public class GridRedisExistsCommandHandler extends GridRedisRestCommandHandler {
         EXISTS
     );
 
-    /** {@inheritDoc} */
-    public GridRedisExistsCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisExistsCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/server/GridRedisDbSizeCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/server/GridRedisDbSizeCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/server/GridRedisDbSizeCommandHandler.java
index 56d5e28..c2a405e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/server/GridRedisDbSizeCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/server/GridRedisDbSizeCommandHandler.java
@@ -44,7 +44,12 @@ public class GridRedisDbSizeCommandHandler extends GridRedisRestCommandHandler {
         DBSIZE
     );
 
-    /** {@inheritDoc} */
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
     public GridRedisDbSizeCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
         super(log, hnd);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisAppendCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisAppendCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisAppendCommandHandler.java
index 7e27621..54b828f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisAppendCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisAppendCommandHandler.java
@@ -50,8 +50,13 @@ public class GridRedisAppendCommandHandler extends GridRedisRestCommandHandler {
     /** Position of the value. */
     private static final int VAL_POS = 2;
 
-    /** {@inheritDoc} */
-    public GridRedisAppendCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisAppendCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
index 7be53b2..a275522 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
@@ -44,8 +44,13 @@ public class GridRedisGetCommandHandler extends GridRedisRestCommandHandler {
         GET
     );
 
-    /** {@inheritDoc} */
-    public GridRedisGetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisGetCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetRangeCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetRangeCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetRangeCommandHandler.java
index 4478c19..38e3ffe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetRangeCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetRangeCommandHandler.java
@@ -51,8 +51,13 @@ public class GridRedisGetRangeCommandHandler extends GridRedisRestCommandHandler
     /** End offset position in Redis message parameters. */
     private static final int END_OFFSET_POS = 2;
 
-    /** {@inheritDoc} */
-    public GridRedisGetRangeCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisGetRangeCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetSetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetSetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetSetCommandHandler.java
index 781b8ab..88067e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetSetCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetSetCommandHandler.java
@@ -48,8 +48,13 @@ public class GridRedisGetSetCommandHandler extends GridRedisRestCommandHandler {
     /** Value position in Redis message. */
     private static final int VAL_POS = 2;
 
-    /** {@inheritDoc} */
-    public GridRedisGetSetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisGetSetCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
index 382696d..4052ad5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
@@ -57,8 +57,13 @@ public class GridRedisIncrDecrCommandHandler extends GridRedisRestCommandHandler
     /** Delta position in the message. */
     private static final int DELTA_POS = 2;
 
-    /** {@inheritDoc} */
-    public GridRedisIncrDecrCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisIncrDecrCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMGetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMGetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMGetCommandHandler.java
index 87557d6..1ef8348 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMGetCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMGetCommandHandler.java
@@ -46,8 +46,13 @@ public class GridRedisMGetCommandHandler extends GridRedisRestCommandHandler {
         MGET
     );
 
-    /** {@inheritDoc} */
-    public GridRedisMGetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisMGetCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMSetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMSetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMSetCommandHandler.java
index 3e9076b..493a699 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMSetCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMSetCommandHandler.java
@@ -46,8 +46,13 @@ public class GridRedisMSetCommandHandler extends GridRedisRestCommandHandler {
         MSET
     );
 
-    /** {@inheritDoc} */
-    public GridRedisMSetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisMSetCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
index c33a8ca..956d4cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
@@ -52,8 +52,13 @@ public class GridRedisSetCommandHandler extends GridRedisRestCommandHandler {
     /** Value position in Redis message. */
     private static final int VAL_POS = 2;
 
-    /** {@inheritDoc} */
-    public GridRedisSetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisSetCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetRangeCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetRangeCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetRangeCommandHandler.java
index 2cd931e..0de4412 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetRangeCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetRangeCommandHandler.java
@@ -55,8 +55,13 @@ public class GridRedisSetRangeCommandHandler extends GridRedisRestCommandHandler
     /** Maximum offset. */
     private static final int MAX_OFFSET = 536870911;
 
-    /** {@inheritDoc} */
-    public GridRedisSetRangeCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisSetRangeCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/deee3b9f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisStrlenCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisStrlenCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisStrlenCommandHandler.java
index b13ae62..79b0433 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisStrlenCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisStrlenCommandHandler.java
@@ -44,8 +44,13 @@ public class GridRedisStrlenCommandHandler extends GridRedisRestCommandHandler {
         STRLEN
     );
 
-    /** {@inheritDoc} */
-    public GridRedisStrlenCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param hnd Handler.
+     */
+    public GridRedisStrlenCommandHandler(IgniteLogger log, GridRestProtocolHandler hnd) {
         super(log, hnd);
     }
 


[10/52] ignite git commit: IGNITE-4096: Added ODBC-DML tests. Fixed issues. This closes #1178.

Posted by vo...@apache.org.
IGNITE-4096: Added ODBC-DML tests. Fixed issues. This closes #1178.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/f4524a6a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/f4524a6a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/f4524a6a

Branch: refs/heads/master
Commit: f4524a6a7b86df9cd008dcd22fa5cfa400c3af27
Parents: 4ba1386
Author: isapego <ig...@gmail.com>
Authored: Wed Nov 23 17:58:41 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Nov 23 17:58:41 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcRequestHandler.java     |   2 +-
 .../cpp/core-test/config/cache-query.xml        |  10 +
 .../src/application_data_buffer_test.cpp        |   2 +-
 .../cpp/odbc-test/src/queries_test.cpp          | 336 +++++++++++++++++++
 .../ignite/odbc/app/application_data_buffer.h   |   4 +-
 .../odbc/include/ignite/odbc/app/parameter.h    |   2 +-
 .../cpp/odbc/include/ignite/odbc/statement.h    |  48 ++-
 .../odbc/src/app/application_data_buffer.cpp    |  57 ++--
 modules/platforms/cpp/odbc/src/odbc.cpp         |   4 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |   4 +
 modules/platforms/cpp/odbc/src/statement.cpp    |  45 ++-
 modules/platforms/cpp/odbc/src/utility.cpp      |   2 +-
 12 files changed, 433 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index eef9945..cf2615d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
@@ -184,7 +184,7 @@ public class OdbcRequestHandler {
             qry.setDistributedJoins(distributedJoins);
             qry.setEnforceJoinOrder(enforceJoinOrder);
 
-            IgniteCache<Object, Object> cache = ctx.grid().cache(req.cacheName());
+            IgniteCache<Object, Object> cache = ctx.grid().cache(req.cacheName()).withKeepBinary();
 
             if (cache == null)
                 return new OdbcResponse(OdbcResponse.STATUS_FAILED,

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/core-test/config/cache-query.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-query.xml b/modules/platforms/cpp/core-test/config/cache-query.xml
index bb18f7c..dead2b1 100644
--- a/modules/platforms/cpp/core-test/config/cache-query.xml
+++ b/modules/platforms/cpp/core-test/config/cache-query.xml
@@ -69,6 +69,16 @@
                                     </list>
                                 </property>
                             </bean>
+
+                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                                <property name="valueType" value="QueryRelation"/>
+                                <property name="queryFields">
+                                    <map>
+                                        <entry key="personId" value="java.lang.Integer"/>
+                                        <entry key="someVal" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                            </bean>
                         </list>
                     </property>
                 </bean>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp b/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp
index f62bcd6..fe50295 100644
--- a/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp
@@ -117,7 +117,7 @@ BOOST_AUTO_TEST_CASE(TestPutGuidToString)
 BOOST_AUTO_TEST_CASE(TestGetGuidFromString)
 {
     char buffer[] = "1da1ef8f-39ff-4d62-8b72-e8e9f3371801";
-    SqlLen reslen = 0;
+    SqlLen reslen = sizeof(buffer) - 1;
 
     ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer) - 1, &reslen, 0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
index db9dafb..554cee2 100644
--- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
@@ -288,6 +288,77 @@ struct QueriesTestSuiteFixture
         return res;
     }
 
+    static std::string getTestString(int64_t ind)
+    {
+        std::stringstream builder;
+
+        builder << "String#" << ind;
+
+        return builder.str();
+    }
+
+    /**
+     * Insert requested number of TestType vlaues with all defaults except
+     * for the strFields, which are generated using getTestString().
+     *
+     * @param num Number of records to insert.
+     * @param merge Set to true to use merge instead.
+     */
+    void InsertTestStrings(int recordsNum, bool merge = false)
+    {
+        SQLCHAR insertReq[] = "INSERT INTO TestType(_key, strField) VALUES(?, ?)";
+        SQLCHAR mergeReq[] = "MERGE INTO TestType(_key, strField) VALUES(?, ?)";
+
+        SQLRETURN ret;
+
+        ret = SQLPrepare(stmt, merge ? mergeReq : insertReq, SQL_NTS);
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+        int64_t key = 0;
+        char strField[1024] = { 0 };
+        SQLLEN strFieldLen = 0;
+
+        // Binding parameters.
+        ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0);
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+        ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_CHAR, SQL_VARCHAR, sizeof(strField),
+            sizeof(strField), &strField, sizeof(strField), &strFieldLen);
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+        // Inserting values.
+        for (SQLSMALLINT i = 0; i < recordsNum; ++i)
+        {
+            key = i + 1;
+            std::string val = getTestString(i);
+
+            strncpy(strField, val.c_str(), sizeof(strField));
+            strFieldLen = SQL_NTS;
+
+            ret = SQLExecute(stmt);
+
+            if (!SQL_SUCCEEDED(ret))
+                BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+            ret = SQLMoreResults(stmt);
+
+            if (ret != SQL_NO_DATA)
+                BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+        }
+
+        // Resetting parameters.
+        ret = SQLFreeStmt(stmt, SQL_RESET_PARAMS);
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+    }
+
     /** Node started during the test. */
     Ignite grid;
 
@@ -939,4 +1010,269 @@ BOOST_AUTO_TEST_CASE(TestDistributedJoinsWithOldVersion)
     BOOST_CHECK_LT(rowsNum, entriesNum);
 }
 
+BOOST_AUTO_TEST_CASE(TestInsertSelect)
+{
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache");
+
+    const int recordsNum = 100;
+
+    // Inserting values.
+    InsertTestStrings(recordsNum);
+
+    int64_t key = 0;
+    char strField[1024] = { 0 };
+    SQLLEN strFieldLen = 0;
+
+    // Binding columns.
+    SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &key, 0, 0);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    // Binding columns.
+    ret = SQLBindCol(stmt, 2, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    // Just selecting everything to make sure everything is OK
+    SQLCHAR selectReq[] = "SELECT _key, strField FROM TestType ORDER BY _key";
+
+    ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq));
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    int selectedRecordsNum = 0;
+
+    ret = SQL_SUCCESS;
+
+    while (ret == SQL_SUCCESS)
+    {
+        ret = SQLFetch(stmt);
+
+        if (ret == SQL_NO_DATA)
+            break;
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+        std::string expectedStr = getTestString(selectedRecordsNum);
+        int64_t expectedKey = selectedRecordsNum + 1;
+
+        BOOST_CHECK_EQUAL(key, expectedKey);
+
+        BOOST_CHECK_EQUAL(std::string(strField, strFieldLen), expectedStr);
+
+        ++selectedRecordsNum;
+    }
+
+    BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum);
+}
+
+BOOST_AUTO_TEST_CASE(TestInsertUpdateSelect)
+{
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache");
+
+    const int recordsNum = 100;
+
+    // Inserting values.
+    InsertTestStrings(recordsNum);
+
+    int64_t key = 0;
+    char strField[1024] = { 0 };
+    SQLLEN strFieldLen = 0;
+
+    SQLCHAR updateReq[] = "UPDATE TestType SET strField = 'Updated value' WHERE _key = 42";
+
+    SQLRETURN ret = SQLExecDirect(stmt, updateReq, SQL_NTS);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLFreeStmt(stmt, SQL_CLOSE);
+
+    // Binding columns.
+    ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &key, 0, 0);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    // Binding columns.
+    ret = SQLBindCol(stmt, 2, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    // Just selecting everything to make sure everything is OK
+    SQLCHAR selectReq[] = "SELECT _key, strField FROM TestType ORDER BY _key";
+
+    ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq));
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    int selectedRecordsNum = 0;
+
+    ret = SQL_SUCCESS;
+
+    while (ret == SQL_SUCCESS)
+    {
+        ret = SQLFetch(stmt);
+
+        if (ret == SQL_NO_DATA)
+            break;
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+        int64_t expectedKey = selectedRecordsNum + 1;
+        std::string expectedStr;
+
+        BOOST_CHECK_EQUAL(key, expectedKey);
+
+        if (expectedKey == 42)
+            expectedStr = "Updated value";
+        else
+            expectedStr = getTestString(selectedRecordsNum);
+
+        BOOST_CHECK_EQUAL(std::string(strField, strFieldLen), expectedStr);
+
+        ++selectedRecordsNum;
+    }
+
+    BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum);
+}
+
+BOOST_AUTO_TEST_CASE(TestInsertDeleteSelect)
+{
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache");
+
+    const int recordsNum = 100;
+
+    // Inserting values.
+    InsertTestStrings(recordsNum);
+
+    int64_t key = 0;
+    char strField[1024] = { 0 };
+    SQLLEN strFieldLen = 0;
+
+    SQLCHAR updateReq[] = "DELETE FROM TestType WHERE (_key % 2) = 1";
+
+    SQLRETURN ret = SQLExecDirect(stmt, updateReq, SQL_NTS);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLFreeStmt(stmt, SQL_CLOSE);
+
+    // Binding columns.
+    ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &key, 0, 0);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    // Binding columns.
+    ret = SQLBindCol(stmt, 2, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    // Just selecting everything to make sure everything is OK
+    SQLCHAR selectReq[] = "SELECT _key, strField FROM TestType ORDER BY _key";
+
+    ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq));
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    int selectedRecordsNum = 0;
+
+    ret = SQL_SUCCESS;
+
+    while (ret == SQL_SUCCESS)
+    {
+        ret = SQLFetch(stmt);
+
+        if (ret == SQL_NO_DATA)
+            break;
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+        int64_t expectedKey = (selectedRecordsNum + 1) * 2;
+        std::string expectedStr = getTestString(expectedKey - 1);
+
+        BOOST_CHECK_EQUAL(key, expectedKey);
+        BOOST_CHECK_EQUAL(std::string(strField, strFieldLen), expectedStr);
+
+        ++selectedRecordsNum;
+    }
+
+    BOOST_CHECK_EQUAL(recordsNum / 2, selectedRecordsNum);
+}
+
+BOOST_AUTO_TEST_CASE(TestInsertMergeSelect)
+{
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache");
+
+    const int recordsNum = 100;
+
+    // Inserting values.
+    InsertTestStrings(recordsNum / 2);
+
+    // Merging values.
+    InsertTestStrings(recordsNum, true);
+
+    int64_t key = 0;
+    char strField[1024] = { 0 };
+    SQLLEN strFieldLen = 0;
+
+    // Binding columns.
+    SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &key, 0, 0);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    // Binding columns.
+    ret = SQLBindCol(stmt, 2, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    // Just selecting everything to make sure everything is OK
+    SQLCHAR selectReq[] = "SELECT _key, strField FROM TestType ORDER BY _key";
+
+    ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq));
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    int selectedRecordsNum = 0;
+
+    ret = SQL_SUCCESS;
+
+    while (ret == SQL_SUCCESS)
+    {
+        ret = SQLFetch(stmt);
+
+        if (ret == SQL_NO_DATA)
+            break;
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+        std::string expectedStr = getTestString(selectedRecordsNum);
+        int64_t expectedKey = selectedRecordsNum + 1;
+
+        BOOST_CHECK_EQUAL(key, expectedKey);
+
+        BOOST_CHECK_EQUAL(std::string(strField, strFieldLen), expectedStr);
+
+        ++selectedRecordsNum;
+    }
+
+    BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum);
+}
+
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h b/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h
index 0ce7818..18ac36a 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h
@@ -311,7 +311,7 @@ namespace ignite
                  * @return Size of the data that is going to be provided
                  *     at execution.
                  */
-                size_t GetDataAtExecSize() const;
+                SqlLen GetDataAtExecSize() const;
 
                 /**
                  * Get size of the input buffer.
@@ -319,7 +319,7 @@ namespace ignite
                  * @return Input buffer size, or zero if the data is going
                  *     to be provided at execution.
                  */
-                size_t GetInputSize() const;
+                SqlLen GetInputSize() const;
 
                 /**
                  * Get buffer type.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h
index 0bd9395..1cf85b5 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h
@@ -81,7 +81,7 @@ namespace ignite
                  * Write request using provided writer.
                  * @param writer Writer.
                  */
-                void Write(ignite::impl::binary::BinaryWriterImpl& writer) const;
+                void Write(impl::binary::BinaryWriterImpl& writer) const;
 
                 /**
                  * Get data buffer.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
index db6205e..981ade1 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
@@ -168,41 +168,19 @@ namespace ignite
             /**
              * Prepare SQL query.
              *
-             * @note Only SELECT queries are supported currently.
              * @param query SQL query.
              */
             void PrepareSqlQuery(const std::string& query);
-
-            /**
-             * Prepare SQL query.
-             *
-             * @note Only SELECT queries are supported currently.
-             * @param query SQL query.
-             * @param len Query length.
-             */
-            void PrepareSqlQuery(const char* query, size_t len);
             
             /**
              * Execute SQL query.
              *
-             * @note Only SELECT queries are supported currently.
              * @param query SQL query.
              */
             void ExecuteSqlQuery(const std::string& query);
 
             /**
              * Execute SQL query.
-             *
-             * @note Only SELECT queries are supported currently.
-             * @param query SQL query.
-             * @param len Query length.
-             */
-            void ExecuteSqlQuery(const char* query, size_t len);
-
-            /**
-             * Execute SQL query.
-             *
-             * @note Only SELECT queries are supported currently.
              */
             void ExecuteSqlQuery();
 
@@ -300,6 +278,13 @@ namespace ignite
             bool DataAvailable() const;
 
             /**
+             * Next results.
+             *
+             * Move to next result set or affected rows number.
+             */
+            void NextResults();
+
+            /**
              * Get column attribute.
              *
              * @param colIdx Column index.
@@ -417,27 +402,22 @@ namespace ignite
             /**
              * Prepare SQL query.
              *
-             * @note Only SELECT queries are supported currently.
              * @param query SQL query.
-             * @param len Query length.
              * @return Operation result.
              */
-            SqlResult InternalPrepareSqlQuery(const char* query, size_t len);
+            SqlResult InternalPrepareSqlQuery(const std::string& query);
             
             /**
              * Execute SQL query.
              *
-             * @note Only SELECT queries are supported currently.
              * @param query SQL query.
-             * @param len Query length.
              * @return Operation result.
              */
-            SqlResult InternalExecuteSqlQuery(const char* query, size_t len);
+            SqlResult InternalExecuteSqlQuery(const std::string& query);
 
             /**
              * Execute SQL query.
              *
-             * @note Only SELECT queries are supported currently.
              * @return Operation result.
              */
             SqlResult InternalExecuteSqlQuery();
@@ -528,10 +508,20 @@ namespace ignite
              * Get type info.
              *
              * @param sqlType SQL type for which to return info or SQL_ALL_TYPES.
+             * @return Operation result.
              */
             SqlResult InternalExecuteGetTypeInfoQuery(int16_t sqlType);
 
             /**
+             * Next results.
+             *
+             * Move to next result set or affected rows number.
+             *
+             * @return Operation result.
+             */
+            SqlResult InternalNextResults();
+
+            /**
              * Get column attribute.
              *
              * @param colIdx Column index.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp
index 078e691..71c5f39 100644
--- a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp
+++ b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp
@@ -368,7 +368,7 @@ namespace ignite
                 PutNum(value);
             }
 
-            int32_t ApplicationDataBuffer::PutString(const std::string & value)
+            int32_t ApplicationDataBuffer::PutString(const std::string& value)
             {
                 using namespace type_traits;
 
@@ -920,8 +920,12 @@ namespace ignite
                         if (!paramLen)
                             break;
 
-                        res.assign(reinterpret_cast<const char*>(GetData()),
-                                   std::min(maxLen, paramLen));
+                        res = utility::SqlStringToString(
+                            reinterpret_cast<const unsigned char*>(GetData()), static_cast<int32_t>(paramLen));
+
+                        if (res.size() > maxLen)
+                            res.resize(maxLen);
+
                         break;
                     }
 
@@ -1024,12 +1028,13 @@ namespace ignite
                 {
                     case IGNITE_ODBC_C_TYPE_CHAR:
                     {
-                        size_t paramLen = GetInputSize();
+                        SqlLen paramLen = GetInputSize();
 
                         if (!paramLen)
                             break;
 
-                        std::string str(reinterpret_cast<const char*>(GetData()), paramLen);
+                        std::string str = utility::SqlStringToString(
+                            reinterpret_cast<const unsigned char*>(GetData()), static_cast<int32_t>(paramLen));
 
                         std::stringstream converter;
 
@@ -1096,7 +1101,7 @@ namespace ignite
                 {
                     case IGNITE_ODBC_C_TYPE_CHAR:
                     {
-                        size_t paramLen = GetInputSize();
+                        SqlLen paramLen = GetInputSize();
 
                         if (!paramLen)
                             break;
@@ -1239,7 +1244,7 @@ namespace ignite
 
                     case IGNITE_ODBC_C_TYPE_CHAR:
                     {
-                        size_t paramLen = GetInputSize();
+                        SqlLen paramLen = GetInputSize();
 
                         if (!paramLen)
                             break;
@@ -1302,7 +1307,7 @@ namespace ignite
 
                     case IGNITE_ODBC_C_TYPE_CHAR:
                     {
-                        size_t paramLen = GetInputSize();
+                        SqlLen paramLen = GetInputSize();
 
                         if (!paramLen)
                             break;
@@ -1334,7 +1339,7 @@ namespace ignite
                 {
                     case IGNITE_ODBC_C_TYPE_CHAR:
                     {
-                        size_t paramLen = GetInputSize();
+                        SqlLen paramLen = GetInputSize();
 
                         if (!paramLen)
                             break;
@@ -1422,7 +1427,7 @@ namespace ignite
                 return ilen <= SQL_LEN_DATA_AT_EXEC_OFFSET || ilen == SQL_DATA_AT_EXEC;
             }
 
-            size_t ApplicationDataBuffer::GetDataAtExecSize() const
+            SqlLen ApplicationDataBuffer::GetDataAtExecSize() const
             {
                 using namespace type_traits;
 
@@ -1440,7 +1445,7 @@ namespace ignite
                         int32_t ilen = static_cast<int32_t>(*resLenPtr);
 
                         if (ilen <= SQL_LEN_DATA_AT_EXEC_OFFSET)
-                            ilen = static_cast<size_t>(SQL_LEN_DATA_AT_EXEC(ilen));
+                            ilen = SQL_LEN_DATA_AT_EXEC(ilen);
                         else
                             ilen = 0;
 
@@ -1452,41 +1457,41 @@ namespace ignite
 
                     case IGNITE_ODBC_C_TYPE_SIGNED_SHORT:
                     case IGNITE_ODBC_C_TYPE_UNSIGNED_SHORT:
-                        return sizeof(short);
+                        return static_cast<SqlLen>(sizeof(short));
 
                     case IGNITE_ODBC_C_TYPE_SIGNED_LONG:
                     case IGNITE_ODBC_C_TYPE_UNSIGNED_LONG:
-                        return sizeof(long);
+                        return static_cast<SqlLen>(sizeof(long));
 
                     case IGNITE_ODBC_C_TYPE_FLOAT:
-                        return sizeof(float);
+                        return static_cast<SqlLen>(sizeof(float));
 
                     case IGNITE_ODBC_C_TYPE_DOUBLE:
-                        return sizeof(double);
+                        return static_cast<SqlLen>(sizeof(double));
 
                     case IGNITE_ODBC_C_TYPE_BIT:
                     case IGNITE_ODBC_C_TYPE_SIGNED_TINYINT:
                     case IGNITE_ODBC_C_TYPE_UNSIGNED_TINYINT:
-                        return sizeof(char);
+                        return static_cast<SqlLen>(sizeof(char));
 
                     case IGNITE_ODBC_C_TYPE_SIGNED_BIGINT:
                     case IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT:
-                        return sizeof(SQLBIGINT);
+                        return static_cast<SqlLen>(sizeof(SQLBIGINT));
 
                     case IGNITE_ODBC_C_TYPE_TDATE:
-                        return sizeof(SQL_DATE_STRUCT);
+                        return static_cast<SqlLen>(sizeof(SQL_DATE_STRUCT));
 
                     case IGNITE_ODBC_C_TYPE_TTIME:
-                        return sizeof(SQL_TIME_STRUCT);
+                        return static_cast<SqlLen>(sizeof(SQL_TIME_STRUCT));
 
                     case IGNITE_ODBC_C_TYPE_TTIMESTAMP:
-                        return sizeof(SQL_TIMESTAMP_STRUCT);
+                        return static_cast<SqlLen>(sizeof(SQL_TIMESTAMP_STRUCT));
 
                     case IGNITE_ODBC_C_TYPE_NUMERIC:
-                        return sizeof(SQL_NUMERIC_STRUCT);
+                        return static_cast<SqlLen>(sizeof(SQL_NUMERIC_STRUCT));
 
                     case IGNITE_ODBC_C_TYPE_GUID:
-                        return sizeof(SQLGUID);
+                        return static_cast<SqlLen>(sizeof(SQLGUID));
 
                     case IGNITE_ODBC_C_TYPE_DEFAULT:
                     case IGNITE_ODBC_C_TYPE_UNSUPPORTED:
@@ -1497,10 +1502,14 @@ namespace ignite
                 return 0;
             }
 
-            size_t ApplicationDataBuffer::GetInputSize() const
+            SqlLen ApplicationDataBuffer::GetInputSize() const
             {
                 if (!IsDataAtExec())
-                    return static_cast<size_t>(GetSize());
+                {
+                    const SqlLen *len = GetResLen();
+
+                    return len ? *len : SQL_DEFAULT_PARAM;
+                }
 
                 return GetDataAtExecSize();
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/odbc/src/odbc.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp
index 684ed08..8ecc0f8 100644
--- a/modules/platforms/cpp/odbc/src/odbc.cpp
+++ b/modules/platforms/cpp/odbc/src/odbc.cpp
@@ -600,7 +600,9 @@ namespace ignite
         if (!statement)
             return SQL_INVALID_HANDLE;
 
-        return statement->DataAvailable() ? SQL_SUCCESS : SQL_NO_DATA;
+        statement->NextResults();
+
+        return statement->GetDiagnosticRecords().GetReturnCode();
     }
 
     SQLRETURN SQLBindParameter(SQLHSTMT     stmt,

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/odbc/src/query/data_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp
index 7a25ccb..8f76153 100644
--- a/modules/platforms/cpp/odbc/src/query/data_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp
@@ -146,8 +146,12 @@ namespace ignite
                 SqlResult result = MakeRequestClose();
 
                 if (result == SQL_RESULT_SUCCESS)
+                {
                     cursor.reset();
 
+                    resultMeta.clear();
+                }
+
                 return result;
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/odbc/src/statement.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp
index f1a577a..49a9acf 100644
--- a/modules/platforms/cpp/odbc/src/statement.cpp
+++ b/modules/platforms/cpp/odbc/src/statement.cpp
@@ -336,39 +336,27 @@ namespace ignite
 
         void Statement::PrepareSqlQuery(const std::string& query)
         {
-            return PrepareSqlQuery(query.data(), query.size());
+            IGNITE_ODBC_API_CALL(InternalPrepareSqlQuery(query));
         }
 
-        void Statement::PrepareSqlQuery(const char* query, size_t len)
-        {
-            IGNITE_ODBC_API_CALL(InternalPrepareSqlQuery(query, len));
-        }
-
-        SqlResult Statement::InternalPrepareSqlQuery(const char* query, size_t len)
+        SqlResult Statement::InternalPrepareSqlQuery(const std::string& query)
         {
             if (currentQuery.get())
                 currentQuery->Close();
 
-            std::string sql(query, len);
-
-            currentQuery.reset(new query::DataQuery(*this, connection, sql, paramBindings));
+            currentQuery.reset(new query::DataQuery(*this, connection, query, paramBindings));
 
             return SQL_RESULT_SUCCESS;
         }
 
         void Statement::ExecuteSqlQuery(const std::string& query)
         {
-            ExecuteSqlQuery(query.data(), query.size());
+            IGNITE_ODBC_API_CALL(InternalExecuteSqlQuery(query));
         }
 
-        void Statement::ExecuteSqlQuery(const char* query, size_t len)
+        SqlResult Statement::InternalExecuteSqlQuery(const std::string& query)
         {
-            IGNITE_ODBC_API_CALL(InternalExecuteSqlQuery(query, len));
-        }
-
-        SqlResult Statement::InternalExecuteSqlQuery(const char* query, size_t len)
-        {
-            SqlResult result = InternalPrepareSqlQuery(query, len);
+            SqlResult result = InternalPrepareSqlQuery(query);
 
             if (result != SQL_RESULT_SUCCESS)
                 return result;
@@ -546,11 +534,7 @@ namespace ignite
         SqlResult Statement::InternalClose()
         {
             if (!currentQuery.get())
-            {
-                AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE, "Cursor is not in the open state.");
-
-                return SQL_RESULT_ERROR;
-            }
+                return SQL_RESULT_SUCCESS;
 
             SqlResult result = currentQuery->Close();
 
@@ -604,6 +588,21 @@ namespace ignite
             return currentQuery.get() && currentQuery->DataAvailable();
         }
 
+        void Statement::NextResults()
+        {
+            IGNITE_ODBC_API_CALL(InternalNextResults());
+        }
+
+        SqlResult Statement::InternalNextResults()
+        {
+            if (!currentQuery.get())
+                return SQL_RESULT_NO_DATA;
+
+            SqlResult result = currentQuery->Close();
+
+            return result == SQL_RESULT_SUCCESS ? SQL_RESULT_NO_DATA : result;
+        }
+
         void Statement::GetColumnAttribute(uint16_t colIdx, uint16_t attrId,
             char* strbuf, int16_t buflen, int16_t* reslen, SqlLen* numbuf)
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4524a6a/modules/platforms/cpp/odbc/src/utility.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/utility.cpp b/modules/platforms/cpp/odbc/src/utility.cpp
index 19be799..d4a48af 100644
--- a/modules/platforms/cpp/odbc/src/utility.cpp
+++ b/modules/platforms/cpp/odbc/src/utility.cpp
@@ -131,7 +131,7 @@ namespace ignite
 
             if (sqlStrLen == SQL_NTS)
                 res.assign(sqlStrC);
-            else
+            else if (sqlStrLen > 0)
                 res.assign(sqlStrC, sqlStrLen);
 
             return res;


[25/52] ignite git commit: IGNITE-4279: Fixed most of the failures in JDBC suite. They were caused by connection leak in old test JdbcNoDefaultCacheTest.

Posted by vo...@apache.org.
IGNITE-4279: Fixed most of the failures in JDBC suite. They were caused by connection leak in old test JdbcNoDefaultCacheTest.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/43ec85f2
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/43ec85f2
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/43ec85f2

Branch: refs/heads/master
Commit: 43ec85f2be5bc4b18f3cff6630d5adef60930fdc
Parents: ccbe600
Author: devozerov <vo...@gridgain.com>
Authored: Fri Nov 25 13:55:50 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 13:55:50 2016 +0300

----------------------------------------------------------------------
 .../internal/jdbc2/JdbcNoDefaultCacheTest.java  | 40 ++++++++++----------
 1 file changed, 21 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/43ec85f2/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoDefaultCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoDefaultCacheTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoDefaultCacheTest.java
index 0acaa05..cdcda48 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoDefaultCacheTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoDefaultCacheTest.java
@@ -127,35 +127,37 @@ public class JdbcNoDefaultCacheTest extends GridCommonAbstractTest {
     public void testNoCacheNameQuery() throws Exception {
         Statement stmt;
 
-        stmt = DriverManager.getConnection(CFG_URL_PREFIX + CFG_URL).createStatement();
+        try (Connection conn = DriverManager.getConnection(CFG_URL_PREFIX + CFG_URL)) {
+            stmt = conn.createStatement();
 
-        assertNotNull(stmt);
-        assertFalse(stmt.isClosed());
+            assertNotNull(stmt);
+            assertFalse(stmt.isClosed());
 
-        stmt.execute("select t._key, t._val from \"cache1\".Integer t");
+            stmt.execute("select t._key, t._val from \"cache1\".Integer t");
 
-        ResultSet rs = stmt.getResultSet();
+            ResultSet rs = stmt.getResultSet();
 
-        while(rs.next())
-            assertEquals(rs.getInt(2), rs.getInt(1) * 2);
+            while(rs.next())
+                assertEquals(rs.getInt(2), rs.getInt(1) * 2);
 
-        stmt.execute("select t._key, t._val from \"cache2\".Integer t");
+            stmt.execute("select t._key, t._val from \"cache2\".Integer t");
 
-        rs = stmt.getResultSet();
+            rs = stmt.getResultSet();
 
-        while(rs.next())
-            assertEquals(rs.getInt(2), rs.getInt(1) * 3);
+            while(rs.next())
+                assertEquals(rs.getInt(2), rs.getInt(1) * 3);
 
-        stmt.execute("select t._key, t._val, v._val " +
-            "from \"cache1\".Integer t join \"cache2\".Integer v on t._key = v._key");
+            stmt.execute("select t._key, t._val, v._val " +
+                "from \"cache1\".Integer t join \"cache2\".Integer v on t._key = v._key");
 
-        rs = stmt.getResultSet();
+            rs = stmt.getResultSet();
 
-        while(rs.next()) {
-            assertEquals(rs.getInt(2), rs.getInt(1) * 2);
-            assertEquals(rs.getInt(3), rs.getInt(1) * 3);
-        }
+            while(rs.next()) {
+                assertEquals(rs.getInt(2), rs.getInt(1) * 2);
+                assertEquals(rs.getInt(3), rs.getInt(1) * 3);
+            }
 
-        stmt.close();
+            stmt.close();
+        }
     }
 }


[29/52] ignite git commit: IGNITE-4309: Commented out failing CacheExampleMultiNodeSelfTest.testCacheLockExample.

Posted by vo...@apache.org.
IGNITE-4309: Commented out failing CacheExampleMultiNodeSelfTest.testCacheLockExample.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/dda4fc94
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/dda4fc94
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/dda4fc94

Branch: refs/heads/master
Commit: dda4fc94e3fb53a0dc458e1035adb2d3d182d7c0
Parents: a6cb18a
Author: devozerov <vo...@gridgain.com>
Authored: Fri Nov 25 15:13:49 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 15:13:49 2016 +0300

----------------------------------------------------------------------
 .../ignite/examples/CacheExamplesMultiNodeSelfTest.java       | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dda4fc94/examples/src/test/java/org/apache/ignite/examples/CacheExamplesMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/examples/CacheExamplesMultiNodeSelfTest.java b/examples/src/test/java/org/apache/ignite/examples/CacheExamplesMultiNodeSelfTest.java
index 06ceaac..6de0273 100644
--- a/examples/src/test/java/org/apache/ignite/examples/CacheExamplesMultiNodeSelfTest.java
+++ b/examples/src/test/java/org/apache/ignite/examples/CacheExamplesMultiNodeSelfTest.java
@@ -35,4 +35,11 @@ public class CacheExamplesMultiNodeSelfTest extends CacheExamplesSelfTest {
     @Override protected long getTestTimeout() {
         return 10 * 60 * 1000;
     }
+
+    /** {@inheritDoc} */
+    @Override public void testCacheLockExample() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-4309");
+
+        super.testCacheLockExample();
+    }
 }
\ No newline at end of file


[37/52] ignite git commit: IGNITE-4026: Fixed BinaryObjectBuilder.build() can fail if one of the fields is Externalizable, enum from binary object. This closes #1281. This closes #1289.

Posted by vo...@apache.org.
IGNITE-4026: Fixed BinaryObjectBuilder.build() can fail if one of the fields is Externalizable, enum from binary object. This closes #1281. This closes #1289.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/0b7c62d2
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/0b7c62d2
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/0b7c62d2

Branch: refs/heads/master
Commit: 0b7c62d2939854223653d832a616ddb61211bf22
Parents: d1cc292
Author: dkarachentsev <dk...@gridgain.com>
Authored: Mon Nov 28 11:30:14 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Nov 28 11:30:14 2016 +0300

----------------------------------------------------------------------
 .../internal/binary/GridBinaryMarshaller.java   |   2 +-
 .../binary/builder/BinaryBuilderReader.java     |  11 ++
 .../binary/builder/BinaryBuilderSerializer.java |   4 +
 .../binary/builder/BinaryObjectBuilderImpl.java |  10 ++
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 157 ++++++++++++++++++-
 5 files changed, 181 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b7c62d2/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
index ad63521..624fa33 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
@@ -43,7 +43,7 @@ public class GridBinaryMarshaller {
     private static final ThreadLocal<BinaryContext> BINARY_CTX = new ThreadLocal<>();
 
     /** */
-    static final byte OPTM_MARSH = -2;
+    public static final byte OPTM_MARSH = -2;
 
     /** */
     public static final byte BYTE = 1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b7c62d2/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java
index 347fb2b..baaabd6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java
@@ -33,6 +33,7 @@ import java.sql.Timestamp;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.Map;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
@@ -757,6 +758,16 @@ public class BinaryBuilderReader implements BinaryPositionReadable {
                 return new BinaryPlainBinaryObject(binaryObj);
             }
 
+            case GridBinaryMarshaller.OPTM_MARSH: {
+                final BinaryHeapInputStream bin = BinaryHeapInputStream.create(arr, pos);
+
+                final Object obj = BinaryUtils.doReadOptimized(bin, ctx, U.resolveClassLoader(ctx.configuration()));
+
+                pos = bin.position();
+
+                return obj;
+            }
+
             default:
                 throw new BinaryObjectException("Invalid flag value: " + type);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b7c62d2/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java
index b296437..6974176 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java
@@ -116,8 +116,12 @@ class BinaryBuilderSerializer {
             String typeName = writer.context().userTypeName(clsName);
 
             BinaryMetadata meta = new BinaryMetadata(typeId, typeName, null, null, null, true);
+
             writer.context().updateMetadata(typeId, meta);
 
+            // Need register class for marshaller to be able to deserialize enum value.
+            writer.context().descriptorForClass(val.getClass(), false);
+
             writer.writeByte(GridBinaryMarshaller.ENUM);
             writer.writeInt(typeId);
             writer.writeInt(((Enum)val).ordinal());

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b7c62d2/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
index ddd2423..a847d04 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
@@ -22,6 +22,7 @@ import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.internal.binary.BinaryEnumObjectImpl;
 import org.apache.ignite.internal.binary.BinaryMetadata;
 import org.apache.ignite.internal.binary.BinaryObjectImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
@@ -391,6 +392,15 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
             if (((BinaryValueWithType)newVal).value() == null)
                 nullFieldVal = true;
         }
+        // Detect Enum and Enum array type.
+        else if (newVal instanceof BinaryEnumObjectImpl)
+            newFldTypeId = GridBinaryMarshaller.ENUM;
+        else if (newVal.getClass().isArray() && newVal.getClass().getComponentType() == BinaryObject.class) {
+            BinaryObject[] arr = (BinaryObject[])newVal;
+
+            newFldTypeId = arr.length > 0 && arr[0] instanceof BinaryEnumObjectImpl ?
+                GridBinaryMarshaller.ENUM_ARR : GridBinaryMarshaller.OBJ_ARR;
+        }
         else
             newFldTypeId = BinaryUtils.typeByClass(newVal.getClass());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b7c62d2/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
index 24806cb..507aa6b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
@@ -21,6 +21,10 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.lang.reflect.Field;
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -1423,11 +1427,19 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
      * @return Wrapper.
      */
     private BinaryObjectBuilderImpl newWrapper(Class<?> aCls) {
+        return newWrapper(aCls.getName());
+    }
+
+    /**
+     * @param typeName Type name.
+     * @return Wrapper.
+     */
+    private BinaryObjectBuilderImpl newWrapper(String typeName) {
         CacheObjectBinaryProcessorImpl processor = (CacheObjectBinaryProcessorImpl)(
             (IgniteBinaryImpl)binaries()).processor();
 
-        return new BinaryObjectBuilderImpl(processor.binaryContext(), processor.typeId(aCls.getName()),
-            processor.binaryContext().userTypeName(aCls.getName()));
+        return new BinaryObjectBuilderImpl(processor.binaryContext(), processor.typeId(typeName),
+            processor.binaryContext().userTypeName(typeName));
     }
 
     /**
@@ -1508,4 +1520,145 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
         assert OBJ.equals(binaryObj.type().fieldTypeName("asSetHint"));
         assert OBJ.equals(binaryObj.type().fieldTypeName("asMapHint"));
     }
+
+    /**
+     * Checks that externalizable value is correctly serialized/deserialized.
+     *
+     * @throws Exception If failed.
+     */
+    public void testBuilderExternalizable() throws Exception {
+        BinaryObjectBuilder builder = newWrapper("TestType");
+
+        final TestObjectExternalizable exp = new TestObjectExternalizable("test");
+        final TestObjectExternalizable[] expArr = new TestObjectExternalizable[]{
+            new TestObjectExternalizable("test1"), new TestObjectExternalizable("test2")};
+
+        BinaryObject extObj = builder.setField("extVal", exp).setField("extArr", expArr).build();
+
+        assertEquals(exp, extObj.field("extVal"));
+        Assert.assertArrayEquals(expArr, (Object[])extObj.field("extArr"));
+
+        builder = extObj.toBuilder();
+
+        extObj = builder.setField("intVal", 10).build();
+
+        assertEquals(exp, extObj.field("extVal"));
+        Assert.assertArrayEquals(expArr, (Object[])extObj.field("extArr"));
+        assertEquals(Integer.valueOf(10), extObj.field("intVal"));
+
+        builder = extObj.toBuilder();
+
+        extObj = builder.setField("strVal", "some string").build();
+
+        assertEquals(exp, extObj.field("extVal"));
+        Assert.assertArrayEquals(expArr, (Object[])extObj.field("extArr"));
+        assertEquals(Integer.valueOf(10), extObj.field("intVal"));
+        assertEquals("some string", extObj.field("strVal"));
+    }
+
+    /**
+     * Checks correct serialization/deserialization of enums in builder.
+     *
+     * @throws Exception If failed.
+     */
+    public void testEnum() throws Exception {
+        BinaryObjectBuilder builder = newWrapper("TestType");
+
+        final TestEnum exp = TestEnum.A;
+        final TestEnum[] expArr = {TestEnum.A, TestEnum.B};
+
+        BinaryObject enumObj = builder.setField("testEnum", exp).setField("testEnumArr", expArr).build();
+
+        assertEquals(exp, ((BinaryObject)enumObj.field("testEnum")).deserialize());
+        Assert.assertArrayEquals(expArr, (Object[])deserializeEnumBinaryArray(enumObj.field("testEnumArr")));
+
+        builder = newWrapper(enumObj.type().typeName());
+
+        enumObj = builder.setField("testEnum", (Object)enumObj.field("testEnum"))
+            .setField("testEnumArr", (Object)enumObj.field("testEnumArr")).build();
+
+        assertEquals(exp, ((BinaryObject)enumObj.field("testEnum")).deserialize());
+        Assert.assertArrayEquals(expArr, (Object[])deserializeEnumBinaryArray(enumObj.field("testEnumArr")));
+    }
+
+    /**
+     * @param obj BinaryObject array.
+     * @return Deserialized enums.
+     */
+    private TestEnum[] deserializeEnumBinaryArray(Object obj) {
+        Object[] arr = (Object[])obj;
+
+        final TestEnum[] res = new TestEnum[arr.length];
+
+        for (int i = 0; i < arr.length; i++)
+            res[i] = ((BinaryObject)arr[i]).deserialize();
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private static class TestObjectExternalizable implements Externalizable {
+        /** */
+        private String val;
+
+        /**
+         *
+         */
+        public TestObjectExternalizable() {
+        }
+
+        /**
+         * @param val Value.
+         */
+        public TestObjectExternalizable(final String val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(final ObjectOutput out) throws IOException {
+            out.writeUTF(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(final ObjectInput in) throws IOException, ClassNotFoundException {
+            val = in.readUTF();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(final Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            final TestObjectExternalizable that = (TestObjectExternalizable)o;
+
+            return val != null ? val.equals(that.val) : that.val == null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val != null ? val.hashCode() : 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "TestObjectExternalizable{" +
+                "val='" + val + '\'' +
+                '}';
+        }
+    }
+
+    /**
+     *
+     */
+    private enum TestEnum {
+        /** */
+        A,
+
+        /** */
+        B
+    }
 }


[51/52] ignite git commit: IGNITE-4340: Fixed implicit type conversion in DML. This closes #1303.

Posted by vo...@apache.org.
IGNITE-4340: Fixed implicit type conversion in DML. This closes #1303.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/97a6515b
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/97a6515b
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/97a6515b

Branch: refs/heads/master
Commit: 97a6515b89cc4dcfa6d5fd971167c27a27c38911
Parents: 445cde7
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Dec 2 17:17:42 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 2 17:17:42 2016 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        |  94 +++++-
 .../processors/query/h2/IgniteH2Indexing.java   |  10 +-
 .../processors/query/h2/dml/UpdatePlan.java     |  29 +-
 .../query/h2/dml/UpdatePlanBuilder.java         |  64 ++--
 .../processors/query/h2/sql/DmlAstUtils.java    |   2 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |   8 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |   2 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |   2 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |   2 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      | 322 +++++++++++++++++++
 10 files changed, 456 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/97a6515b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index c2c8726..4030758 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -17,11 +17,13 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
+import java.lang.reflect.Array;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -73,6 +75,8 @@ import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.h2.command.Prepared;
 import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.table.Column;
+import org.h2.value.DataType;
+import org.h2.value.Value;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
@@ -487,7 +491,8 @@ public class DmlStatementsProcessor {
                 if (hasNewVal && i == valColIdx - 2)
                     continue;
 
-                newColVals.put(plan.colNames[i], e.get(i + 2));
+                newColVals.put(plan.colNames[i], convert(e.get(i + 2), plan.colNames[i],
+                    plan.tbl.rowDescriptor(), plan.colTypes[i]));
             }
 
             newVal = plan.valSupplier.apply(e);
@@ -575,6 +580,64 @@ public class DmlStatementsProcessor {
     }
 
     /**
+     * Convert value to column's expected type by means of H2.
+     *
+     * @param val Source value.
+     * @param colName Column name to search for property.
+     * @param desc Row descriptor.
+     * @param type Expected column type to convert to.
+     * @return Converted object.
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings({"ConstantConditions", "SuspiciousSystemArraycopy"})
+    private static Object convert(Object val, String colName, GridH2RowDescriptor desc, int type)
+        throws IgniteCheckedException {
+        if (val == null)
+            return null;
+
+        GridQueryProperty prop = desc.type().property(colName);
+
+        assert prop != null;
+
+        Class<?> expCls = prop.type();
+
+        Class<?> currCls = val.getClass();
+
+        if (val instanceof Date && currCls != Date.class && expCls == Date.class) {
+            // H2 thinks that java.util.Date is always a Timestamp, while binary marshaller expects
+            // precise Date instance. Let's satisfy it.
+            return new Date(((Date) val).getTime());
+        }
+
+        // We have to convert arrays of reference types manually - see https://issues.apache.org/jira/browse/IGNITE-4327
+        // Still, we only can convert from Object[] to something more precise.
+        if (type == Value.ARRAY && currCls != expCls) {
+            if (currCls != Object[].class)
+                throw new IgniteCheckedException("Unexpected array type - only conversion from Object[] is assumed");
+
+            // Why would otherwise type be Value.ARRAY?
+            assert expCls.isArray();
+
+            Object[] curr = (Object[]) val;
+
+            Object newArr = Array.newInstance(expCls.getComponentType(), curr.length);
+
+            System.arraycopy(curr, 0, newArr, 0, curr.length);
+
+            return newArr;
+        }
+
+        int objType = DataType.getTypeFromClass(val.getClass());
+
+        if (objType == type)
+            return val;
+
+        Value h2Val = desc.wrap(val, objType);
+
+        return h2Val.convertTo(type).getObject();
+    }
+
+    /**
      * Process errors of entry processor - split the keys into duplicated/concurrently modified and those whose
      * processing yielded an exception.
      *
@@ -633,8 +696,8 @@ public class DmlStatementsProcessor {
 
         // If we have just one item to put, just do so
         if (plan.rowsNum == 1) {
-            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.keySupplier,
-                plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc.type());
+            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.colTypes, plan.keySupplier,
+                plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc);
 
             cctx.cache().put(t.getKey(), t.getValue());
             return 1;
@@ -646,8 +709,8 @@ public class DmlStatementsProcessor {
             for (Iterator<List<?>> it = cursor.iterator(); it.hasNext();) {
                 List<?> row = it.next();
 
-                IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.keySupplier, plan.valSupplier,
-                    plan.keyColIdx, plan.valColIdx, desc.type());
+                IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.colTypes, plan.keySupplier, plan.valSupplier,
+                    plan.keyColIdx, plan.valColIdx, desc);
 
                 rows.put(t.getKey(), t.getValue());
 
@@ -679,8 +742,8 @@ public class DmlStatementsProcessor {
 
         // If we have just one item to put, just do so
         if (plan.rowsNum == 1) {
-            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.keySupplier,
-                plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc.type());
+            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.colTypes,
+                plan.keySupplier, plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc);
 
             if (cctx.cache().putIfAbsent(t.getKey(), t.getValue()))
                 return 1;
@@ -705,8 +768,8 @@ public class DmlStatementsProcessor {
             while (it.hasNext()) {
                 List<?> row = it.next();
 
-                final IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.keySupplier,
-                    plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc.type());
+                final IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.colTypes, plan.keySupplier,
+                    plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc);
 
                 rows.put(t.getKey(), new InsertEntryProcessor(t.getValue()));
 
@@ -772,22 +835,21 @@ public class DmlStatementsProcessor {
 
     /**
      * Convert row presented as an array of Objects into key-value pair to be inserted to cache.
-     *
      * @param cctx Cache context.
      * @param row Row to process.
      * @param cols Query cols.
+     * @param colTypes Column types to convert data from {@code row} to.
      * @param keySupplier Key instantiation method.
      * @param valSupplier Key instantiation method.
      * @param keyColIdx Key column index, or {@code -1} if no key column is mentioned in {@code cols}.
      * @param valColIdx Value column index, or {@code -1} if no value column is mentioned in {@code cols}.
-     * @param desc Table descriptor.
-     * @return Key-value pair.
+     * @param rowDesc Row descriptor.
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions", "ResultOfMethodCallIgnored"})
     private IgniteBiTuple<?, ?> rowToKeyValue(GridCacheContext cctx, Object[] row, String[] cols,
-        KeyValueSupplier keySupplier, KeyValueSupplier valSupplier, int keyColIdx, int valColIdx,
-        GridQueryTypeDescriptor desc) throws IgniteCheckedException {
+        int[] colTypes, KeyValueSupplier keySupplier, KeyValueSupplier valSupplier, int keyColIdx, int valColIdx,
+        GridH2RowDescriptor rowDesc) throws IgniteCheckedException {
         Object key = keySupplier.apply(F.asList(row));
         Object val = valSupplier.apply(F.asList(row));
 
@@ -797,11 +859,13 @@ public class DmlStatementsProcessor {
         if (val == null)
             throw new IgniteSQLException("Value for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_VALUE);
 
+        GridQueryTypeDescriptor desc = rowDesc.type();
+
         for (int i = 0; i < cols.length; i++) {
             if (i == keyColIdx || i == valColIdx)
                 continue;
 
-            desc.setValue(cols[i], key, val, row[i]);
+            desc.setValue(cols[i], key, val, convert(row[i], cols[i], rowDesc, colTypes[i]));
         }
 
         if (cctx.binaryMarshaller()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/97a6515b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index f4ffbc7..5df44db 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -212,10 +212,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private static final int TWO_STEP_QRY_CACHE_SIZE = 1024;
 
     /** Field name for key. */
-    public static final String KEY_FIELD_NAME = "_key";
+    public static final String KEY_FIELD_NAME = "_KEY";
 
     /** Field name for value. */
-    public static final String VAL_FIELD_NAME = "_val";
+    public static final String VAL_FIELD_NAME = "_VAL";
 
     /** */
     private static final Field COMMAND_FIELD;
@@ -1246,8 +1246,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                             cachesCreated = true;
                         }
                         else
-                            throw new IgniteSQLException("Failed to parse query: " + sqlQry, e.getSQLState(),
-                                IgniteQueryErrorCode.PARSING);
+                            throw new IgniteSQLException("Failed to parse query: " + sqlQry,
+                                IgniteQueryErrorCode.PARSING, e);
                     }
                 }
             }
@@ -1459,7 +1459,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         String ptrn = "Name ''{0}'' is reserved and cannot be used as a field name [type=" + type.name() + "]";
 
         for (String name : names) {
-            if (name.equals(KEY_FIELD_NAME) || name.equals(VAL_FIELD_NAME))
+            if (name.equalsIgnoreCase(KEY_FIELD_NAME) || name.equalsIgnoreCase(VAL_FIELD_NAME))
                 throw new IgniteCheckedException(MessageFormat.format(ptrn, name));
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/97a6515b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
index 5976f4c..b81ac60 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
@@ -33,6 +33,12 @@ public final class UpdatePlan {
     /** Column names to set or update. */
     public final String[] colNames;
 
+    /**
+     * Expected column types to set or insert/merge.
+     * @see org.h2.value.Value
+     */
+    public final int[] colTypes;
+
     /** Method to create key for INSERT or MERGE, ignored for UPDATE and DELETE. */
     public final KeyValueSupplier keySupplier;
 
@@ -59,10 +65,11 @@ public final class UpdatePlan {
     public final FastUpdateArguments fastUpdateArgs;
 
     /** */
-    private UpdatePlan(UpdateMode mode, GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+    private UpdatePlan(UpdateMode mode, GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
         KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
         int rowsNum, FastUpdateArguments fastUpdateArgs) {
         this.colNames = colNames;
+        this.colTypes = colTypes;
         this.rowsNum = rowsNum;
         assert mode != null;
         assert tbl != null;
@@ -79,43 +86,43 @@ public final class UpdatePlan {
     }
 
     /** */
-    public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+    public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
         KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
         int rowsNum) {
         assert !F.isEmpty(colNames);
 
-        return new UpdatePlan(UpdateMode.MERGE, tbl, colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+        return new UpdatePlan(UpdateMode.MERGE, tbl, colNames, colTypes, keySupplier, valSupplier, keyColIdx, valColIdx,
             selectQry, isLocSubqry, rowsNum, null);
     }
 
     /** */
-    public static UpdatePlan forInsert(GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+    public static UpdatePlan forInsert(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
         KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry, int rowsNum) {
         assert !F.isEmpty(colNames);
 
-        return new UpdatePlan(UpdateMode.INSERT, tbl, colNames, keySupplier, valSupplier, keyColIdx, valColIdx, selectQry,
-            isLocSubqry, rowsNum, null);
+        return new UpdatePlan(UpdateMode.INSERT, tbl, colNames, colTypes, keySupplier, valSupplier, keyColIdx, valColIdx,
+            selectQry, isLocSubqry, rowsNum, null);
     }
 
     /** */
-    public static UpdatePlan forUpdate(GridH2Table tbl, String[] colNames, KeyValueSupplier valSupplier, int valColIdx,
-        String selectQry) {
+    public static UpdatePlan forUpdate(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier valSupplier,
+        int valColIdx, String selectQry) {
         assert !F.isEmpty(colNames);
 
-        return new UpdatePlan(UpdateMode.UPDATE, tbl, colNames, null, valSupplier, -1, valColIdx, selectQry,
+        return new UpdatePlan(UpdateMode.UPDATE, tbl, colNames, colTypes, null, valSupplier, -1, valColIdx, selectQry,
             false, 0, null);
     }
 
     /** */
     public static UpdatePlan forDelete(GridH2Table tbl, String selectQry) {
-        return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, -1, -1, selectQry, false, 0, null);
+        return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, null, -1, -1, selectQry, false, 0, null);
     }
 
     /** */
     public static UpdatePlan forFastUpdate(UpdateMode mode, GridH2Table tbl, FastUpdateArguments fastUpdateArgs) {
         assert mode == UpdateMode.UPDATE || mode == UpdateMode.DELETE;
 
-        return new UpdatePlan(mode, tbl, null, null, null, -1, -1, null, false, 0, fastUpdateArgs);
+        return new UpdatePlan(mode, tbl, null, null, null, null, -1, -1, null, false, 0, fastUpdateArgs);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/97a6515b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
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 549b901..fdcd164 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
@@ -132,7 +132,7 @@ public final class UpdatePlanBuilder {
             // not for updates, and hence will allow putting new pairs only.
             // We don't quote _key and _val column names on CREATE TABLE, so they are always uppercase here.
             GridSqlColumn[] keys = merge.keys();
-            if (keys.length != 1 || IgniteH2Indexing.KEY_FIELD_NAME.equals(keys[0].columnName()))
+            if (keys.length != 1 || !IgniteH2Indexing.KEY_FIELD_NAME.equals(keys[0].columnName()))
                 throw new CacheException("SQL MERGE does not support arbitrary keys");
 
             cols = merge.columns();
@@ -161,22 +161,30 @@ public final class UpdatePlanBuilder {
 
         String[] colNames = new String[cols.length];
 
+        int[] colTypes = new int[cols.length];
+
         for (int i = 0; i < cols.length; i++) {
-            colNames[i] = cols[i].columnName();
+            GridSqlColumn col = cols[i];
+
+            String colName = col.columnName();
+
+            colNames[i] = colName;
+
+            colTypes[i] = col.resultType().type();
 
-            if (isKeyColumn(cols[i].columnName(), desc)) {
+            if (KEY_FIELD_NAME.equals(colName)) {
                 keyColIdx = i;
                 continue;
             }
 
-            if (isValColumn(cols[i].columnName(), desc)) {
+            if (VAL_FIELD_NAME.equals(colName)) {
                 valColIdx = i;
                 continue;
             }
 
-            GridQueryProperty prop = desc.type().property(cols[i].columnName());
+            GridQueryProperty prop = desc.type().property(colName);
 
-            assert prop != null : "Property '" + cols[i].columnName() + "' not found.";
+            assert prop != null : "Property '" + colName + "' not found.";
 
             if (prop.key())
                 hasKeyProps = true;
@@ -188,11 +196,11 @@ public final class UpdatePlanBuilder {
         KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false);
 
         if (stmt instanceof GridSqlMerge)
-            return UpdatePlan.forMerge(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
-                sel.getSQL(), !isTwoStepSubqry, rowsNum);
+            return UpdatePlan.forMerge(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
+                valColIdx, sel.getSQL(), !isTwoStepSubqry, rowsNum);
         else
-            return UpdatePlan.forInsert(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
-                sel.getSQL(), !isTwoStepSubqry, rowsNum);
+            return UpdatePlan.forInsert(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
+                valColIdx, sel.getSQL(), !isTwoStepSubqry, rowsNum);
     }
 
     /**
@@ -253,10 +261,14 @@ public final class UpdatePlanBuilder {
 
                 String[] colNames = new String[updatedCols.size()];
 
+                int[] colTypes = new int[updatedCols.size()];
+
                 for (int i = 0; i < updatedCols.size(); i++) {
                     colNames[i] = updatedCols.get(i).columnName();
 
-                    if (isValColumn(colNames[i], desc))
+                    colTypes[i] = updatedCols.get(i).resultType().type();
+
+                    if (VAL_FIELD_NAME.equals(colNames[i]))
                         valColIdx = i;
                 }
 
@@ -287,7 +299,7 @@ public final class UpdatePlanBuilder {
 
                 sel = DmlAstUtils.selectForUpdate((GridSqlUpdate) stmt, errKeysPos);
 
-                return UpdatePlan.forUpdate(gridTbl, colNames, newValSupplier, valColIdx, sel.getSQL());
+                return UpdatePlan.forUpdate(gridTbl, colNames, colTypes, newValSupplier, valColIdx, sel.getSQL());
             }
             else {
                 sel = DmlAstUtils.selectForDelete((GridSqlDelete) stmt, errKeysPos);
@@ -471,32 +483,4 @@ public final class UpdatePlanBuilder {
 
         return false;
     }
-
-    /**
-     * Check that given column corresponds to the key with respect to case sensitivity, if needed (should be considered
-     * when the schema escapes all identifiers on table creation).
-     * @param colName Column name.
-     * @param desc Row descriptor.
-     * @return {@code true} if column name corresponds to _key with respect to case sensitivity depending on schema.
-     */
-    private static boolean isKeyColumn(String colName, GridH2RowDescriptor desc) {
-        if (desc.quoteAllIdentifiers())
-            return KEY_FIELD_NAME.equals(colName);
-        else
-            return KEY_FIELD_NAME.equalsIgnoreCase(colName);
-    }
-
-    /**
-     * Check that given column corresponds to the key with respect to case sensitivity, if needed (should be considered
-     * when the schema escapes all identifiers on table creation).
-     * @param colName Column name.
-     * @param desc Row descriptor.
-     * @return {@code true} if column name corresponds to _key with respect to case sensitivity depending on schema.
-     */
-    private static boolean isValColumn(String colName, GridH2RowDescriptor desc) {
-        if (desc.quoteAllIdentifiers())
-            return VAL_FIELD_NAME.equals(colName);
-        else
-            return VAL_FIELD_NAME.equalsIgnoreCase(colName);
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/97a6515b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
index 5ff715e..6deb146 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
@@ -280,7 +280,7 @@ public final class DmlAstUtils {
         GridSqlElement right = op.child(1);
 
         return left instanceof GridSqlColumn &&
-            colName.equalsIgnoreCase(((GridSqlColumn) left).columnName()) &&
+            colName.equals(((GridSqlColumn) left).columnName()) &&
             (right instanceof GridSqlConst || right instanceof GridSqlParameter);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/97a6515b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
index 47369ee..df4259e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
@@ -217,7 +217,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             flds.put("Id", Integer.class.getName());
             flds.put("id", Integer.class.getName());
             flds.put("name", String.class.getName());
-            flds.put("_Val", Integer.class.getName());
+            flds.put("IntVal", Integer.class.getName());
 
             k22p.setFields(flds);
 
@@ -336,7 +336,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
         if (!isBinaryMarshaller()) {
             Person2 p = new Person2(id);
             p.name = name;
-            p._Val = valFld;
+            p.IntVal = valFld;
 
             return p;
         }
@@ -344,7 +344,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             BinaryObjectBuilder o = grid(0).binary().builder("Person2");
             o.setField("id", id);
             o.setField("name", name);
-            o.setField("_Val", valFld);
+            o.setField("IntVal", valFld);
 
             return o.build();
         }
@@ -554,6 +554,6 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
 
         /** */
         @QuerySqlField
-        public int _Val;
+        public int IntVal;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/97a6515b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
index 1424163..7f79ec4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
@@ -166,7 +166,7 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
      * @param escapeSql whether identifiers should be quoted - see {@link CacheConfiguration#setSqlEscapeAll}
      * @return Cache configuration.
      */
-    private static CacheConfiguration cacheConfig(String name, boolean partitioned, boolean escapeSql) {
+    protected static CacheConfiguration cacheConfig(String name, boolean partitioned, boolean escapeSql) {
         return new CacheConfiguration()
             .setName(name)
             .setCacheMode(partitioned ? CacheMode.PARTITIONED : CacheMode.REPLICATED)

http://git-wip-us.apache.org/repos/asf/ignite/blob/97a6515b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
index 4f87740..04a352f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
@@ -119,7 +119,7 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
     public void testFieldsCaseSensitivity() {
         IgniteCache<Key2, Person> p = ignite(0).cache("K22P").withKeepBinary();
 
-        p.query(new SqlFieldsQuery("insert into \"Person2\" (\"Id\", \"id\", \"name\", \"_Val\") values (1, ?, ?, 5), " +
+        p.query(new SqlFieldsQuery("insert into \"Person2\" (\"Id\", \"id\", \"name\", \"IntVal\") values (1, ?, ?, 5), " +
             "(2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
 
         assertEquals(createPerson2(4, "Sergi", 5), p.get(new Key2(1)));

http://git-wip-us.apache.org/repos/asf/ignite/blob/97a6515b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
index e487564..0ff3fda 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
@@ -95,7 +95,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
     public void testFieldsCaseSensitivity() {
         IgniteCache<Key2, Person> p = ignite(0).cache("K22P").withKeepBinary();
 
-        p.query(new SqlFieldsQuery("merge into \"Person2\" (\"Id\", \"id\", \"name\", \"_Val\") values (1, ?, ?, 5), " +
+        p.query(new SqlFieldsQuery("merge into \"Person2\" (\"Id\", \"id\", \"name\", \"IntVal\") values (1, ?, ?, 5), " +
             "(2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
 
         assertEquals(createPerson2(4, "Sergi", 5), p.get(new Key2(1)));

http://git-wip-us.apache.org/repos/asf/ignite/blob/97a6515b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
index 538141f..332a082 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
@@ -17,17 +17,44 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Date;
 import java.util.List;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
 
 /**
  *
  */
 @SuppressWarnings("unchecked")
 public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDmlQuerySelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        ignite(0).createCache(createAllTypesCacheConfig());
+    }
+
+    /**
+     *
+     */
+    private static CacheConfiguration createAllTypesCacheConfig() {
+        CacheConfiguration ccfg = cacheConfig("L2AT", true, true);
+
+        ccfg.setIndexedTypes(Long.class, AllTypes.class);
+
+        return ccfg;
+    }
+
     /**
      *
      */
@@ -147,4 +174,299 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
         assertEqualsCollections(Arrays.asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
             leftovers.get(3));
     }
+
+    /** */
+    public void testTypeConversions() throws ParseException {
+        IgniteCache cache = ignite(0).cache("L2AT");
+
+        cache.query(new SqlFieldsQuery("insert into \"AllTypes\"(_key, _val, \"dateCol\", \"booleanCol\"," +
+            "\"tsCol\") values(2, ?, '2016-11-30 12:00:00', false, DATE '2016-12-01')").setArgs(new AllTypes(2L)));
+
+        List<?> ll = cache.query(new SqlFieldsQuery("select \"primitiveIntsCol\" from \"AllTypes\"")).getAll();
+
+        cache.query(new SqlFieldsQuery("update \"AllTypes\" set \"doubleCol\" = CAST('50' as INT)," +
+            " \"booleanCol\" = 80, \"innerTypeCol\" = ?, \"strCol\" = PI(), \"shortCol\" = " +
+            "CAST(WEEK(PARSEDATETIME('2016-11-30', 'yyyy-MM-dd')) as VARCHAR), " +
+            "\"sqlDateCol\"=TIMESTAMP '2016-12-02 13:47:00', \"tsCol\"=TIMESTAMPADD('MI', 2, " +
+            "DATEADD('DAY', 2, \"tsCol\")), \"primitiveIntsCol\" = ?, \"bytesCol\" = ?")
+            .setArgs(new AllTypes.InnerType(80L), new int[] {2, 3}, new Byte[] {4, 5, 6}));
+
+        AllTypes res = (AllTypes) cache.get(2L);
+
+        assertEquals(new BigDecimal(301.0).doubleValue(), res.bigDecimalCol.doubleValue());
+        assertEquals(50.0, res.doubleCol);
+        assertEquals(2L, (long) res.longCol);
+        assertTrue(res.booleanCol);
+        assertEquals("3.141592653589793", res.strCol);
+        assertTrue(Arrays.equals(new byte[] {0, 1}, res.primitiveBytesCol));
+        assertTrue(Arrays.equals(new Byte[] {4, 5, 6}, res.bytesCol));
+        assertTrue(Arrays.deepEquals(new Integer[] {0, 1}, res.intsCol));
+        assertTrue(Arrays.equals(new int[] {2, 3}, res.primitiveIntsCol));
+        assertEquals(new AllTypes.InnerType(80L), res.innerTypeCol);
+        assertEquals(new SimpleDateFormat("yyyy-MM-dd HH:mm:SS").parse("2016-11-30 12:00:00"), res.dateCol);
+        assertEquals(new SimpleDateFormat("yyyy-MM-dd HH:mm:SS").parse("2016-12-03 00:02:00"), res.tsCol);
+        assertEquals(2, res.intCol);
+        assertEquals(AllTypes.EnumType.ENUMTRUE, res.enumCol);
+        assertEquals(new java.sql.Date(new SimpleDateFormat("yyyy-MM-dd").parse("2016-12-02").getTime()), res.sqlDateCol);
+
+        // 49th week, right?
+        assertEquals(49, res.shortCol);
+    }
+
+    /**
+     *
+     */
+    static final class AllTypes implements Serializable {
+        /**
+         * Data Long.
+         */
+        @QuerySqlField
+        Long longCol;
+
+        /**
+         * Data double.
+         */
+        @QuerySqlField
+        double doubleCol;
+
+        /**
+         * Data String.
+         */
+        @QuerySqlField
+        String strCol;
+
+        /**
+         * Data boolean.
+         */
+        @QuerySqlField
+        boolean booleanCol;
+
+        /**
+         * Date.
+         */
+        @QuerySqlField
+        Date dateCol;
+
+        /**
+         * SQL date (non timestamp).
+         */
+        @QuerySqlField
+        java.sql.Date sqlDateCol;
+
+        /**
+         * Timestamp.
+         */
+        @QuerySqlField
+        Timestamp tsCol;
+
+        /**
+         * Data int.
+         */
+        @QuerySqlField
+        int intCol;
+
+        /**
+         * BigDecimal
+         */
+        @QuerySqlField
+        BigDecimal bigDecimalCol;
+
+        /**
+         * Data bytes array.
+         */
+        @QuerySqlField
+        Byte[] bytesCol;
+
+        /**
+         * Data bytes primitive array.
+         */
+        @QuerySqlField
+        byte[] primitiveBytesCol;
+
+        /**
+         * Data bytes array.
+         */
+        @QuerySqlField
+        Integer[] intsCol;
+
+        /**
+         * Data bytes primitive array.
+         */
+        @QuerySqlField
+        int[] primitiveIntsCol;
+
+        /**
+         * Data bytes array.
+         */
+        @QuerySqlField
+        short shortCol;
+
+        /**
+         * Inner type object.
+         */
+        @QuerySqlField
+        InnerType innerTypeCol;
+
+        /** */
+        static final class InnerType implements Serializable {
+            /** */
+            @QuerySqlField
+            Long innerLongCol;
+
+            /** */
+            @QuerySqlField
+            String innerStrCol;
+
+            /** */
+            @QuerySqlField
+            ArrayList<Long> arrListCol = new ArrayList<>();
+
+            /** */
+            InnerType(Long key) {
+                innerLongCol = key;
+                innerStrCol = Long.toString(key);
+
+                Long m = key % 8;
+
+                for (Integer i = 0; i < m; i++)
+                    arrListCol.add(key + i);
+            }
+
+            /**
+             * {@inheritDoc}
+             */
+            @Override public String toString() {
+                return "[Long=" + Long.toString(innerLongCol) +
+                    ", String='" + innerStrCol + "'" +
+                    ", ArrayList=" + arrListCol.toString() +
+                    "]";
+            }
+
+            /** {@inheritDoc} */
+            @Override public boolean equals(Object o) {
+                if (this == o) return true;
+                if (o == null || getClass() != o.getClass()) return false;
+
+                InnerType innerType = (InnerType) o;
+
+                if (innerLongCol != null ? !innerLongCol.equals(innerType.innerLongCol) : innerType.innerLongCol != null)
+                    return false;
+                if (innerStrCol != null ? !innerStrCol.equals(innerType.innerStrCol) : innerType.innerStrCol != null)
+                    return false;
+                return arrListCol != null ? arrListCol.equals(innerType.arrListCol) : innerType.arrListCol == null;
+
+            }
+
+            /** {@inheritDoc} */
+            @Override public int hashCode() {
+                int res = innerLongCol != null ? innerLongCol.hashCode() : 0;
+                res = 31 * res + (innerStrCol != null ? innerStrCol.hashCode() : 0);
+                res = 31 * res + (arrListCol != null ? arrListCol.hashCode() : 0);
+                return res;
+            }
+        }
+
+        /** */
+        @QuerySqlField
+        EnumType enumCol;
+
+        /** */
+        enum EnumType {
+            /** */
+            ENUMTRUE,
+
+            /** */
+            ENUMFALSE
+        }
+
+        /** */
+        private void init(Long key, String str) {
+            this.longCol = key;
+            this.doubleCol = Math.round(1000 * Math.log10(longCol.doubleValue()));
+            this.bigDecimalCol = BigDecimal.valueOf(doubleCol);
+            this.doubleCol = doubleCol / 100;
+            this.strCol = str;
+            if (key % 2 == 0) {
+                this.booleanCol = true;
+                this.enumCol = EnumType.ENUMTRUE;
+                this.innerTypeCol = new InnerType(key);
+            }
+            else {
+                this.booleanCol = false;
+                this.enumCol = EnumType.ENUMFALSE;
+                this.innerTypeCol = null;
+            }
+            this.intCol = key.intValue();
+            this.bytesCol = new Byte[(int) (key % 10)];
+            this.intsCol = new Integer[(int) (key % 10)];
+            this.primitiveBytesCol = new byte[(int) (key % 10)];
+            this.primitiveIntsCol = new int[(int) (key % 10)];
+            //this.bytesCol = new Byte[10];
+            int b = 0;
+            for (int j = 0; j < bytesCol.length; j++) {
+                if (b == 256)
+                    b = 0;
+                bytesCol[j] = (byte) b;
+                primitiveBytesCol[j] = (byte) b;
+                intsCol[j] = b;
+                primitiveIntsCol[j] = b;
+                b++;
+            }
+            this.shortCol = (short) (((1000 * key) % 50000) - 25000);
+
+            dateCol = new Date();
+        }
+
+        /** */
+        AllTypes(Long key) {
+            this.init(key, Long.toString(key));
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            AllTypes allTypes = (AllTypes) o;
+
+            if (Double.compare(allTypes.doubleCol, doubleCol) != 0) return false;
+            if (booleanCol != allTypes.booleanCol) return false;
+            if (intCol != allTypes.intCol) return false;
+            if (shortCol != allTypes.shortCol) return false;
+            if (longCol != null ? !longCol.equals(allTypes.longCol) : allTypes.longCol != null) return false;
+            if (strCol != null ? !strCol.equals(allTypes.strCol) : allTypes.strCol != null) return false;
+            if (dateCol != null ? !dateCol.equals(allTypes.dateCol) : allTypes.dateCol != null) return false;
+            if (sqlDateCol != null ? !sqlDateCol.equals(allTypes.sqlDateCol) : allTypes.sqlDateCol != null) return false;
+            if (tsCol != null ? !tsCol.equals(allTypes.tsCol) : allTypes.tsCol != null) return false;
+            if (bigDecimalCol != null ? !bigDecimalCol.equals(allTypes.bigDecimalCol) : allTypes.bigDecimalCol != null)
+                return false;
+            // Probably incorrect - comparing Object[] arrays with Arrays.equals
+            if (!Arrays.equals(bytesCol, allTypes.bytesCol)) return false;
+            if (innerTypeCol != null ? !innerTypeCol.equals(allTypes.innerTypeCol) : allTypes.innerTypeCol != null)
+                return false;
+            return enumCol == allTypes.enumCol;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res;
+            long temp;
+            res = longCol != null ? longCol.hashCode() : 0;
+            temp = Double.doubleToLongBits(doubleCol);
+            res = 31 * res + (int) (temp ^ (temp >>> 32));
+            res = 31 * res + (strCol != null ? strCol.hashCode() : 0);
+            res = 31 * res + (booleanCol ? 1 : 0);
+            res = 31 * res + (dateCol != null ? dateCol.hashCode() : 0);
+            res = 31 * res + (sqlDateCol != null ? sqlDateCol.hashCode() : 0);
+            res = 31 * res + (tsCol != null ? tsCol.hashCode() : 0);
+            res = 31 * res + intCol;
+            res = 31 * res + (bigDecimalCol != null ? bigDecimalCol.hashCode() : 0);
+            res = 31 * res + Arrays.hashCode(bytesCol);
+            res = 31 * res + (int) shortCol;
+            res = 31 * res + (innerTypeCol != null ? innerTypeCol.hashCode() : 0);
+            res = 31 * res + (enumCol != null ? enumCol.hashCode() : 0);
+            return res;
+        }
+    }
 }


[24/52] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.8' into ignite-1.8

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.8' into ignite-1.8


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ccbe6007
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ccbe6007
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ccbe6007

Branch: refs/heads/master
Commit: ccbe60076f32edc642515351cb69044dabea3436
Parents: 0822dc2 3f797b6
Author: devozerov <vo...@gridgain.com>
Authored: Fri Nov 25 12:38:45 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 12:38:45 2016 +0300

----------------------------------------------------------------------
 .../Examples/Example.cs                         |  6 +--
 .../Examples/ExamplesTest.cs                    |  7 +++-
 .../Process/IgniteProcess.cs                    | 40 ++++++--------------
 3 files changed, 17 insertions(+), 36 deletions(-)
----------------------------------------------------------------------



[06/52] ignite git commit: IGNITE-2294: Implemented DML.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
deleted file mode 100644
index 8dcba2f..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
+++ /dev/null
@@ -1,253 +0,0 @@
-/*
- * 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.query;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- * Two step map-reduce style query.
- */
-public class GridCacheTwoStepQuery {
-    /** */
-    public static final int DFLT_PAGE_SIZE = 1000;
-
-    /** */
-    @GridToStringInclude
-    private List<GridCacheSqlQuery> mapQrys = new ArrayList<>();
-
-    /** */
-    @GridToStringInclude
-    private GridCacheSqlQuery rdc;
-
-    /** */
-    private int pageSize = DFLT_PAGE_SIZE;
-
-    /** */
-    private boolean explain;
-
-    /** */
-    private Collection<String> spaces;
-
-    /** */
-    private Set<String> schemas;
-
-    /** */
-    private Set<String> tbls;
-
-    /** */
-    private boolean distributedJoins;
-
-    /** */
-    private boolean skipMergeTbl;
-
-    /** */
-    private List<Integer> caches;
-
-    /** */
-    private List<Integer> extraCaches;
-
-    /**
-     * @param schemas Schema names in query.
-     * @param tbls Tables in query.
-     */
-    public GridCacheTwoStepQuery(Set<String> schemas, Set<String> tbls) {
-        this.schemas = schemas;
-        this.tbls = tbls;
-    }
-
-    /**
-     * Specify if distributed joins are enabled for this query.
-     *
-     * @param distributedJoins Distributed joins enabled.
-     */
-    public void distributedJoins(boolean distributedJoins) {
-        this.distributedJoins = distributedJoins;
-    }
-
-    /**
-     * Check if distributed joins are enabled for this query.
-     *
-     * @return {@code true} If distributed joind enabled.
-     */
-    public boolean distributedJoins() {
-        return distributedJoins;
-    }
-
-
-    /**
-     * @return {@code True} if reduce query can skip merge table creation and get data directly from merge index.
-     */
-    public boolean skipMergeTable() {
-        return skipMergeTbl;
-    }
-
-    /**
-     * @param skipMergeTbl Skip merge table.
-     */
-    public void skipMergeTable(boolean skipMergeTbl) {
-        this.skipMergeTbl = skipMergeTbl;
-    }
-
-    /**
-     * @return If this is explain query.
-     */
-    public boolean explain() {
-        return explain;
-    }
-
-    /**
-     * @param explain If this is explain query.
-     */
-    public void explain(boolean explain) {
-        this.explain = explain;
-    }
-
-    /**
-     * @param pageSize Page size.
-     */
-    public void pageSize(int pageSize) {
-        this.pageSize = pageSize;
-    }
-
-    /**
-     * @return Page size.
-     */
-    public int pageSize() {
-        return pageSize;
-    }
-
-    /**
-     * @param qry SQL Query.
-     * @return {@code this}.
-     */
-    public GridCacheTwoStepQuery addMapQuery(GridCacheSqlQuery qry) {
-        mapQrys.add(qry);
-
-        return this;
-    }
-
-    /**
-     * @return Reduce query.
-     */
-    public GridCacheSqlQuery reduceQuery() {
-        return rdc;
-    }
-
-    /**
-     * @param rdc Reduce query.
-     */
-    public void reduceQuery(GridCacheSqlQuery rdc) {
-        this.rdc = rdc;
-    }
-
-    /**
-     * @return Map queries.
-     */
-    public List<GridCacheSqlQuery> mapQueries() {
-        return mapQrys;
-    }
-
-    /**
-     * @return Caches.
-     */
-    public List<Integer> caches() {
-        return caches;
-    }
-
-    /**
-     * @param caches Caches.
-     */
-    public void caches(List<Integer> caches) {
-        this.caches = caches;
-    }
-
-    /**
-     * @return Caches.
-     */
-    public List<Integer> extraCaches() {
-        return extraCaches;
-    }
-
-    /**
-     * @param extraCaches Caches.
-     */
-    public void extraCaches(List<Integer> extraCaches) {
-        this.extraCaches = extraCaches;
-    }
-
-    /**
-     * @return Spaces.
-     */
-    public Collection<String> spaces() {
-        return spaces;
-    }
-
-    /**
-     * @param spaces Spaces.
-     */
-    public void spaces(Collection<String> spaces) {
-        this.spaces = spaces;
-    }
-
-    /**
-     * @return Schemas.
-     */
-    public Set<String> schemas() {
-        return schemas;
-    }
-
-    /**
-     * @param args New arguments to copy with.
-     * @return Copy.
-     */
-    public GridCacheTwoStepQuery copy(Object[] args) {
-        assert !explain;
-
-        GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(schemas, tbls);
-
-        cp.caches = caches;
-        cp.extraCaches = extraCaches;
-        cp.spaces = spaces;
-        cp.rdc = rdc.copy(args);
-        cp.skipMergeTbl = skipMergeTbl;
-        cp.pageSize = pageSize;
-        cp.distributedJoins = distributedJoins;
-
-        for (int i = 0; i < mapQrys.size(); i++)
-            cp.mapQrys.add(mapQrys.get(i).copy(args));
-
-        return cp;
-    }
-
-    /**
-     * @return Tables.
-     */
-    public Set<String> tables() {
-        return tbls;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridCacheTwoStepQuery.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
new file mode 100644
index 0000000..93b8d47
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
@@ -0,0 +1,91 @@
+/*
+ * 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.query;
+
+import java.sql.SQLException;
+import javax.cache.processor.EntryProcessor;
+import org.apache.ignite.IgniteCache;
+
+/**
+ * Error codes for query operations.
+ */
+public final class IgniteQueryErrorCode {
+    /** Unknown error, or the one without specific code. */
+    public final static int UNKNOWN = 1;
+
+    /* 1xxx - parsing errors */
+
+    /** General parsing error - for the cases when there's no more specific code available. */
+    public final static int PARSING = 1001;
+
+    /** Code encountered unexpected type of SQL operation - like {@code EXPLAIN MERGE}. */
+    public final static int UNSUPPORTED_OPERATION = 1002;
+
+    /* 2xxx - analysis errors */
+
+    /** Code encountered SQL statement of some type that it did not expect in current analysis context. */
+    public final static int UNEXPECTED_OPERATION = 2001;
+
+    /** Code encountered SQL expression of some type that it did not expect in current analysis context. */
+    public final static int UNEXPECTED_ELEMENT_TYPE = 2002;
+
+    /** Analysis detected that the statement is trying to directly {@code UPDATE} key or its fields. */
+    public final static int KEY_UPDATE = 2003;
+
+    /* 3xxx - database API related runtime errors */
+
+    /** Required table not found. */
+    public final static int TABLE_NOT_FOUND = 3001;
+
+    /** Required table does not have a descriptor set. */
+    public final static int NULL_TABLE_DESCRIPTOR = 3002;
+
+    /** Statement type does not match that declared by JDBC driver. */
+    public final static int STMT_TYPE_MISMATCH = 3003;
+
+    /** Statement type does not match that declared by JDBC driver. */
+    public final static int TABLE_DROP_FAILED = 3004;
+
+    /* 4xxx - cache related runtime errors */
+
+    /** Attempt to INSERT a key that is already in cache. */
+    public final static int DUPLICATE_KEY = 4001;
+
+    /** Attempt to UPDATE or DELETE a key whose value has been updated concurrently by someone else. */
+    public final static int CONCURRENT_UPDATE = 4002;
+
+    /** Attempt to INSERT or MERGE {@code null} key. */
+    public final static int NULL_KEY = 4003;
+
+    /** Attempt to INSERT or MERGE {@code null} value. */
+    public final static int NULL_VALUE = 4004;
+
+    /** {@link EntryProcessor} has thrown an exception during {@link IgniteCache#invokeAll}. */
+    public final static int ENTRY_PROCESSING = 4005;
+
+    /**
+     * Create a {@link SQLException} for given code and message with null state.
+     *
+     * @param msg Message.
+     * @param code Ignite status code.
+     * @return {@link SQLException} with given details.
+     */
+    public static SQLException createJdbcSqlException(String msg, int code) {
+        return new SQLException(msg, null, code);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
index 5e26905..859de39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
@@ -259,6 +259,11 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     }
 
     /** {@inheritDoc} */
+    @Override public int capacity() {
+        return len;
+    }
+
+    /** {@inheritDoc} */
     @Override public int position() {
         return pos;
     }
@@ -293,6 +298,11 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     }
 
     /** {@inheritDoc} */
+    @Override public long rawOffheapPointer() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean hasArray() {
         return false;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
index cb30336..1ece10b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
@@ -208,6 +208,13 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
     }
 
     /** {@inheritDoc} */
+    @Override public long rawOffheapPointer() {
+        assert false;
+
+        throw new UnsupportedOperationException("Should not be called.");
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean hasArray() {
         assert false;
 
@@ -331,4 +338,9 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
 
         shift(len);
     }
+
+    /** {@inheritDoc} */
+    public int capacity() {
+        return cap;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 6bffa5d..13c1b3d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.query;
 
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
 import java.util.Collection;
 import java.util.List;
 import javax.cache.Cache;
@@ -61,9 +63,12 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param cancel Query cancel.
      * @return Cursor.
+     * @throws IgniteCheckedException If failed.
      */
-    public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, SqlFieldsQuery qry);
+    public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry, GridQueryCancel cancel)
+        throws IgniteCheckedException;
 
     /**
      * Parses SQL query into two step query and executes it.
@@ -71,8 +76,10 @@ public interface GridQueryIndexing {
      * @param cctx Cache context.
      * @param qry Query.
      * @return Cursor.
+     * @throws IgniteCheckedException If failed.
      */
-    public <K,V> QueryCursor<Cache.Entry<K,V>> queryTwoStep(GridCacheContext<?,?> cctx, SqlQuery qry);
+    public <K,V> QueryCursor<Cache.Entry<K,V>> queryTwoStep(GridCacheContext<?,?> cctx, SqlQuery qry)
+        throws IgniteCheckedException;
 
     /**
      * Queries individual fields (generally used by JDBC drivers).
@@ -88,7 +95,8 @@ public interface GridQueryIndexing {
      * @throws IgniteCheckedException If failed.
      */
     public GridQueryFieldsResult queryLocalSqlFields(@Nullable String spaceName, String qry,
-        Collection<Object> params, IndexingQueryFilter filter, boolean enforceJoinOrder, int timeout, GridQueryCancel cancel) throws IgniteCheckedException;
+        Collection<Object> params, IndexingQueryFilter filter, boolean enforceJoinOrder, int timeout,
+        GridQueryCancel cancel) throws IgniteCheckedException;
 
     /**
      * Executes regular query.
@@ -222,6 +230,15 @@ public interface GridQueryIndexing {
     public void onDisconnected(IgniteFuture<?> reconnectFut);
 
     /**
+     * Prepare native statement to retrieve JDBC metadata from.
+     *
+     * @param schema Schema.
+     * @param sql Query.
+     * @return {@link PreparedStatement} from underlying engine to supply metadata to Prepared - most likely H2.
+     */
+    public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException;
+
+    /**
      * Cancels all executing queries.
      */
     public void cancelAllQueries();

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 8befa0e..fe773c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -17,10 +17,10 @@
 
 package org.apache.ignite.internal.processors.query;
 
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
 import java.util.concurrent.TimeUnit;
-import java.lang.reflect.AccessibleObject;
 import java.lang.reflect.Field;
-import java.lang.reflect.Member;
 import java.lang.reflect.Method;
 import java.math.BigDecimal;
 import java.sql.Time;
@@ -47,6 +47,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.binary.BinaryField;
 import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.cache.CacheTypeMetadata;
@@ -279,6 +280,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         desc.keyClass(keyCls);
                     }
 
+                    desc.keyTypeName(qryEntity.getKeyType());
+                    desc.valueTypeName(qryEntity.getValueType());
+
                     if (binaryEnabled && keyOrValMustDeserialize) {
                         if (mustDeserializeClss == null)
                             mustDeserializeClss = new ArrayList<>();
@@ -385,6 +389,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         desc.keyClass(keyCls);
                     }
 
+                    desc.keyTypeName(meta.getKeyType());
+                    desc.valueTypeName(meta.getValueType());
+
                     if (binaryEnabled && keyOrValMustDeserialize) {
                         if (mustDeserializeClss == null)
                             mustDeserializeClss = new ArrayList<>();
@@ -802,8 +809,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         try {
             return executeQuery(GridCacheQueryType.SQL_FIELDS, qry.getSql(), cctx, new IgniteOutClosureX<QueryCursor<List<?>>>() {
-                @Override public QueryCursor<List<?>> applyx() {
-                    return idx.queryTwoStep(cctx, qry);
+                @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                    return idx.queryTwoStep(cctx, qry, null);
                 }
             }, true);
         }
@@ -938,6 +945,18 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     *
+     * @param schema Schema.
+     * @param sql Query.
+     * @return {@link PreparedStatement} from underlying engine to supply metadata to Prepared - most likely H2.
+     */
+    public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException {
+        checkxEnabled();
+
+        return idx.prepareNativeStatement(schema, sql);
+    }
+
+    /**
      * @param timeout Timeout.
      * @param timeUnit Time unit.
      * @return Converted time.
@@ -967,7 +986,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Iterator.
      */
-    public QueryCursor<List<?>> queryLocalFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
+    public QueryCursor<List<?>> queryLocalFields(final GridCacheContext<?, ?> cctx, final SqlFieldsQuery qry) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
@@ -1406,7 +1425,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             aliases = Collections.emptyMap();
 
         for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases);
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null, d);
 
             d.addProperty(prop, false);
 
@@ -1418,7 +1437,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases);
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null, d);
 
             d.addProperty(prop, false);
 
@@ -1430,7 +1449,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (String txtIdx : meta.getTextFields()) {
-            BinaryProperty prop = buildBinaryProperty(txtIdx, String.class, aliases);
+            BinaryProperty prop = buildBinaryProperty(txtIdx, String.class, aliases, null, d);
 
             d.addProperty(prop, false);
 
@@ -1448,7 +1467,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 int order = 0;
 
                 for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
-                    BinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(), aliases);
+                    BinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(), aliases,
+                        null, d);
 
                     d.addProperty(prop, false);
 
@@ -1462,7 +1482,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases);
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null, d);
 
             if (!d.props.containsKey(prop.name()))
                 d.addProperty(prop, false);
@@ -1482,8 +1502,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         if (aliases == null)
             aliases = Collections.emptyMap();
 
+        Set<String> keyFields = qryEntity.getKeyFields();
+        boolean hasKeyFields = (keyFields != null);
+
         for (Map.Entry<String, String> entry : qryEntity.getFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), U.classForName(entry.getValue(), Object.class, true), aliases);
+            Boolean isKeyField = (hasKeyFields ? keyFields.contains(entry.getKey()) : null);
+
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), U.classForName(entry.getValue(), Object.class, true),
+                aliases, isKeyField, d);
 
             d.addProperty(prop, false);
         }
@@ -1583,9 +1609,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      *      nested fields.
      * @param resType Result type.
      * @param aliases Aliases.
-     * @return Binary property.
+     * @param isKeyField
+     *@param d Type descriptor.  @return Binary property.
      */
-    private BinaryProperty buildBinaryProperty(String pathStr, Class<?> resType, Map<String,String> aliases) {
+    private BinaryProperty buildBinaryProperty(String pathStr, Class<?> resType, Map<String, String> aliases,
+        @Nullable Boolean isKeyField, TypeDescriptor d) throws IgniteCheckedException {
+
         String[] path = pathStr.split("\\.");
 
         BinaryProperty res = null;
@@ -1600,7 +1629,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String alias = aliases.get(fullName.toString());
 
-            res = new BinaryProperty(prop, res, resType, alias);
+            // The key flag that we've found out is valid for the whole path.
+            res = new BinaryProperty(prop, res, resType, isKeyField, alias);
         }
 
         return res;
@@ -1629,8 +1659,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             res = buildClassProperty(false, valCls, pathStr, resType, aliases, coCtx);
 
         if (res == null)
-            throw new IgniteCheckedException("Failed to initialize property '" + pathStr + "' for " +
-                "key class '" + keyCls + "' and value class '" + valCls + "'. " +
+            throw new IgniteCheckedException("Failed to initialize property '" + pathStr + "' of type '" +
+                resType.getName() + "' for key class '" + keyCls + "' and value class '" + valCls + "'. " +
                 "Make sure that one of these classes contains respective getter method or field.");
 
         return res;
@@ -1644,7 +1674,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param aliases Aliases.
      * @return Property instance corresponding to the given path.
      */
-    static ClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType,
+    private static ClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType,
         Map<String,String> aliases, CacheObjectContext coCtx) {
         String[] path = pathStr.split("\\.");
 
@@ -1660,58 +1690,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String alias = aliases.get(fullName.toString());
 
-            StringBuilder bld = new StringBuilder("get");
-
-            bld.append(prop);
-
-            bld.setCharAt(3, Character.toUpperCase(bld.charAt(3)));
-
-            ClassProperty tmp = null;
-
-            try {
-                tmp = new ClassProperty(cls.getMethod(bld.toString()), key, alias, coCtx);
-            }
-            catch (NoSuchMethodException ignore) {
-                // No-op.
-            }
-
-            if (tmp == null) { // Boolean getter can be defined as is###().
-                bld = new StringBuilder("is");
-
-                bld.append(prop);
-
-                bld.setCharAt(2, Character.toUpperCase(bld.charAt(2)));
-
-                try {
-                    tmp = new ClassProperty(cls.getMethod(bld.toString()), key, alias, coCtx);
-                }
-                catch (NoSuchMethodException ignore) {
-                    // No-op.
-                }
-            }
-
-            Class cls0 = cls;
-
-            while (tmp == null && cls0 != null)
-                try {
-                    tmp = new ClassProperty(cls0.getDeclaredField(prop), key, alias, coCtx);
-                }
-                catch (NoSuchFieldException ignored) {
-                    cls0 = cls0.getSuperclass();
-                }
-
-            if (tmp == null) {
-                try {
-                    tmp = new ClassProperty(cls.getMethod(prop), key, alias, coCtx);
-                }
-                catch (NoSuchMethodException ignored) {
-                    // No-op.
-                }
-            }
+            PropertyAccessor accessor = findProperty(prop, cls);
 
-            if (tmp == null)
+            if (accessor == null)
                 return null;
 
+            ClassProperty tmp = new ClassProperty(accessor, key, alias, coCtx);
+
             tmp.parent(res);
 
             cls = tmp.type();
@@ -1819,6 +1804,104 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Find a member (either a getter method or a field) with given name of given class.
+     * @param prop Property name.
+     * @param cls Class to search for a member in.
+     * @return Member for given name.
+     */
+    @Nullable private static PropertyAccessor findProperty(String prop, Class<?> cls) {
+        StringBuilder getBldr = new StringBuilder("get");
+        getBldr.append(prop);
+        getBldr.setCharAt(3, Character.toUpperCase(getBldr.charAt(3)));
+
+        StringBuilder setBldr = new StringBuilder("set");
+        setBldr.append(prop);
+        setBldr.setCharAt(3, Character.toUpperCase(setBldr.charAt(3)));
+
+        try {
+            Method getter = cls.getMethod(getBldr.toString());
+
+            Method setter;
+
+            try {
+                // Setter has to have the same name like 'setXxx' and single param of the same type
+                // as the return type of the getter.
+                setter = cls.getMethod(setBldr.toString(), getter.getReturnType());
+            }
+            catch (NoSuchMethodException ignore) {
+                // Have getter, but no setter - return read-only accessor.
+                return new ReadOnlyMethodsAccessor(getter, prop);
+            }
+
+            return new MethodsAccessor(getter, setter, prop);
+        }
+        catch (NoSuchMethodException ignore) {
+            // No-op.
+        }
+
+        getBldr = new StringBuilder("is");
+        getBldr.append(prop);
+        getBldr.setCharAt(2, Character.toUpperCase(getBldr.charAt(2)));
+
+        // We do nothing about setBldr here as it corresponds to setProperty name which is what we need
+        // for boolean property setter as well
+        try {
+            Method getter = cls.getMethod(getBldr.toString());
+
+            Method setter;
+
+            try {
+                // Setter has to have the same name like 'setXxx' and single param of the same type
+                // as the return type of the getter.
+                setter = cls.getMethod(setBldr.toString(), getter.getReturnType());
+            }
+            catch (NoSuchMethodException ignore) {
+                // Have getter, but no setter - return read-only accessor.
+                return new ReadOnlyMethodsAccessor(getter, prop);
+            }
+
+            return new MethodsAccessor(getter, setter, prop);
+        }
+        catch (NoSuchMethodException ignore) {
+            // No-op.
+        }
+
+        Class cls0 = cls;
+
+        while (cls0 != null)
+            try {
+                return new FieldAccessor(cls0.getDeclaredField(prop));
+            }
+            catch (NoSuchFieldException ignored) {
+                cls0 = cls0.getSuperclass();
+            }
+
+        try {
+            Method getter = cls.getMethod(prop);
+
+            Method setter;
+
+            try {
+                // Setter has to have the same name and single param of the same type
+                // as the return type of the getter.
+                setter = cls.getMethod(prop, getter.getReturnType());
+            }
+            catch (NoSuchMethodException ignore) {
+                // Have getter, but no setter - return read-only accessor.
+                return new ReadOnlyMethodsAccessor(getter, prop);
+            }
+
+            return new MethodsAccessor(getter, setter, prop);
+        }
+        catch (NoSuchMethodException ignored) {
+            // No-op.
+        }
+
+        // No luck.
+        return null;
+    }
+
+    /**
      * @param ver Version.
      */
     public static void setRequestAffinityTopologyVersion(AffinityTopologyVersion ver) {
@@ -1837,39 +1920,31 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     private static class ClassProperty extends GridQueryProperty {
         /** */
-        private final Member member;
+        private final PropertyAccessor accessor;
 
         /** */
-        private ClassProperty parent;
+        private final boolean key;
 
         /** */
-        private String name;
-
-        /** */
-        private boolean field;
+        private ClassProperty parent;
 
         /** */
-        private boolean key;
+        private final String name;
 
         /** */
-        private CacheObjectContext coCtx;
+        private final CacheObjectContext coCtx;
 
         /**
          * Constructor.
          *
-         * @param member Element.
+         * @param accessor Way of accessing the property.
          */
-        ClassProperty(Member member, boolean key, String name, @Nullable CacheObjectContext coCtx) {
-            this.member = member;
-            this.key = key;
-
-            this.name = !F.isEmpty(name) ? name :
-                member instanceof Method && member.getName().startsWith("get") && member.getName().length() > 3 ?
-                member.getName().substring(3) : member.getName();
+        ClassProperty(PropertyAccessor accessor, boolean key, String name, @Nullable CacheObjectContext coCtx) {
+            this.accessor = accessor;
 
-            ((AccessibleObject) member).setAccessible(true);
+            this.key = key;
 
-            field = member instanceof Field;
+            this.name = !F.isEmpty(name) ? name : accessor.getPropertyName();
 
             this.coCtx = coCtx;
         }
@@ -1884,21 +1959,25 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (x == null)
                 return null;
 
-            try {
-                if (field) {
-                    Field field = (Field)member;
+            return accessor.getValue(x);
+        }
 
-                    return field.get(x);
-                }
-                else {
-                    Method mtd = (Method)member;
+        /** {@inheritDoc} */
+        @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
+            Object x = unwrap(this.key ? key : val);
 
-                    return mtd.invoke(x);
-                }
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException(e);
-            }
+            if (parent != null)
+                x = parent.value(key, val);
+
+            if (x == null)
+                return;
+
+            accessor.setValue(x, propVal);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean key() {
+            return key;
         }
 
         /**
@@ -1918,7 +1997,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Class<?> type() {
-            return member instanceof Field ? ((Field)member).getType() : ((Method)member).getReturnType();
+            return accessor.getType();
         }
 
         /**
@@ -1932,14 +2011,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @Override public String toString() {
             return S.toString(ClassProperty.class, this);
         }
-
-        /**
-         * @param cls Class.
-         * @return {@code true} If this property or some parent relates to member of the given class.
-         */
-        public boolean knowsClass(Class<?> cls) {
-            return member.getDeclaringClass() == cls || (parent != null && parent.knowsClass(cls));
-        }
     }
 
     /**
@@ -1976,12 +2047,18 @@ public class GridQueryProcessor extends GridProcessorAdapter {
          * @param propName Property name.
          * @param parent Parent property.
          * @param type Result type.
+         * @param key {@code true} if key property, {@code false} otherwise, {@code null}  if unknown.
+         * @param alias Field alias.
          */
-        private BinaryProperty(String propName, BinaryProperty parent, Class<?> type, String alias) {
+        private BinaryProperty(String propName, BinaryProperty parent, Class<?> type, @Nullable Boolean key, String alias) {
+            super();
             this.propName = propName;
             this.alias = F.isEmpty(alias) ? propName : alias;
             this.parent = parent;
             this.type = type;
+
+            if (key != null)
+                this.isKeyProp = key ? 1 : -1;
         }
 
         /** {@inheritDoc} */
@@ -2030,6 +2107,31 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return fieldValue(obj0);
         }
 
+        /** {@inheritDoc} */
+        @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
+            Object obj = key() ? key : val;
+
+            if (obj == null)
+                return;
+
+            if (!(obj instanceof BinaryObjectBuilder))
+                throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
+
+            setValue0((BinaryObjectBuilder) obj, name(), propVal, type());
+        }
+
+        /**
+         * @param builder Object builder.
+         * @param field Field name.
+         * @param val Value to set.
+         * @param valType Type of {@code val}.
+         * @param <T> Value type.
+         */
+        private <T> void setValue0(BinaryObjectBuilder builder, String field, Object val, Class<T> valType) {
+            //noinspection unchecked
+            builder.setField(field, (T)val, valType);
+        }
+
         /**
          * Get binary field for the property.
          *
@@ -2081,6 +2183,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @Override public Class<?> type() {
             return type;
         }
+
+        /** {@inheritDoc} */
+        @Override public boolean key() {
+            int isKeyProp0 = isKeyProp;
+
+            if (isKeyProp0 == 0)
+                throw new IllegalStateException("Ownership flag not set for binary property. Have you set 'keyFields'" +
+                    " property of QueryEntity in configuration XML?");
+
+            return isKeyProp0 == 1;
+        }
     }
 
     /**
@@ -2098,6 +2211,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @GridToStringExclude
         private final Map<String, GridQueryProperty> props = new HashMap<>();
 
+        /** Map with upper cased property names to help find properties based on SQL INSERT and MERGE queries. */
+        private final Map<String, GridQueryProperty> uppercaseProps = new HashMap<>();
+
         /** */
         @GridToStringInclude
         private final Map<String, IndexDescriptor> indexes = new HashMap<>();
@@ -2112,6 +2228,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         private Class<?> valCls;
 
         /** */
+        private String keyTypeName;
+
+        /** */
+        private String valTypeName;
+
+        /** */
         private boolean valTextIdx;
 
         /** */
@@ -2155,7 +2277,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public GridQueryProperty property(String name) {
-            return props.get(name);
+            return getProperty(name);
         }
 
         /** {@inheritDoc} */
@@ -2163,7 +2285,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @Override public <T> T value(String field, Object key, Object val) throws IgniteCheckedException {
             assert field != null;
 
-            GridQueryProperty prop = props.get(field);
+            GridQueryProperty prop = getProperty(field);
 
             if (prop == null)
                 throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
@@ -2172,6 +2294,20 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void setValue(String field, Object key, Object val, Object propVal)
+            throws IgniteCheckedException {
+            assert field != null;
+
+            GridQueryProperty prop = getProperty(field);
+
+            if (prop == null)
+                throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
+
+            prop.setValue(key, val, propVal);
+        }
+
+        /** {@inheritDoc} */
         @Override public Map<String, GridQueryIndexDescriptor> indexes() {
             return Collections.<String, GridQueryIndexDescriptor>unmodifiableMap(indexes);
         }
@@ -2238,6 +2374,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
          * @param valCls Value class.
          */
         void valueClass(Class<?> valCls) {
+            A.notNull(valCls, "Value class must not be null");
             this.valCls = valCls;
         }
 
@@ -2255,6 +2392,34 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             this.keyCls = keyCls;
         }
 
+        /** {@inheritDoc} */
+        @Override public String keyTypeName() {
+            return keyTypeName;
+        }
+
+        /**
+         * Set key type name.
+         *
+         * @param keyTypeName Key type name.
+         */
+        public void keyTypeName(String keyTypeName) {
+            this.keyTypeName = keyTypeName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String valueTypeName() {
+            return valTypeName;
+        }
+
+        /**
+         * Set value type name.
+         *
+         * @param valTypeName Value type name.
+         */
+        public void valueTypeName(String valTypeName) {
+            this.valTypeName = valTypeName;
+        }
+
         /**
          * Adds property to the type descriptor.
          *
@@ -2268,9 +2433,25 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (props.put(name, prop) != null && failOnDuplicate)
                 throw new IgniteCheckedException("Property with name '" + name + "' already exists.");
 
+            if (uppercaseProps.put(name.toUpperCase(), prop) != null && failOnDuplicate)
+                throw new IgniteCheckedException("Property with upper cased name '" + name + "' already exists.");
+
             fields.put(name, prop.type());
         }
 
+        /**
+         * @param field Property name.
+         * @return Property with given field name.
+         */
+        private GridQueryProperty getProperty(String field) {
+            GridQueryProperty res = props.get(field);
+
+            if (res == null)
+                res = uppercaseProps.get(field.toUpperCase());
+
+            return res;
+        }
+
         /** {@inheritDoc} */
         @Override public boolean valueTextIndex() {
             return valTextIdx;
@@ -2501,4 +2682,183 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         /** Text index. */
         TEXT
     }
+
+    /** Way of accessing a property - either via field or getter and setter methods. */
+    private interface PropertyAccessor {
+        /**
+         * Get property value from given object.
+         *
+         * @param obj Object to retrieve property value from.
+         * @return Property value.
+         * @throws IgniteCheckedException if failed.
+         */
+        public Object getValue(Object obj) throws IgniteCheckedException;
+
+        /**
+         * Set property value on given object.
+         *
+         * @param obj Object to set property value on.
+         * @param newVal Property value.
+         * @throws IgniteCheckedException if failed.
+         */
+        public void setValue(Object obj, Object newVal)throws IgniteCheckedException;
+
+        /**
+         * @return Name of this property.
+         */
+        public String getPropertyName();
+
+        /**
+         * @return Type of the value of this property.
+         */
+        public Class<?> getType();
+    }
+
+    /** Accessor that deals with fields. */
+    private final static class FieldAccessor implements PropertyAccessor {
+        /** Field to access. */
+        private final Field fld;
+
+        /** */
+        private FieldAccessor(Field fld) {
+            fld.setAccessible(true);
+
+            this.fld = fld;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object getValue(Object obj) throws IgniteCheckedException {
+            try {
+                return fld.get(obj);
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException("Failed to get field value", e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setValue(Object obj, Object newVal) throws IgniteCheckedException {
+            try {
+                fld.set(obj, newVal);
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException("Failed to set field value", e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getPropertyName() {
+            return fld.getName();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Class<?> getType() {
+            return fld.getType();
+        }
+    }
+
+    /** Getter and setter methods based accessor. */
+    private final static class MethodsAccessor implements PropertyAccessor {
+        /** */
+        private final Method getter;
+
+        /** */
+        private final Method setter;
+
+        /** */
+        private final String propName;
+
+        /**
+         * @param getter Getter method.
+         * @param setter Setter method.
+         * @param propName Property name.
+         */
+        private MethodsAccessor(Method getter, Method setter, String propName) {
+            getter.setAccessible(true);
+            setter.setAccessible(true);
+
+            this.getter = getter;
+            this.setter = setter;
+            this.propName = propName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object getValue(Object obj) throws IgniteCheckedException {
+            try {
+                return getter.invoke(obj);
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException("Failed to invoke getter method " +
+                    "[type=" + getType() + ", property=" + propName + ']', e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setValue(Object obj, Object newVal) throws IgniteCheckedException {
+            try {
+                setter.invoke(obj, newVal);
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException("Failed to invoke setter method " +
+                    "[type=" + getType() + ", property=" + propName + ']', e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getPropertyName() {
+            return propName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Class<?> getType() {
+            return getter.getReturnType();
+        }
+    }
+
+    /** Accessor with getter only. */
+    private final static class ReadOnlyMethodsAccessor implements PropertyAccessor {
+        /** */
+        private final Method getter;
+
+        /** */
+        private final String propName;
+
+        /**
+         * @param getter Getter method.
+         * @param propName Property name.
+         */
+        private ReadOnlyMethodsAccessor(Method getter, String propName) {
+            getter.setAccessible(true);
+
+            this.getter = getter;
+            this.propName = propName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object getValue(Object obj) throws IgniteCheckedException {
+            try {
+                return getter.invoke(obj);
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException("Failed to invoke getter method " +
+                    "[type=" + getType() + ", property=" + propName + ']', e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setValue(Object obj, Object newVal) throws IgniteCheckedException {
+            throw new UnsupportedOperationException("Property is read-only [type=" + getType() +
+                ", property=" + propName + ']');
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getPropertyName() {
+            return propName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Class<?> getType() {
+            return getter.getReturnType();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
index d623d25..5d74a2e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
@@ -23,6 +23,10 @@ import org.apache.ignite.IgniteCheckedException;
  * Description and access method for query entity field.
  */
 public abstract class GridQueryProperty {
+    /** */
+    public GridQueryProperty() {
+    }
+
     /**
      * Gets this property value from the given object.
      *
@@ -34,6 +38,16 @@ public abstract class GridQueryProperty {
     public abstract Object value(Object key, Object val) throws IgniteCheckedException;
 
     /**
+     * Sets this property value for the given object.
+     *
+     * @param key Key.
+     * @param val Value.
+     * @param propVal Property value.
+     * @throws IgniteCheckedException If failed.
+     */
+    public abstract void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException;
+
+    /**
      * @return Property name.
      */
     public abstract String name();
@@ -42,4 +56,10 @@ public abstract class GridQueryProperty {
      * @return Class member type.
      */
     public abstract Class<?> type();
+
+    /**
+     * Property ownership flag.
+     * @return {@code true} if this property belongs to key, {@code false} if it belongs to value.
+     */
+    public abstract boolean key();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
index b636841..dcc01af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
@@ -50,6 +50,17 @@ public interface GridQueryTypeDescriptor {
     public <T> T value(String field, Object key, Object val) throws IgniteCheckedException;
 
     /**
+     * Sets field value for given key and value.
+     *
+     * @param field Field name.
+     * @param key Key.
+     * @param val Value.
+     * @param propVal Value for given field.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void setValue(String field, Object key, Object val, Object propVal) throws IgniteCheckedException;
+
+    /**
      * @param name Property name.
      * @return Property.
      */
@@ -77,6 +88,20 @@ public interface GridQueryTypeDescriptor {
     public Class<?> keyClass();
 
     /**
+     * Gets key type name.
+     *
+     * @return Key type name.
+     */
+    public String keyTypeName();
+
+    /**
+     * Gets value type name.
+     *
+     * @return Value type name.
+     */
+    public String valueTypeName();
+
+    /**
      * Returns {@code true} if string representation of value should be indexed as text.
      *
      * @return If string representation of value should be full-text indexed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/query/IgniteSQLException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/IgniteSQLException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/IgniteSQLException.java
new file mode 100644
index 0000000..b15007e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/IgniteSQLException.java
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.SQLException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Specific exception bearing information about query processing errors for more detailed
+ * errors in JDBC driver.
+ *
+ * @see IgniteQueryErrorCode
+ */
+public class IgniteSQLException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** State to return as {@link SQLException#SQLState} */
+    private final String sqlState;
+
+    /** Code to return as {@link SQLException#vendorCode} */
+    private final int statusCode;
+
+    /** */
+    public IgniteSQLException(String msg) {
+        this(msg, null, 0);
+    }
+
+    /**
+     * Minimalistic ctor accepting only {@link SQLException} as the cause.
+     */
+    public IgniteSQLException(SQLException cause) {
+        super(cause);
+        this.sqlState = null;
+        this.statusCode = 0;
+    }
+
+    /** */
+    public IgniteSQLException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+        this.sqlState = null;
+        this.statusCode = 0;
+    }
+
+    /** */
+    public IgniteSQLException(String msg, int statusCode, @Nullable Throwable cause) {
+        super(msg, cause);
+        this.sqlState = null;
+        this.statusCode = statusCode;
+    }
+
+    /** */
+    public IgniteSQLException(String msg, String sqlState, int statusCode) {
+        super(msg);
+        this.sqlState = sqlState;
+        this.statusCode = statusCode;
+    }
+
+    /** */
+    public IgniteSQLException(String msg, int statusCode) {
+        super(msg);
+        this.sqlState = null;
+        this.statusCode = statusCode;
+    }
+
+    /**
+     * @return JDBC exception containing details from this instance.
+     */
+    public SQLException toJdbcException() {
+        return new SQLException(getMessage(), sqlState, statusCode, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteSingletonIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteSingletonIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteSingletonIterator.java
new file mode 100644
index 0000000..0b0561b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteSingletonIterator.java
@@ -0,0 +1,56 @@
+/*
+ * 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.util.lang;
+
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
+
+/**
+ * Trivial iterator to return single item.
+ */
+public class IgniteSingletonIterator<T> extends GridCloseableIteratorAdapter<T> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final T val;
+
+    /** */
+    private boolean hasNext = true;
+
+    /** */
+    public IgniteSingletonIterator(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean onHasNext() throws IgniteCheckedException {
+        return hasNext;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected T onNext() throws IgniteCheckedException {
+        if (!hasNext)
+            throw new NoSuchElementException();
+
+        hasNext = false;
+
+        return val;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java
new file mode 100644
index 0000000..4a16662
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java
@@ -0,0 +1,300 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.binary.BinaryArrayIdentityResolver;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Array identity resolver self test.
+ */
+public class BinaryArrayIdentityResolverSelfTest extends GridCommonAbstractTest {
+    /** Pointers to release. */
+    private final Set<Long> ptrs = new ConcurrentHashSet<>();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        for (Long ptr : ptrs)
+            GridUnsafe.freeMemory(ptr);
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        BinaryConfiguration binCfg = new BinaryConfiguration();
+
+        BinaryTypeConfiguration binTypCfg1 = new BinaryTypeConfiguration();
+        BinaryTypeConfiguration binTypCfg2 = new BinaryTypeConfiguration();
+
+        binTypCfg1.setTypeName(InnerClass.class.getName());
+        binTypCfg2.setTypeName(InnerClassBinarylizable.class.getName());
+
+        binTypCfg1.setIdentityResolver(BinaryArrayIdentityResolver.instance());
+        binTypCfg2.setIdentityResolver(BinaryArrayIdentityResolver.instance());
+
+        List<BinaryTypeConfiguration> binTypCfgs = new ArrayList<>();
+
+        binTypCfgs.add(binTypCfg1);
+        binTypCfgs.add(binTypCfg2);
+
+        binCfg.setTypeConfigurations(binTypCfgs);
+
+        cfg.setBinaryConfiguration(binCfg);
+
+        return cfg;
+    }
+
+    /**
+     * Test hash code generation for simple object.
+     */
+    public void testHashCode() {
+        InnerClass obj = new InnerClass(1, "2", 3);
+
+        int expHash = BinaryArrayIdentityResolver.instance().hashCode(asBinary(obj));
+
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+    }
+
+    /**
+     * Test hash code generation for simple object.
+     */
+    public void testHashCodeBinarylizable() {
+        InnerClassBinarylizable obj = new InnerClassBinarylizable(1, "2", 3);
+
+        int expHash = BinaryArrayIdentityResolver.instance().hashCode(asBinary(obj));
+
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+    }
+
+    /**
+     * Test equals for simple object.
+     */
+    public void testEquals() {
+        InnerClass obj = new InnerClass(1, "2", 3);
+
+        // Positive cases.
+        compareTwo(asBinary(obj), asBinary(obj), true);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c), true);
+
+        // Negative cases.
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c, "d", "d"), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b + "1"), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a + 1, "b", obj.b), false);
+    }
+
+    /**
+     * Test equals for simple object.
+     */
+    public void testEqualsBinarilyzable() {
+        InnerClassBinarylizable obj = new InnerClassBinarylizable(1, "2", 3);
+
+        // Positive cases.
+        compareTwo(asBinary(obj), asBinary(obj), true);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c),
+            true);
+
+        // Negative cases.
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b), false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c, "d", "d"),
+            false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a), false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b + "1"), false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a + 1, "b", obj.b), false);
+    }
+
+    /**
+     * Test equals for different type IDs.
+     */
+    public void testEqualsDifferenTypes() {
+        InnerClass obj1 = new InnerClass(1, "2", 3);
+        InnerClassBinarylizable obj2 = new InnerClassBinarylizable(1, "2", 3);
+
+        compareTwo(asBinary(obj1), asBinary(obj2), false);
+    }
+
+    /**
+     * Compare two objects in different heap/offheap modes.
+     *
+     * @param obj1 Object 1.
+     * @param obj2 Object 2.
+     * @param expRes Expected result.
+     */
+    private void compareTwo(BinaryObject obj1, BinaryObject obj2, boolean expRes) {
+        if (expRes) {
+            assertEquals(convert(obj1, false), convert(obj2, false));
+            assertEquals(convert(obj1, false), convert(obj2, true));
+            assertEquals(convert(obj1, true), convert(obj2, false));
+            assertEquals(convert(obj1, true), convert(obj2, true));
+        }
+        else {
+            assertNotEquals(convert(obj1, false), convert(obj2, false));
+            assertNotEquals(convert(obj1, false), convert(obj2, true));
+            assertNotEquals(convert(obj1, true), convert(obj2, false));
+            assertNotEquals(convert(obj1, true), convert(obj2, true));
+        }
+    }
+
+    /**
+     * Convert to binary object.
+     *
+     * @param obj Original object.
+     * @return Binary object.
+     */
+    private BinaryObject asBinary(Object obj) {
+        return grid().binary().toBinary(obj);
+    }
+
+    /**
+     * Build object of the given type with provided fields.
+     *
+     * @param cls Class.
+     * @param parts Parts.
+     * @return Result.
+     */
+    private BinaryObject build(Class cls, Object... parts) {
+        BinaryObjectBuilder builder = grid().binary().builder(cls.getName());
+
+        if (!F.isEmpty(parts)) {
+            for (int i = 0; i < parts.length; )
+                builder.setField((String)parts[i++], parts[i++]);
+        }
+
+        return builder.build();
+    }
+
+    /**
+     * Inner class.
+     */
+    private static class InnerClass {
+        /** Field a. */
+        public int a;
+
+        /** Field b. */
+        public String b;
+
+        /** Field c. */
+        public long c;
+
+        /**
+         * Constructor.
+         *
+         * @param a Field a.
+         * @param b Field b.
+         * @param c Field c.
+         */
+        public InnerClass(int a, String b, long c) {
+            this.a = a;
+            this.b = b;
+            this.c = c;
+        }
+    }
+
+    /**
+     * Convert binary object to it's final state.
+     *
+     * @param obj Object.
+     * @param offheap Offheap flag.
+     * @return Result.
+     */
+    private BinaryObjectExImpl convert(BinaryObject obj, boolean offheap) {
+        BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
+
+        if (offheap) {
+            byte[] arr = obj0.array();
+
+            long ptr = GridUnsafe.allocateMemory(arr.length);
+
+            ptrs.add(ptr);
+
+            GridUnsafe.copyMemory(arr, GridUnsafe.BYTE_ARR_OFF, null, ptr, arr.length);
+
+            obj0 = new BinaryObjectOffheapImpl(obj0.context(), ptr, 0, obj0.array().length);
+        }
+
+        return obj0;
+    }
+
+    /**
+     * Inner class with Binarylizable interface.
+     */
+    private static class InnerClassBinarylizable extends InnerClass implements Binarylizable {
+        /**
+         * Constructor.
+         *
+         * @param a Field a.
+         * @param b Field b.
+         * @param c Field c.
+         */
+        public InnerClassBinarylizable(int a, String b, long c) {
+            super(a, b, c);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.writeInt("a", a);
+            writer.writeString("b", b);
+            writer.writeLong("c", c);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            a = reader.readInt("a");
+            b = reader.readString("b");
+            c = reader.readLong("c");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldIdentityResolverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldIdentityResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldIdentityResolverSelfTest.java
new file mode 100644
index 0000000..ba58392
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldIdentityResolverSelfTest.java
@@ -0,0 +1,333 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.binary.BinaryFieldIdentityResolver;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Field identity resolver self test.
+ */
+public class BinaryFieldIdentityResolverSelfTest extends GridCommonAbstractTest {
+    /** Pointers to release. */
+    private final Set<Long> ptrs = new ConcurrentHashSet<>();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        for (Long ptr : ptrs)
+            GridUnsafe.freeMemory(ptr);
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        BinaryConfiguration binCfg = new BinaryConfiguration();
+
+        BinaryTypeConfiguration binTypCfg1 = new BinaryTypeConfiguration();
+        BinaryTypeConfiguration binTypCfg2 = new BinaryTypeConfiguration();
+
+        binTypCfg1.setTypeName(InnerClass.class.getName());
+        binTypCfg2.setTypeName(InnerClassBinarylizable.class.getName());
+
+        BinaryFieldIdentityResolver binTypIdentityRslvr = new BinaryFieldIdentityResolver();
+
+        binTypIdentityRslvr.setFieldNames("a", "b");
+
+        binTypCfg1.setIdentityResolver(binTypIdentityRslvr);
+        binTypCfg2.setIdentityResolver(binTypIdentityRslvr);
+
+        List<BinaryTypeConfiguration> binTypCfgs = new ArrayList<>();
+
+        binTypCfgs.add(binTypCfg1);
+        binTypCfgs.add(binTypCfg2);
+
+        binCfg.setTypeConfigurations(binTypCfgs);
+
+        cfg.setBinaryConfiguration(binCfg);
+
+        return cfg;
+    }
+
+    /**
+     * Test hash code generation for simple object.
+     */
+    public void testHashCode() {
+        // Simple case.
+        InnerClass obj = new InnerClass(1, "2", 3);
+
+        int expHash = 31 * obj.a + obj.b.hashCode();
+
+        assertEquals(expHash, asBinary(obj).hashCode());
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+
+        // Different fields count.
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c, "d", "d").hashCode());
+
+        // Null fields.
+        obj = new InnerClass(1, null, 3);
+
+        expHash = 31 * obj.a;
+
+        assertEquals(expHash, asBinary(obj).hashCode());
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a).hashCode());
+    }
+
+    /**
+     * Test hash code generation for simple object.
+     */
+    public void testHashCodeBinarylizable() {
+        // Simple case.
+        InnerClassBinarylizable obj = new InnerClassBinarylizable(1, "2", 3);
+
+        int expHash = 31 * obj.a + obj.b.hashCode();
+
+        assertEquals(expHash, asBinary(obj).hashCode());
+        assertEquals(expHash, build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+
+        // Different fields count.
+        assertEquals(expHash, build(InnerClassBinarylizable.class,"a", obj.a, "b", obj.b, "c", obj.c, "d", "d")
+            .hashCode());
+
+        // Null fields.
+        obj = new InnerClassBinarylizable(1, null, 3);
+
+        expHash = 31 * obj.a;
+
+        assertEquals(expHash, asBinary(obj).hashCode());
+        assertEquals(expHash, build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+        assertEquals(expHash, build(InnerClassBinarylizable.class, "a", obj.a).hashCode());
+    }
+
+    /**
+     * Test equals for simple object.
+     */
+    public void testEquals() {
+        InnerClass obj = new InnerClass(1, "2", 3);
+
+        // Positive cases.
+        compareTwo(asBinary(obj), asBinary(obj), true);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b), true);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c), true);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c, "d", "d"), true);
+
+        // Negative cases.
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b + "1"), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a + 1, "b", obj.b), false);
+    }
+
+    /**
+     * Test equals for simple object.
+     */
+    public void testEqualsBinarilyzable() {
+        InnerClassBinarylizable obj = new InnerClassBinarylizable(1, "2", 3);
+
+        // Positive cases.
+        compareTwo(asBinary(obj), asBinary(obj), true);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b), true);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c),
+            true);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c, "d", "d"),
+            true);
+
+        // Negative cases.
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a), false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b + "1"), false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a + 1, "b", obj.b), false);
+    }
+
+    /**
+     * Test equals for different type IDs.
+     */
+    public void testEqualsDifferenTypes() {
+        InnerClass obj1 = new InnerClass(1, "2", 3);
+        InnerClassBinarylizable obj2 = new InnerClassBinarylizable(1, "2", 3);
+
+        compareTwo(asBinary(obj1), asBinary(obj2), false);
+    }
+
+    /**
+     * Compare two objects in different heap/offheap modes.
+     *
+     * @param obj1 Object 1.
+     * @param obj2 Object 2.
+     * @param expRes Expected result.
+     */
+    private void compareTwo(BinaryObject obj1, BinaryObject obj2, boolean expRes) {
+        if (expRes) {
+            assertEquals(convert(obj1, false), convert(obj2, false));
+            assertEquals(convert(obj1, false), convert(obj2, true));
+            assertEquals(convert(obj1, true), convert(obj2, false));
+            assertEquals(convert(obj1, true), convert(obj2, true));
+        }
+        else {
+            assertNotEquals(convert(obj1, false), convert(obj2, false));
+            assertNotEquals(convert(obj1, false), convert(obj2, true));
+            assertNotEquals(convert(obj1, true), convert(obj2, false));
+            assertNotEquals(convert(obj1, true), convert(obj2, true));
+        }
+    }
+
+    /**
+     * Convert to binary object.
+     *
+     * @param obj Original object.
+     * @return Binary object.
+     */
+    private BinaryObject asBinary(Object obj) {
+        return grid().binary().toBinary(obj);
+    }
+
+    /**
+     * Build object of the given type with provided fields.
+     *
+     * @param cls Class.
+     * @param parts Parts.
+     * @return Result.
+     */
+    private BinaryObject build(Class cls, Object... parts) {
+        BinaryObjectBuilder builder = grid().binary().builder(cls.getName());
+
+        if (!F.isEmpty(parts)) {
+            for (int i = 0; i < parts.length; )
+                builder.setField((String)parts[i++], parts[i++]);
+        }
+
+        return builder.build();
+    }
+
+    /**
+     * Inner class.
+     */
+    private static class InnerClass {
+        /** Field a. */
+        public int a;
+
+        /** Field b. */
+        public String b;
+
+        /** Field c. */
+        public long c;
+
+        /**
+         * Constructor.
+         *
+         * @param a Field a.
+         * @param b Field b.
+         * @param c Field c.
+         */
+        public InnerClass(int a, String b, long c) {
+            this.a = a;
+            this.b = b;
+            this.c = c;
+        }
+    }
+
+    /**
+     * Convert binary object to it's final state.
+     *
+     * @param obj Object.
+     * @param offheap Offheap flag.
+     * @return Result.
+     */
+    private BinaryObjectExImpl convert(BinaryObject obj, boolean offheap) {
+        BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
+
+        if (offheap) {
+            byte[] arr = obj0.array();
+
+            long ptr = GridUnsafe.allocateMemory(arr.length);
+
+            ptrs.add(ptr);
+
+            GridUnsafe.copyMemory(arr, GridUnsafe.BYTE_ARR_OFF, null, ptr, arr.length);
+
+            obj0 = new BinaryObjectOffheapImpl(obj0.context(), ptr, 0, obj0.array().length);
+        }
+
+        return obj0;
+    }
+
+    /**
+     * Inner class with Binarylizable interface.
+     */
+    private static class InnerClassBinarylizable extends InnerClass implements Binarylizable {
+        /**
+         * Constructor.
+         *
+         * @param a Field a.
+         * @param b Field b.
+         * @param c Field c.
+         */
+        public InnerClassBinarylizable(int a, String b, long c) {
+            super(a, b, c);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.writeLong("c", c);
+            writer.writeInt("a", a);
+            writer.writeString("b", b);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            c = reader.readLong("c");
+            a = reader.readInt("a");
+            b = reader.readString("b");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryIdentityResolverConfigurationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryIdentityResolverConfigurationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryIdentityResolverConfigurationSelfTest.java
new file mode 100644
index 0000000..3a2753b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryIdentityResolverConfigurationSelfTest.java
@@ -0,0 +1,138 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.binary.BinaryArrayIdentityResolver;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test for identity resolver configuration.
+ */
+public class BinaryIdentityResolverConfigurationSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        BinaryConfiguration binCfg = new BinaryConfiguration();
+
+        BinaryTypeConfiguration binTypCfg = new BinaryTypeConfiguration();
+
+        binTypCfg.setTypeName(MyClass.class.getName());
+        binTypCfg.setIdentityResolver(new CustomResolver());
+
+        List<BinaryTypeConfiguration> binTypCfgs = new ArrayList<>();
+
+        binTypCfgs.add(binTypCfg);
+
+        binCfg.setTypeConfigurations(binTypCfgs);
+
+        cfg.setBinaryConfiguration(binCfg);
+
+        return cfg;
+    }
+
+    /**
+     * Test type resolver.
+     */
+    public void testTypeResolver() {
+        MyClass obj = new MyClass(1, 2);
+
+        int expHash = hash(obj.a, obj.b);
+
+        BinaryObject binObj1 = binary().toBinary(obj);
+        BinaryObject binObj2 =
+            binary().builder(MyClass.class.getName()).setField("a", obj.a).setField("b", obj.b).build();
+
+        assertEquals(expHash, binObj1.hashCode());
+        assertEquals(expHash, binObj2.hashCode());
+    }
+
+    /**
+     * @return Binary interface for current Ignite instance.
+     */
+    public IgniteBinary binary() {
+        return grid().binary();
+    }
+
+    /**
+     * Second hash function.
+     *
+     * @param a First value.
+     * @param b Second value.
+     * @return Result.
+     */
+    public static int hash(Object a, Object b) {
+        return 31 * a.hashCode() + b.hashCode();
+    }
+
+    /**
+     * First class.
+     */
+    private static class MyClass {
+        /** Value 1. */
+        public int a;
+
+        /** Value 2. */
+        public int b;
+
+        /**
+         * Constructor.
+         *
+         * @param a Value 1.
+         * @param b Value 2.
+         */
+        public MyClass(int a, int b) {
+            this.a = a;
+            this.b = b;
+        }
+    }
+
+    /**
+     * First custom identity resolver.
+     */
+    private static class CustomResolver extends BinaryArrayIdentityResolver {
+        /** {@inheritDoc} */
+        @Override protected int hashCode0(BinaryObject obj) {
+            return hash(obj.field("a"), obj.field("b"));
+        }
+    }
+}


[47/52] ignite git commit: IGNITE-4353 Parent Cassandra module deployed on maven repository

Posted by vo...@apache.org.
IGNITE-4353 Parent Cassandra module deployed on maven repository


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/4ba8fcd0
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4ba8fcd0
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4ba8fcd0

Branch: refs/heads/master
Commit: 4ba8fcd08e9fb7c876e3532a0caef6d416df8a66
Parents: f2d8d07
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 1 17:45:20 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 1 17:45:20 2016 +0300

----------------------------------------------------------------------
 modules/cassandra/pom.xml | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4ba8fcd0/modules/cassandra/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/pom.xml b/modules/cassandra/pom.xml
index 733d53c..124c9b2 100644
--- a/modules/cassandra/pom.xml
+++ b/modules/cassandra/pom.xml
@@ -49,4 +49,20 @@
         <module>store</module>
         <module>serializers</module>
     </modules>
+
+    <build>
+        <pluginManagement>
+            <plugins>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-deploy-plugin</artifactId>
+                    <version>2.8.2</version>
+                    <inherited>false</inherited>
+                    <configuration>
+                        <skip>true</skip>
+                    </configuration>
+                </plugin>
+            </plugins>
+        </pluginManagement>
+    </build>
 </project>


[50/52] ignite git commit: IGNITE-4310: Added release notes for 1.8.

Posted by vo...@apache.org.
IGNITE-4310: Added release notes for 1.8.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/445cde77
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/445cde77
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/445cde77

Branch: refs/heads/master
Commit: 445cde777370fd1320dfec65c377c32acf3be1c7
Parents: b2692ad
Author: devozerov <vo...@gridgain.com>
Authored: Fri Dec 2 13:16:32 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 2 13:16:32 2016 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 20 +++++++++++++++++++-
 1 file changed, 19 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/445cde77/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index 5f5d190..ea88455 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -3,7 +3,25 @@ Apache Ignite Release Notes
 
 Apache Ignite In-Memory Data Fabric 1.8
 ---------------------------------------
-TBD
+Ignite:
+* SQL: Added DML operations support (INSERT, UPDATE, DELETE, MERGE)
+* SQL: Improved DISTINCT keyword handling in aggregates
+* Hadoop: Added MapR distribution support
+* Visor: Improved SQL statistics
+* Added Redis protocol support
+* Added transactions deadlock detection
+* Many stability and fault-tolerance fixes
+
+Ignite.NET:
+* ASP.NET session state store provider
+* Entity Framework second level cache
+* Custom loggers support: NLog, Apache log4Net
+
+ODBC driver:
+* Added DML operations support
+* Added distributed joins support
+* Added DSN support
+* Performance improvements
 
 Apache Ignite In-Memory Data Fabric 1.7
 ---------------------------------------


[32/52] ignite git commit: IGNITE-4311 .NET: Fix comments in MultiTieredCacheExample

Posted by vo...@apache.org.
IGNITE-4311 .NET: Fix comments in MultiTieredCacheExample


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/4a4082a6
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4a4082a6
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4a4082a6

Branch: refs/heads/master
Commit: 4a4082a6ec59211585a9fb16b8e2a5a2b2cec16a
Parents: 30b442e
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Nov 25 16:30:54 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Nov 25 16:31:24 2016 +0300

----------------------------------------------------------------------
 .../Datagrid/MultiTieredCacheExample.cs                      | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4a4082a6/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/MultiTieredCacheExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/MultiTieredCacheExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/MultiTieredCacheExample.cs
index 096c743..629ed89 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/MultiTieredCacheExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/MultiTieredCacheExample.cs
@@ -52,7 +52,7 @@ namespace Apache.Ignite.Examples.Datagrid
         public static void Main()
         {
             Console.WriteLine();
-            Console.WriteLine(">>> Lifecycle example started.");
+            Console.WriteLine(">>> Multi-tiered cache example started.");
 
             // Configure swap in the current bin directory (where our assembly is located).
             var binDir = Path.GetDirectoryName(typeof(MultiTieredCacheExample).Assembly.Location);
@@ -85,8 +85,10 @@ namespace Apache.Ignite.Examples.Datagrid
                     {
                         MaxSize = 10 // Maximum number of entries that will be stored in Java heap. 
                     },
-                    OffHeapMaxMemory = EntrySize * 10, // Limit off-heap to 10 entries.
-                    EnableSwap = true // Data will be swapped to disk if there is no more space in off-heap space.
+                    // Limit off-heap to roughly 10 entries. Actual entry count will be lower due to metadata overhead.
+                    OffHeapMaxMemory = EntrySize * 10,
+                    // Data will be swapped to disk if there is no more space in off-heap space.
+                    EnableSwap = true 
                 };
 
                 ICache<int, byte[]> cache = ignite.GetOrCreateCache<int, byte[]>(cacheCfg);


[03/52] ignite git commit: IGNITE-2294: Implemented DML.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
new file mode 100644
index 0000000..8eacf02
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
@@ -0,0 +1,46 @@
+/*
+ * 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.h2.sql;
+
+import java.util.Collections;
+import org.h2.command.Parser;
+import org.h2.expression.ValueExpression;
+
+/** Keyword (like DEFAULT). */
+public final class GridSqlKeyword extends GridSqlElement {
+    /**
+     * Default update value - analogous to H2.
+     * @see ValueExpression#getDefault()
+     * @see Parser#parseUpdate()
+     */
+    public final static GridSqlKeyword DEFAULT = new GridSqlKeyword("DEFAULT");
+
+    /** Keyword to return as SQL. */
+    private final String keyword;
+
+    /** */
+    private GridSqlKeyword(String keyword) {
+        super(Collections.<GridSqlElement>emptyList());
+        this.keyword = keyword;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        return keyword;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlMerge.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..4cdb314
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlMerge.java
@@ -0,0 +1,143 @@
+/*
+ * 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.h2.sql;
+
+import java.util.List;
+import org.h2.util.StatementBuilder;
+
+/** */
+public class GridSqlMerge extends GridSqlStatement {
+    /** */
+    private GridSqlElement into;
+
+    /** */
+    private GridSqlColumn[] cols;
+
+    /** */
+    private GridSqlColumn[] keys;
+
+    /** */
+    private List<GridSqlElement[]> rows;
+
+    /** Insert subquery. */
+    private GridSqlQuery qry;
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StatementBuilder buff = new StatementBuilder(explain() ? "EXPLAIN " : "");
+        buff.append("MERGE INTO ")
+            .append(into.getSQL())
+            .append("(");
+
+        for (GridSqlColumn col : cols) {
+            buff.appendExceptFirst(", ");
+            buff.append('\n')
+                .append(col.getSQL());
+        }
+        buff.append("\n)\n");
+
+        if (keys != null) {
+            buff.append("KEY(\n");
+            buff.resetCount();
+            for (GridSqlColumn c : keys) {
+                buff.appendExceptFirst(", ");
+                buff.append(c.getSQL())
+                    .append('\n');
+            }
+            buff.append(")\n");
+        }
+
+        if (!rows.isEmpty()) {
+            buff.append("VALUES\n");
+            StatementBuilder valuesBuff = new StatementBuilder();
+
+            for (GridSqlElement[] row : rows()) {
+                valuesBuff.appendExceptFirst(",\n");
+                StatementBuilder rowBuff = new StatementBuilder("(");
+                for (GridSqlElement e : row) {
+                    rowBuff.appendExceptFirst(", ");
+                    rowBuff.append(e != null ? e.getSQL() : "DEFAULT");
+                }
+                rowBuff.append(')');
+                valuesBuff.append(rowBuff.toString());
+            }
+            buff.append(valuesBuff.toString());
+        }
+        else
+            buff.append('\n')
+                .append(qry.getSQL());
+
+        return buff.toString();
+    }
+
+    /** */
+    public GridSqlElement into() {
+        return into;
+    }
+
+    /** */
+    public GridSqlMerge into(GridSqlElement from) {
+        this.into = from;
+        return this;
+    }
+
+    /** */
+    public List<GridSqlElement[]> rows() {
+        return rows;
+    }
+
+    /** */
+    public GridSqlMerge rows(List<GridSqlElement[]> rows) {
+        assert rows != null;
+        this.rows = rows;
+        return this;
+    }
+
+    /** */
+    public GridSqlQuery query() {
+        return qry;
+    }
+
+    /** */
+    public GridSqlMerge query(GridSqlQuery qry) {
+        this.qry = qry;
+        return this;
+    }
+
+    /** */
+    public GridSqlColumn[] columns() {
+        return cols;
+    }
+
+    /** */
+    public GridSqlMerge columns(GridSqlColumn[] cols) {
+        this.cols = cols;
+        return this;
+    }
+
+    /** */
+    public GridSqlColumn[] keys() {
+        return keys;
+    }
+
+    /** */
+    public GridSqlMerge keys(GridSqlColumn[] keys) {
+        this.keys = keys;
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
index d9784c8..331aa65 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
@@ -25,7 +25,7 @@ import org.h2.util.StringUtils;
 /**
  * Select query.
  */
-public abstract class GridSqlQuery {
+public abstract class GridSqlQuery extends GridSqlStatement {
     /** */
     protected boolean distinct;
 
@@ -35,29 +35,6 @@ public abstract class GridSqlQuery {
     /** */
     protected GridSqlElement offset;
 
-    /** */
-    protected GridSqlElement limit;
-
-    /** */
-    private boolean explain;
-
-    /**
-     * @param explain Explain.
-     * @return {@code this}.
-     */
-    public GridSqlQuery explain(boolean explain) {
-        this.explain = explain;
-
-        return this;
-    }
-
-    /**
-     * @return {@code true} If explain.
-     */
-    public boolean explain() {
-        return explain;
-    }
-
     /**
      * @return Offset.
      */
@@ -73,20 +50,6 @@ public abstract class GridSqlQuery {
     }
 
     /**
-     * @param limit Limit.
-     */
-    public void limit(GridSqlElement limit) {
-        this.limit = limit;
-    }
-
-    /**
-     * @return Limit.
-     */
-    public GridSqlElement limit() {
-        return limit;
-    }
-
-    /**
      * @return Distinct.
      */
     public boolean distinct() {
@@ -101,11 +64,6 @@ public abstract class GridSqlQuery {
     }
 
     /**
-     * @return Generate sql.
-     */
-    public abstract String getSQL();
-
-    /**
      * @return Sort.
      */
     public List<GridSqlSortColumn> sort() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
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 a7451c1..d9c546c 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
@@ -19,17 +19,27 @@ package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.lang.reflect.Field;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.IdentityHashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.h2.command.Command;
 import org.h2.command.CommandContainer;
 import org.h2.command.Prepared;
+import org.h2.command.dml.Delete;
 import org.h2.command.dml.Explain;
+import org.h2.command.dml.Insert;
+import org.h2.command.dml.Merge;
 import org.h2.command.dml.Query;
 import org.h2.command.dml.Select;
 import org.h2.command.dml.SelectUnion;
+import org.h2.command.dml.Update;
 import org.h2.engine.FunctionAlias;
 import org.h2.expression.Aggregate;
 import org.h2.expression.Alias;
@@ -224,6 +234,64 @@ public class GridSqlQueryParser {
     private static final Getter<Explain, Prepared> EXPLAIN_COMMAND = getter(Explain.class, "command");
 
     /** */
+    private static final Getter<Merge, Table> MERGE_TABLE = getter(Merge.class, "table");
+
+    /** */
+    private static final Getter<Merge, Column[]> MERGE_COLUMNS = getter(Merge.class, "columns");
+
+    /** */
+    private static final Getter<Merge, Column[]> MERGE_KEYS = getter(Merge.class, "keys");
+
+    /** */
+    private static final Getter<Merge, List<Expression[]>> MERGE_ROWS = getter(Merge.class, "list");
+
+    /** */
+    private static final Getter<Merge, Query> MERGE_QUERY = getter(Merge.class, "query");
+
+    /** */
+    private static final Getter<Insert, Table> INSERT_TABLE = getter(Insert.class, "table");
+
+    /** */
+    private static final Getter<Insert, Column[]> INSERT_COLUMNS = getter(Insert.class, "columns");
+
+    /** */
+    private static final Getter<Insert, List<Expression[]>> INSERT_ROWS = getter(Insert.class, "list");
+
+    /** */
+    private static final Getter<Insert, Query> INSERT_QUERY = getter(Insert.class, "query");
+
+    /** */
+    private static final Getter<Insert, Boolean> INSERT_DIRECT = getter(Insert.class, "insertFromSelect");
+
+    /** */
+    private static final Getter<Insert, Boolean> INSERT_SORTED = getter(Insert.class, "sortedInsertMode");
+
+    /** */
+    private static final Getter<Delete, TableFilter> DELETE_FROM = getter(Delete.class, "tableFilter");
+
+    /** */
+    private static final Getter<Delete, Expression> DELETE_WHERE = getter(Delete.class, "condition");
+
+    /** */
+    private static final Getter<Delete, Expression> DELETE_LIMIT = getter(Delete.class, "limitExpr");
+
+    /** */
+    private static final Getter<Update, TableFilter> UPDATE_TARGET = getter(Update.class, "tableFilter");
+
+    /** */
+    private static final Getter<Update, ArrayList<Column>> UPDATE_COLUMNS = getter(Update.class, "columns");
+
+    /** */
+    private static final Getter<Update, HashMap<Column, Expression>> UPDATE_SET = getter(Update.class,
+        "expressionMap");
+
+    /** */
+    private static final Getter<Update, Expression> UPDATE_WHERE = getter(Update.class, "condition");
+
+    /** */
+    private static final Getter<Update, Expression> UPDATE_LIMIT = getter(Update.class, "limitExpr");
+
+    /** */
     private static final Getter<Command, Prepared> PREPARED =
         GridSqlQueryParser.<Command, Prepared>getter(CommandContainer.class, "prepared");
 
@@ -249,14 +317,32 @@ public class GridSqlQueryParser {
         GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(filter);
 
         if (res == null) {
-            Table tbl = filter.getTable();
+            res = parseTable(filter.getTable(), filter.getIndex(),
+                filter.getSelect() != null ? filter.getSelect().getSQL() : null);
 
+            String alias = ALIAS.get(filter);
+
+            if (alias != null)
+                res = new GridSqlAlias(alias, res, false);
+
+            h2ObjToGridObj.put(filter, res);
+        }
+
+        return res;
+    }
+
+
+    /**
+     * @param tbl Table.
+     */
+    private GridSqlElement parseTable(Table tbl, @Nullable Index idx, String sql) {
+        GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(tbl);
+
+        if (res == null) {
             if (tbl instanceof TableBase)
                 res = new GridSqlTable(tbl);
             else if (tbl instanceof TableView) {
-                Query qry = VIEW_QUERY.get((TableView)tbl);
-
-                Index idx = filter.getIndex();
+                Query qry = VIEW_QUERY.get((TableView) tbl);
 
                 Query idxQry = idx instanceof ViewIndex ? ((ViewIndex)idx).getQuery() : null;
 
@@ -271,14 +357,10 @@ public class GridSqlQueryParser {
                 res.addChild(parseExpression(RANGE_MAX.get((RangeTable)tbl), false));
             }
             else
-                assert0(false, filter.getSelect().getSQL());
-
-            String alias = ALIAS.get(filter);
-
-            if (alias != null)
-                res = new GridSqlAlias(alias, res, false);
+                assert0(false, "Unexpected Table implementation [cls=" + tbl.getClass().getSimpleName() + ", " +
+                    "sql=" + sql + ']');
 
-            h2ObjToGridObj.put(filter, res);
+            h2ObjToGridObj.put(tbl, res);
         }
 
         return res;
@@ -348,6 +430,171 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * @param merge Merge.
+     * @see <a href="http://h2database.com/html/grammar.html#merge">H2 merge spec</a>
+     */
+    private GridSqlMerge parseMerge(Merge merge) {
+        GridSqlMerge res = (GridSqlMerge)h2ObjToGridObj.get(merge);
+
+        if (res != null)
+            return res;
+
+        res = new GridSqlMerge();
+        h2ObjToGridObj.put(merge, res);
+
+        Table srcTbl = MERGE_TABLE.get(merge);
+        GridSqlElement tbl = parseTable(srcTbl, null, merge.getSQL());
+
+        res.into(tbl);
+
+        Column[] srcCols = MERGE_COLUMNS.get(merge);
+
+        GridSqlColumn[] cols = new GridSqlColumn[srcCols.length];
+
+        for (int i = 0; i < srcCols.length; i++) {
+            cols[i] = new GridSqlColumn(srcCols[i], tbl, srcCols[i].getName(), srcCols[i].getSQL());
+
+            cols[i].resultType(fromColumn(srcCols[i]));
+        }
+
+        res.columns(cols);
+
+        Column[] srcKeys = MERGE_KEYS.get(merge);
+
+        GridSqlColumn[] keys = new GridSqlColumn[srcKeys.length];
+        for (int i = 0; i < srcKeys.length; i++)
+            keys[i] = new GridSqlColumn(srcKeys[i], tbl, srcKeys[i].getName(), srcKeys[i].getSQL());
+        res.keys(keys);
+
+        List<Expression[]> srcRows = MERGE_ROWS.get(merge);
+        if (!srcRows.isEmpty()) {
+            List<GridSqlElement[]> rows = new ArrayList<>(srcRows.size());
+            for (Expression[] srcRow : srcRows) {
+                GridSqlElement[] row = new GridSqlElement[srcRow.length];
+
+                for (int i = 0; i < srcRow.length; i++)
+                    row[i] = parseExpression(srcRow[i], false);
+
+                rows.add(row);
+            }
+            res.rows(rows);
+        }
+        else {
+            res.rows(Collections.<GridSqlElement[]>emptyList());
+            res.query(parse(MERGE_QUERY.get(merge), null));
+        }
+
+        return res;
+    }
+
+    /**
+     * @param insert Insert.
+     * @see <a href="http://h2database.com/html/grammar.html#insert">H2 insert spec</a>
+     */
+    private GridSqlInsert parseInsert(Insert insert) {
+        GridSqlInsert res = (GridSqlInsert)h2ObjToGridObj.get(insert);
+
+        if (res != null)
+            return res;
+
+        res = new GridSqlInsert();
+        h2ObjToGridObj.put(insert, res);
+
+        Table srcTbl = INSERT_TABLE.get(insert);
+        GridSqlElement tbl = parseTable(srcTbl, null, insert.getSQL());
+
+        res.into(tbl).
+            direct(INSERT_DIRECT.get(insert)).
+            sorted(INSERT_SORTED.get(insert));
+
+        Column[] srcCols = INSERT_COLUMNS.get(insert);
+        GridSqlColumn[] cols = new GridSqlColumn[srcCols.length];
+
+        for (int i = 0; i < srcCols.length; i++) {
+            cols[i] = new GridSqlColumn(srcCols[i], tbl, srcCols[i].getName(), srcCols[i].getSQL());
+
+            cols[i].resultType(fromColumn(srcCols[i]));
+        }
+
+        res.columns(cols);
+
+        List<Expression[]> srcRows = INSERT_ROWS.get(insert);
+        if (!srcRows.isEmpty()) {
+            List<GridSqlElement[]> rows = new ArrayList<>(srcRows.size());
+            for (Expression[] srcRow : srcRows) {
+                GridSqlElement[] row = new GridSqlElement[srcRow.length];
+
+                for (int i = 0; i < srcRow.length; i++)
+                    row[i] = parseExpression(srcRow[i], false);
+
+                rows.add(row);
+            }
+            res.rows(rows);
+        }
+        else {
+            res.rows(Collections.<GridSqlElement[]>emptyList());
+            res.query(parse(INSERT_QUERY.get(insert), null));
+        }
+
+        return res;
+    }
+
+    /**
+     * @param del Delete.
+     * @see <a href="http://h2database.com/html/grammar.html#delete">H2 delete spec</a>
+     */
+    private GridSqlDelete parseDelete(Delete del) {
+        GridSqlDelete res = (GridSqlDelete)h2ObjToGridObj.get(del);
+
+        if (res != null)
+            return res;
+
+        res = new GridSqlDelete();
+        h2ObjToGridObj.put(del, res);
+
+        GridSqlElement tbl = parseTable(DELETE_FROM.get(del));
+        GridSqlElement where = parseExpression(DELETE_WHERE.get(del), true);
+        GridSqlElement limit = parseExpression(DELETE_LIMIT.get(del), true);
+        res.from(tbl).where(where).limit(limit);
+        return res;
+    }
+
+    /**
+     * @param update Update.
+     * @see <a href="http://h2database.com/html/grammar.html#update">H2 update spec</a>
+     */
+    private GridSqlUpdate parseUpdate(Update update) {
+        GridSqlUpdate res = (GridSqlUpdate)h2ObjToGridObj.get(update);
+
+        if (res != null)
+            return res;
+
+        res = new GridSqlUpdate();
+        h2ObjToGridObj.put(update, res);
+
+        GridSqlElement tbl = parseTable(UPDATE_TARGET.get(update));
+
+        List<Column> srcCols = UPDATE_COLUMNS.get(update);
+        Map<Column, Expression> srcSet = UPDATE_SET.get(update);
+
+        ArrayList<GridSqlColumn> cols = new ArrayList<>(srcCols.size());
+        LinkedHashMap<String, GridSqlElement> set = new LinkedHashMap<>(srcSet.size());
+
+        for (Column c : srcCols) {
+            GridSqlColumn col = new GridSqlColumn(c, tbl, c.getName(), c.getSQL());
+            col.resultType(fromColumn(c));
+            cols.add(col);
+            set.put(col.columnName(), parseExpression(srcSet.get(c), true));
+        }
+
+        GridSqlElement where = parseExpression(UPDATE_WHERE.get(update), true);
+        GridSqlElement limit = parseExpression(UPDATE_LIMIT.get(update), true);
+
+        res.target(tbl).cols(cols).set(set).where(where).limit(limit);
+        return res;
+    }
+
+    /**
      * @param sortOrder Sort order.
      * @param qry Query.
      */
@@ -387,26 +634,56 @@ public class GridSqlQueryParser {
      * @param qry Prepared.
      * @return Query.
      */
-    public GridSqlQuery parse(Prepared qry) {
+    public GridSqlStatement parse(Prepared qry) {
         return parse(qry, null);
     }
 
     /**
      * @param qry Select.
      */
-    public GridSqlQuery parse(Prepared qry, @Nullable Query idxQry) {
+    public GridSqlStatement parse(Prepared qry, @Nullable Query idxQry) {
         assert qry != null;
 
+        if (qry instanceof Query)
+            return parse((Query)qry, idxQry);
+
+        if (qry instanceof Merge)
+            return parseMerge((Merge)qry);
+
+        if (qry instanceof Insert)
+            return parseInsert((Insert)qry);
+
+        if (qry instanceof Delete)
+            return parseDelete((Delete)qry);
+
+        if (qry instanceof Update)
+            return parseUpdate((Update)qry);
+
+        if (qry instanceof Explain) {
+            GridSqlStatement stmt = parse(EXPLAIN_COMMAND.get((Explain) qry));
+
+            if (!(stmt instanceof GridSqlQuery))
+                throw new IgniteSQLException("EXPLAIN is not supported for DML statement: " + qry,
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            return stmt.explain(true);
+        }
+
+        throw new IgniteSQLException("Unsupported statement: " + qry, IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+    }
+
+    /**
+     * @param qry Query.
+     * @return Grid SQL query.
+     */
+    private GridSqlQuery parse(Query qry, @Nullable Query idxQry) {
         if (qry instanceof Select)
             return parse((Select)qry, idxQry);
 
         if (qry instanceof SelectUnion)
             return parse((SelectUnion)qry);
 
-        if (qry instanceof Explain)
-            return parse(EXPLAIN_COMMAND.get((Explain)qry)).explain(true);
-
-        throw new CacheException("Unsupported query: " + qry);
+        throw new UnsupportedOperationException("Unknown query type: " + qry);
     }
 
     /**
@@ -420,8 +697,8 @@ public class GridSqlQueryParser {
 
         res = new GridSqlUnion();
 
-        res.right(parse(union.getRight()));
-        res.left(parse(union.getLeft()));
+        res.right(parse(union.getRight(), null));
+        res.left(parse(union.getLeft(), null));
 
         res.unionType(union.getUnionType());
 
@@ -478,7 +755,9 @@ public class GridSqlQueryParser {
                 parseExpression(expression.getNonAliasExpression(), calcTypes), true);
 
         if (expression instanceof ValueExpression)
-            return new GridSqlConst(expression.getValue(null));
+            // == comparison is legit, see ValueExpression#getSQL()
+            return expression == ValueExpression.getDefault() ? GridSqlKeyword.DEFAULT :
+                new GridSqlConst(expression.getValue(null));
 
         if (expression instanceof Operation) {
             Operation operation = (Operation)expression;
@@ -535,7 +814,7 @@ public class GridSqlQueryParser {
 
             assert0(qry instanceof Select, expression);
 
-            return new GridSqlSubquery(parse((Select)qry));
+            return new GridSqlSubquery(parse(qry, null));
         }
 
         if (expression instanceof ConditionIn) {
@@ -579,7 +858,7 @@ public class GridSqlQueryParser {
 
             assert0(qry instanceof Select, qry);
 
-            res.addChild(new GridSqlSubquery(parse((Select)qry)));
+            res.addChild(new GridSqlSubquery(parse(qry, null)));
 
             return res;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
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 7d43bf6..09952cf 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
@@ -165,7 +165,11 @@ public class GridSqlQuerySplitter {
 
         final Prepared prepared = prepared(stmt);
 
-        GridSqlQuery qry = new GridSqlQueryParser().parse(prepared);
+        GridSqlStatement gridStmt = new GridSqlQueryParser().parse(prepared);
+
+        assert gridStmt instanceof GridSqlQuery;
+
+        GridSqlQuery qry = (GridSqlQuery) gridStmt;
 
         qry = collectAllTables(qry, schemas, tbls);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
index a38ae68..f49a714 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
@@ -74,7 +74,8 @@ public class GridSqlSelect extends GridSqlQuery {
             buff.append(expression.getSQL());
         }
 
-        buff.append("\nFROM ").append(from.getSQL());
+        if (from != null)
+            buff.append("\nFROM ").append(from.getSQL());
 
         if (where != null)
             buff.append("\nWHERE ").append(StringUtils.unEnclose(where.getSQL()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlStatement.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlStatement.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlStatement.java
new file mode 100644
index 0000000..6eda0d7
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlStatement.java
@@ -0,0 +1,64 @@
+/*
+ * 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.h2.sql;
+
+/**
+ * SQL statement to query or update grid caches.
+ */
+public abstract class GridSqlStatement {
+    /** */
+    protected GridSqlElement limit;
+    /** */
+    private boolean explain;
+
+    /**
+     * @return Generate sql.
+     */
+    public abstract String getSQL();
+
+    /**
+     * @param explain Explain.
+     * @return {@code this}.
+     */
+    public GridSqlStatement explain(boolean explain) {
+        this.explain = explain;
+
+        return this;
+    }
+
+    /**
+     * @return {@code true} If explain.
+     */
+    public boolean explain() {
+        return explain;
+    }
+
+    /**
+     * @param limit Limit.
+     */
+    public void limit(GridSqlElement limit) {
+        this.limit = limit;
+    }
+
+    /**
+     * @return Limit.
+     */
+    public GridSqlElement limit() {
+        return limit;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUpdate.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUpdate.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUpdate.java
new file mode 100644
index 0000000..1623d3e
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUpdate.java
@@ -0,0 +1,105 @@
+/*
+ * 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.h2.sql;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import org.h2.util.StatementBuilder;
+import org.h2.util.StringUtils;
+
+/** */
+public class GridSqlUpdate extends GridSqlStatement {
+    /** */
+    private GridSqlElement target;
+
+    /** */
+    private ArrayList<GridSqlColumn> cols;
+
+    /** */
+    private LinkedHashMap<String, GridSqlElement> set;
+
+    /** */
+    private GridSqlElement where;
+
+    /** */
+    public GridSqlUpdate target(GridSqlElement target) {
+        this.target = target;
+        return this;
+    }
+
+    /** */
+    public GridSqlElement target() {
+        return target;
+    }
+
+    /** */
+    public GridSqlUpdate cols(ArrayList<GridSqlColumn> cols) {
+        this.cols = cols;
+        return this;
+    }
+
+    /** */
+    public ArrayList<GridSqlColumn> cols() {
+        return cols;
+    }
+
+
+    /** */
+    public GridSqlUpdate set(LinkedHashMap<String, GridSqlElement> set) {
+        this.set = set;
+        return this;
+    }
+
+    /** */
+    public GridSqlUpdate where(GridSqlElement where) {
+        this.where = where;
+        return this;
+    }
+
+    /** */
+    public GridSqlElement where() {
+        return where;
+    }
+
+    /** */
+    public LinkedHashMap<String, GridSqlElement> set() {
+        return set;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StatementBuilder buff = new StatementBuilder(explain() ? "EXPLAIN " : "");
+        buff.append("UPDATE ")
+            .append(target.getSQL())
+            .append("\nSET\n");
+
+        for (GridSqlColumn c : cols) {
+            GridSqlElement e = set.get(c.columnName());
+            buff.appendExceptFirst(",\n    ");
+            buff.append(c.columnName()).append(" = ").append(e != null ? e.getSQL() : "DEFAULT");
+        }
+
+        if (where != null)
+            buff.append("\nWHERE ").append(StringUtils.unEnclose(where.getSQL()));
+
+        if (limit != null)
+            buff.append("\nLIMIT ").append(StringUtils.unEnclose(limit.getSQL()));
+
+        return buff.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 48567da..0eaed12 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -65,7 +65,6 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
-import org.apache.ignite.internal.processors.query.h2.GridH2ResultSetIterator;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
@@ -715,7 +714,7 @@ public class GridReduceQueryExecutor {
                                 timeoutMillis,
                                 cancel);
 
-                            resIter = new Iter(res);
+                            resIter = new IgniteH2Indexing.FieldsIterator(res);
                         }
                         finally {
                             GridH2QueryContext.clearThreadLocal();
@@ -1377,31 +1376,6 @@ public class GridReduceQueryExecutor {
     /**
      *
      */
-    private static class Iter extends GridH2ResultSetIterator<List<?>> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         * @param data Data array.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected Iter(ResultSet data) throws IgniteCheckedException {
-            super(data, true, false);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected List<?> createRow() {
-            ArrayList<Object> res = new ArrayList<>(row.length);
-
-            Collections.addAll(res, row);
-
-            return res;
-        }
-    }
-
-    /**
-     *
-     */
     private class ExplicitPartitionsSpecializer implements IgniteBiClosure<ClusterNode,Message,Message> {
         /** */
         private final Map<ClusterNode,IntArray> partsMap;

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
new file mode 100644
index 0000000..2c3ab23
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
@@ -0,0 +1,567 @@
+/*
+ * 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.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryAbstractIdentityResolver;
+import org.apache.ignite.binary.BinaryArrayIdentityResolver;
+import org.apache.ignite.binary.BinaryFieldIdentityResolver;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.IgniteTestResources;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    protected final Marshaller marsh;
+
+    /**
+     *
+     */
+    IgniteCacheAbstractInsertSqlQuerySelfTest() {
+        try {
+            marsh = IgniteTestResources.getMarshaller();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /**
+     * @return whether {@link #marsh} is an instance of {@link BinaryMarshaller} or not.
+     */
+    boolean isBinaryMarshaller() {
+        return marsh instanceof BinaryMarshaller;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        BinaryConfiguration binCfg = new BinaryConfiguration();
+
+        binCfg.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration() {{
+                setTypeName(Key.class.getName());
+
+                setIdentityResolver(BinaryArrayIdentityResolver.instance());
+            }},
+            new BinaryTypeConfiguration() {{
+                setTypeName(Key2.class.getName());
+
+                setIdentityResolver(BinaryArrayIdentityResolver.instance());
+            }},
+            new BinaryTypeConfiguration() {{
+                setTypeName(Key3.class.getName());
+
+                setIdentityResolver(new BinaryFieldIdentityResolver().setFieldNames("key"));
+            }},
+            new BinaryTypeConfiguration() {{
+                setTypeName(Key4.class.getName());
+
+                setIdentityResolver(new Key4Id());
+            }}
+        ));
+
+        cfg.setBinaryConfiguration(binCfg);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3, false);
+
+        if (!isBinaryMarshaller())
+            createCaches();
+        else
+            createBinaryCaches();
+    }
+
+    /**
+     *
+     */
+    protected void createCaches() {
+        ignite(0).createCache(cacheConfig("S2P", true, false, String.class, Person.class, String.class, String.class));
+        ignite(0).createCache(cacheConfig("I2P", true, false, Integer.class, Person.class));
+        ignite(0).createCache(cacheConfig("K2P", true, false, Key.class, Person.class));
+        ignite(0).createCache(cacheConfig("K22P", true, true, Key2.class, Person2.class));
+        ignite(0).createCache(cacheConfig("I2I", true, false, Integer.class, Integer.class));
+    }
+
+    /**
+     *
+     */
+    final void createBinaryCaches() {
+        {
+            CacheConfiguration s2pCcfg = cacheConfig("S2P", true, false);
+
+            QueryEntity s2p = new QueryEntity(String.class.getName(), "Person");
+
+            s2p.setKeyFields(Collections.<String>emptySet());
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+
+            s2p.setFields(flds);
+
+            s2p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            QueryEntity s2s = new QueryEntity(String.class.getName(), String.class.getName());
+
+            s2s.setKeyFields(Collections.<String>emptySet());
+
+            s2pCcfg.setQueryEntities(Arrays.asList(s2p, s2s));
+
+            ignite(0).createCache(s2pCcfg);
+        }
+
+        {
+            CacheConfiguration i2pCcfg = cacheConfig("I2P", true, false);
+
+            QueryEntity i2p = new QueryEntity(Integer.class.getName(), "Person");
+
+            i2p.setKeyFields(Collections.<String>emptySet());
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+
+            i2p.setFields(flds);
+
+            i2p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            i2pCcfg.setQueryEntities(Collections.singletonList(i2p));
+
+            ignite(0).createCache(i2pCcfg);
+        }
+
+        {
+            CacheConfiguration k2pCcfg = cacheConfig("K2P", true, false);
+
+            QueryEntity k2p = new QueryEntity(Key.class.getName(), "Person");
+
+            k2p.setKeyFields(Collections.singleton("key"));
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("key", Integer.class.getName());
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+
+            k2p.setFields(flds);
+
+            k2p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            k2pCcfg.setQueryEntities(Collections.singletonList(k2p));
+
+            ignite(0).createCache(k2pCcfg);
+        }
+
+        {
+            CacheConfiguration k22pCcfg = cacheConfig("K22P", true, true);
+
+            QueryEntity k22p = new QueryEntity(Key2.class.getName(), "Person2");
+
+            k22p.setKeyFields(Collections.singleton("Id"));
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("Id", Integer.class.getName());
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+            flds.put("_Val", Integer.class.getName());
+
+            k22p.setFields(flds);
+
+            k22p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            k22pCcfg.setQueryEntities(Collections.singletonList(k22p));
+
+            ignite(0).createCache(k22pCcfg);
+        }
+
+        {
+            CacheConfiguration k32pCcfg = cacheConfig("K32P", true, false);
+
+            QueryEntity k32p = new QueryEntity(Key3.class.getName(), "Person");
+
+            k32p.setKeyFields(new HashSet<>(Arrays.asList("key", "strKey")));
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("key", Integer.class.getName());
+            flds.put("strKey", String.class.getName());
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+
+            k32p.setFields(flds);
+
+            k32p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            k32pCcfg.setQueryEntities(Collections.singletonList(k32p));
+
+            ignite(0).createCache(k32pCcfg);
+        }
+
+        {
+            CacheConfiguration k42pCcfg = cacheConfig("K42P", true, false);
+
+            QueryEntity k42p = new QueryEntity(Key4.class.getName(), "Person");
+
+            k42p.setKeyFields(new HashSet<>(Arrays.asList("key", "strKey")));
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("key", Integer.class.getName());
+            flds.put("strKey", String.class.getName());
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+
+            k42p.setFields(flds);
+
+            k42p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            k42pCcfg.setQueryEntities(Collections.singletonList(k42p));
+
+            ignite(0).createCache(k42pCcfg);
+        }
+
+        {
+            CacheConfiguration i2iCcfg = cacheConfig("I2I", true, false);
+
+            QueryEntity i2i = new QueryEntity(Integer.class.getName(), Integer.class.getName());
+
+            i2i.setKeyFields(Collections.<String>emptySet());
+
+            i2i.setFields(new LinkedHashMap<String, String>());
+
+            i2i.setIndexes(Collections.<QueryIndex>emptyList());
+
+            i2iCcfg.setQueryEntities(Collections.singletonList(i2i));
+
+            ignite(0).createCache(i2iCcfg);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        ignite(0).cache("S2P").clear();
+        ignite(0).cache("I2P").clear();
+        ignite(0).cache("K2P").clear();
+        ignite(0).cache("K22P").clear();
+        ignite(0).cache("I2I").clear();
+
+        if (isBinaryMarshaller()) {
+            ignite(0).cache("K32P").clear();
+            ignite(0).cache("K42P").clear();
+        }
+
+        super.afterTest();
+    }
+
+    /**
+     *
+     */
+    Object createPerson(int id, String name) {
+        if (!isBinaryMarshaller()) {
+            Person p = new Person(id);
+            p.name = name;
+
+            return p;
+        }
+        else {
+            BinaryObjectBuilder o = grid(0).binary().builder("Person");
+            o.setField("id", id);
+            o.setField("name", name);
+
+            return o.build();
+        }
+    }
+
+    /**
+     *
+     */
+    Object createPerson2(int id, String name, int valFld) {
+        if (!isBinaryMarshaller()) {
+            Person2 p = new Person2(id);
+            p.name = name;
+            p._Val = valFld;
+
+            return p;
+        }
+        else {
+            BinaryObjectBuilder o = grid(0).binary().builder("Person2");
+            o.setField("id", id);
+            o.setField("name", name);
+            o.setField("_Val", valFld);
+
+            return o.build();
+        }
+    }
+
+    /**
+     * @param name Cache name.
+     * @param partitioned Partition or replicated cache.
+     * @param escapeSql whether identifiers should be quoted - see {@link CacheConfiguration#setSqlEscapeAll}
+     * @param idxTypes Indexed types.
+     * @return Cache configuration.
+     */
+    private static CacheConfiguration cacheConfig(String name, boolean partitioned, boolean escapeSql, Class<?>... idxTypes) {
+        return new CacheConfiguration()
+            .setName(name)
+            .setCacheMode(partitioned ? CacheMode.PARTITIONED : CacheMode.REPLICATED)
+            .setAtomicityMode(CacheAtomicityMode.ATOMIC)
+            .setBackups(1)
+            .setSqlEscapeAll(escapeSql)
+            .setIndexedTypes(idxTypes);
+    }
+
+    /**
+     *
+     */
+    protected final static class Key implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        public Key(int key) {
+            this.key = key;
+        }
+
+        /** */
+        @QuerySqlField
+        public final int key;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Key key1 = (Key) o;
+
+            return key == key1.key;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return key;
+        }
+    }
+
+    /**
+     *
+     */
+    protected final static class Key2 implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        public Key2(int Id) {
+            this.Id = Id;
+        }
+
+        /** */
+        @QuerySqlField
+        public final int Id;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Key2 key1 = (Key2) o;
+
+            return Id == key1.Id;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return Id;
+        }
+    }
+
+    /**
+     *
+     */
+    final static class Key3 implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        public Key3(int key) {
+            this.key = key;
+            this.strKey = Integer.toString(key);
+        }
+
+        /** */
+        @QuerySqlField
+        public final int key;
+
+        /** */
+        @QuerySqlField
+        public final String strKey;
+    }
+
+    /**
+     *
+     */
+    final static class Key4 implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        public Key4(int key) {
+            this.key = key;
+            this.strKey = Integer.toString(key);
+        }
+
+        /** */
+        @QuerySqlField
+        public final int key;
+
+        /** */
+        @QuerySqlField
+        public final String strKey;
+    }
+
+    /**
+     *
+     */
+    final static class Key4Id extends BinaryAbstractIdentityResolver {
+        /** {@inheritDoc} */
+        @Override protected int hashCode0(BinaryObject obj) {
+            return (int) obj.field("key") * 100;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected boolean equals0(BinaryObject o1, BinaryObject o2) {
+            return (int) o1.field("key") == (int) o2.field("key");
+        }
+    }
+
+    /**
+     *
+     */
+    protected static class Person implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        @SuppressWarnings("unused")
+        private Person() {
+            // No-op.
+        }
+
+        /** */
+        public Person(int id) {
+            this.id = id;
+        }
+
+        /** */
+        @QuerySqlField
+        protected int id;
+
+        /** */
+        @QuerySqlField
+        protected String name;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Person person = (Person) o;
+
+            if (id != person.id) return false;
+            return name != null ? name.equals(person.name) : person.name == null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int result = id;
+            result = 31 * result + (name != null ? name.hashCode() : 0);
+            return result;
+        }
+    }
+
+    /**
+     *
+     */
+    protected static class Person2 extends Person {
+        /** */
+        @SuppressWarnings("unused")
+        private Person2() {
+            // No-op.
+        }
+
+        /** */
+        public Person2(int id) {
+            super(id);
+        }
+
+        /** */
+        @QuerySqlField
+        public int _Val;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
new file mode 100644
index 0000000..2dbf1b4
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
@@ -0,0 +1,219 @@
+/*
+ * 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.io.Serializable;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.IgniteTestResources;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    protected final Marshaller marsh;
+
+    /**
+     *
+     */
+    IgniteCacheAbstractSqlDmlQuerySelfTest() {
+        try {
+            marsh = IgniteTestResources.getMarshaller();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /**
+     * @return whether {@link #marsh} is an instance of {@link BinaryMarshaller} or not.
+     */
+    private boolean isBinaryMarshaller() {
+        return marsh instanceof BinaryMarshaller;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3, true);
+
+        ignite(0).createCache(cacheConfig());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+        ignite(0).cache("S2P").put("FirstKey", createPerson(1, "John", "White"));
+        ignite(0).cache("S2P").put("SecondKey", createPerson(2, "Joe", "Black"));
+        ignite(0).cache("S2P").put("k3", createPerson(3, "Sylvia", "Green"));
+        ignite(0).cache("S2P").put("f0u4thk3y", createPerson(4, "Jane", "Silver"));
+    }
+
+    /** */
+    Object createPerson(int id, String name, String secondName) {
+        if (!isBinaryMarshaller())
+            return new Person(id, name, secondName);
+        else {
+            BinaryObjectBuilder bldr = ignite(0).binary().builder("Person");
+
+            bldr.setField("id", id);
+            bldr.setField("name", name);
+            bldr.setField("secondName", secondName);
+
+            return bldr.build();
+        }
+
+    }
+
+    /** */
+    protected IgniteCache<?, ?> cache() {
+        return ignite(0).cache("S2P").withKeepBinary();
+    }
+
+    /** */
+    protected CacheConfiguration cacheConfig() {
+        if (!isBinaryMarshaller())
+            return cacheConfig("S2P", true, false).setIndexedTypes(String.class, Person.class);
+        else
+            return createBinCacheConfig();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @param name Cache name.
+     * @param partitioned Partition or replicated cache.
+     * @param escapeSql whether identifiers should be quoted - see {@link CacheConfiguration#setSqlEscapeAll}
+     * @return Cache configuration.
+     */
+    private static CacheConfiguration cacheConfig(String name, boolean partitioned, boolean escapeSql) {
+        return new CacheConfiguration()
+            .setName(name)
+            .setCacheMode(partitioned ? CacheMode.PARTITIONED : CacheMode.REPLICATED)
+            .setAtomicityMode(CacheAtomicityMode.ATOMIC)
+            .setBackups(1)
+            .setSqlEscapeAll(escapeSql);
+    }
+
+    /**
+     *
+     */
+    private static CacheConfiguration createBinCacheConfig() {
+        CacheConfiguration ccfg = cacheConfig("S2P", true, false);
+
+        QueryEntity e = new QueryEntity(String.class.getName(), "Person");
+
+        e.setKeyFields(Collections.<String>emptySet());
+
+        LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+        flds.put("id", Integer.class.getName());
+        flds.put("name", String.class.getName());
+        flds.put("secondName", String.class.getName());
+
+        e.setFields(flds);
+
+        e.setIndexes(Collections.<QueryIndex>emptyList());
+
+        ccfg.setQueryEntities(Collections.singletonList(e));
+
+        return ccfg;
+    }
+
+    /**
+     *
+     */
+    private static class Person implements Serializable {
+        /** */
+        public Person(int id, String name, String secondName) {
+            this.id = id;
+            this.name = name;
+            this.secondName = secondName;
+        }
+
+        /** */
+        @QuerySqlField
+        protected int id;
+
+        /** */
+        @QuerySqlField
+        protected final String name;
+
+        /** */
+        @QuerySqlField
+        final String secondName;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Person person = (Person) o;
+
+            return id == person.id && name.equals(person.name) && secondName.equals(person.secondName);
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = id;
+            res = 31 * res + name.hashCode();
+            res = 31 * res + secondName.hashCode();
+            return res;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
new file mode 100644
index 0000000..8b0a033
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.Arrays;
+import java.util.List;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public class IgniteCacheDeleteSqlQuerySelfTest extends IgniteCacheAbstractSqlDmlQuerySelfTest {
+    /**
+     *
+     */
+    public void testDeleteSimple() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("delete from Person p where length(p._key) = 2 " +
+            "or p.secondName like '%ite'"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by id"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(2, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(1));
+    }
+
+    /**
+     *
+     */
+    public void testDeleteSingle() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("delete from Person where _key = ?")
+            .setArgs("FirstKey"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by id, _key"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(3, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
+            leftovers.get(1));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(2));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
new file mode 100644
index 0000000..4f87740
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
@@ -0,0 +1,203 @@
+/*
+ * 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.concurrent.Callable;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsertSqlQuerySelfTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /**
+     *
+     */
+    public void testInsertWithExplicitKey() {
+        IgniteCache<String, Person> p = ignite(0).cache("S2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("insert into Person (_key, id, name) values ('s', ?, ?), " +
+            "('a', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get("s"));
+
+        assertEquals(createPerson(2, "Alex"), p.get("a"));
+    }
+
+    /**
+     *
+     */
+    public void testInsertFromSubquery() {
+        IgniteCache p = ignite(0).cache("S2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("insert into String (_key, _val) values ('s', ?), " +
+            "('a', ?)").setArgs("Sergi", "Alex").setLocal(true));
+
+        assertEquals("Sergi", p.get("s"));
+        assertEquals("Alex", p.get("a"));
+
+        p.query(new SqlFieldsQuery("insert into Person(_key, id, name) " +
+            "(select substring(lower(_val), 0, 2), cast(length(_val) as int), _val from String)"));
+
+        assertEquals(createPerson(5, "Sergi"), p.get("se"));
+
+        assertEquals(createPerson(4, "Alex"), p.get("al"));
+    }
+
+    /**
+     *
+     */
+    public void testInsertWithExplicitPrimitiveKey() {
+        IgniteCache<Integer, Person> p = ignite(0).cache("I2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "insert into Person (_key, id, name) values (cast('1' as int), ?, ?), (2, (5 - 3), 'Alex')")
+            .setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(1));
+
+        assertEquals(createPerson(2, "Alex"), p.get(2));
+    }
+
+    /**
+     *
+     */
+    public void testInsertWithDynamicKeyInstantiation() {
+        IgniteCache<Key, Person> p = ignite(0).cache("K2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "insert into Person (key, id, name) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key(2)));
+    }
+
+    /**
+     *
+     */
+    public void testFieldsCaseSensitivity() {
+        IgniteCache<Key2, Person> p = ignite(0).cache("K22P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("insert into \"Person2\" (\"Id\", \"id\", \"name\", \"_Val\") values (1, ?, ?, 5), " +
+            "(2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
+
+        assertEquals(createPerson2(4, "Sergi", 5), p.get(new Key2(1)));
+
+        assertEquals(createPerson2(3, "Alex", 6), p.get(new Key2(2)));
+    }
+
+    /**
+     *
+     */
+    public void testPrimitives() {
+        IgniteCache<Integer, Integer> p = ignite(0).cache("I2I");
+
+        p.query(new SqlFieldsQuery("insert into Integer(_key, _val) values (1, ?), " +
+            "(?, 4)").setArgs(2, 3));
+
+        assertEquals(2, (int)p.get(1));
+
+        assertEquals(4, (int)p.get(3));
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testDuplicateKeysException() {
+        final IgniteCache<Integer, Integer> p = ignite(0).cache("I2I");
+
+        p.clear();
+
+        p.put(3, 5);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            /** {@inheritDoc} */
+            @Override public Void call() throws Exception {
+                p.query(new SqlFieldsQuery("insert into Integer(_key, _val) values (1, ?), " +
+                    "(?, 4), (5, 6)").setArgs(2, 3));
+
+                return null;
+            }
+        }, CacheException.class, "Failed to INSERT some keys because they are already in cache [keys=[3]]");
+
+        assertEquals(2, (int)p.get(1));
+        assertEquals(5, (int)p.get(3));
+        assertEquals(6, (int)p.get(5));
+    }
+
+    /**
+     *
+     */
+    public void testFieldsListIdentity() {
+        if (!isBinaryMarshaller())
+            return;
+
+        IgniteCache<Key3, Person> p = ignite(0).cache("K32P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "insert into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key3(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key3(2)));
+    }
+
+    /**
+     *
+     */
+    public void testCustomIdentity() {
+        if (!isBinaryMarshaller())
+            return;
+
+        IgniteCache<Key4, Person> p = ignite(0).cache("K42P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "insert into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key4(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key4(2)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
new file mode 100644
index 0000000..e487564
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
@@ -0,0 +1,153 @@
+/*
+ * 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 org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertSqlQuerySelfTest {
+    /**
+     *
+     */
+    public void testMergeWithExplicitKey() {
+        IgniteCache<String, Person> p = ignite(0).cache("S2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("merge into Person (_key, id, name) values ('s', ?, ?), " +
+            "('a', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get("s"));
+
+        assertEquals(createPerson(2, "Alex"), p.get("a"));
+    }
+
+    /**
+     *
+     */
+    public void testMergeFromSubquery() {
+        IgniteCache p = ignite(0).cache("S2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("merge into String (_key, _val) values ('s', ?), " +
+            "('a', ?)").setArgs("Sergi", "Alex").setLocal(true));
+
+        assertEquals("Sergi", p.get("s"));
+        assertEquals("Alex", p.get("a"));
+
+        p.query(new SqlFieldsQuery("merge into Person(_key, id, name) " +
+            "(select substring(lower(_val), 0, 2), cast(length(_val) as int), _val from String)"));
+
+        assertEquals(createPerson(5, "Sergi"), p.get("se"));
+
+        assertEquals(createPerson(4, "Alex"), p.get("al"));
+    }
+
+    /**
+     *
+     */
+    public void testMergeWithExplicitPrimitiveKey() {
+        IgniteCache<Integer, Person> p = ignite(0).cache("I2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "merge into Person (_key, id, name) values (cast(? as int), ?, ?), (2, (5 - 3), 'Alex')")
+            .setArgs("1", 1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(1));
+
+        assertEquals(createPerson(2, "Alex"), p.get(2));
+    }
+
+    /**
+     *
+     */
+    public void testMergeWithDynamicKeyInstantiation() {
+        IgniteCache<Key, Person> p = ignite(0).cache("K2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "merge into Person (key, id, name) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key(2)));
+    }
+
+    /**
+     *
+     */
+    public void testFieldsCaseSensitivity() {
+        IgniteCache<Key2, Person> p = ignite(0).cache("K22P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("merge into \"Person2\" (\"Id\", \"id\", \"name\", \"_Val\") values (1, ?, ?, 5), " +
+            "(2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
+
+        assertEquals(createPerson2(4, "Sergi", 5), p.get(new Key2(1)));
+
+        assertEquals(createPerson2(3, "Alex", 6), p.get(new Key2(2)));
+    }
+
+    /**
+     *
+     */
+    public void testPrimitives() {
+        IgniteCache<Integer, Integer> p = ignite(0).cache("I2I").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("merge into Integer(_key, _val) values (1, ?), " +
+            "(?, 4)").setArgs(2, 3));
+
+        assertEquals(2, (int)p.get(1));
+
+        assertEquals(4, (int)p.get(3));
+    }
+
+    /**
+     *
+     */
+    public void testFieldsListIdentity() {
+        if (!isBinaryMarshaller())
+            return;
+
+        IgniteCache<Key3, Person> p = ignite(0).cache("K32P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "merge into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key3(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key3(2)));
+    }
+
+    /**
+     *
+     */
+    public void testCustomIdentity() {
+        if (!isBinaryMarshaller())
+            return;
+
+        IgniteCache<Key4, Person> p = ignite(0).cache("K42P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "merge into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key4(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key4(2)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
new file mode 100644
index 0000000..538141f
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
@@ -0,0 +1,150 @@
+/*
+ * 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.Arrays;
+import java.util.List;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDmlQuerySelfTest {
+    /**
+     *
+     */
+    public void testUpdateSimple() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set p.id = p.id * 2, p.name = " +
+            "substring(p.name, 0, 2) where length(p._key) = ? or p.secondName like ?").setArgs(2, "%ite"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by _key, id"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(4, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("FirstKey", createPerson(2, "Jo", "White"), 2, "Jo", "White"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(1));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(2));
+
+        assertEqualsCollections(Arrays.asList("k3", createPerson(6, "Sy", "Green"), 6, "Sy", "Green"),
+            leftovers.get(3));
+    }
+
+    /**
+     *
+     */
+    public void testUpdateSingle() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set _val = ? where _key = ?")
+            .setArgs(createPerson(2, "Jo", "White"), "FirstKey"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by id, _key"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(4, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("FirstKey", createPerson(2, "Jo", "White"), 2, "Jo", "White"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(1));
+
+        assertEqualsCollections(Arrays.asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
+            leftovers.get(2));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(3));
+    }
+
+    /**
+     *
+     */
+    public void testUpdateValueAndFields() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set id = ?, _val = ? where _key = ?")
+            .setArgs(44, createPerson(2, "Jo", "Woo"), "FirstKey"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by _key, id"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(4, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("FirstKey", createPerson(44, "Jo", "Woo"), 44, "Jo", "Woo"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(1));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(2));
+
+        assertEqualsCollections(Arrays.asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
+            leftovers.get(3));
+    }
+
+    /**
+     *
+     */
+    public void testDefault() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set id = DEFAULT, _val = ? where _key = ?")
+            .setArgs(createPerson(2, "Jo", "Woo"), "FirstKey"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by _key, id"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(4, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("FirstKey", createPerson(0, "Jo", "Woo"), 0, "Jo", "Woo"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(1));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(2));
+
+        assertEqualsCollections(Arrays.asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
+            leftovers.get(3));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheAtomicFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheAtomicFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheAtomicFieldsQuerySelfTest.java
index d9c4ac2..ef111e3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheAtomicFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheAtomicFieldsQuerySelfTest.java
@@ -17,12 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
-import java.util.List;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.query.QueryCursor;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.util.typedef.X;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 
@@ -39,21 +35,4 @@ public class IgniteCacheAtomicFieldsQuerySelfTest extends IgniteCachePartitioned
     @Override protected NearCacheConfiguration nearConfiguration() {
         return null;
     }
-
-    /**
-     *
-     */
-    public void testUnsupportedOperations() {
-        try {
-            QueryCursor<List<?>> qry = grid(0).cache(null).query(new SqlFieldsQuery(
-                "update Person set name = ?").setArgs("Mary Poppins"));
-
-            qry.getAll();
-
-            fail("We don't support updates.");
-        }
-        catch (Exception e) {
-            X.println("___ " + e.getMessage());
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index b4abbf6..512001f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -542,7 +542,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         /**
          * @return Space name.
          */
-        public String space() {
+        String space() {
             return space;
         }
 
@@ -554,18 +554,30 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         /** {@inheritDoc} */
         @Override public GridQueryProperty property(final String name) {
             return new GridQueryProperty() {
+                /** */
                 @Override public Object value(Object key, Object val) throws IgniteCheckedException {
                     return TypeDesc.this.value(name, key, val);
                 }
 
+                /** */
+                @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
+                    throw new UnsupportedOperationException();
+                }
+
+                /** */
                 @Override public String name() {
                     return name;
                 }
 
-                @Override
-                public Class<?> type() {
+                /** */
+                @Override public Class<?> type() {
                     return Object.class;
                 }
+
+                /** */
+                @Override public boolean key() {
+                    return false;
+                }
             };
         }
 
@@ -584,6 +596,18 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
             return null;
         }
 
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void setValue(String field, Object key, Object val, Object propVal) throws IgniteCheckedException {
+            assert !F.isEmpty(field);
+
+            assert key instanceof Integer;
+
+            Map<String, Object> m = (Map<String, Object>)val;
+
+            m.put(field, propVal);
+        }
+
         /** */
         @Override public Map<String, GridQueryIndexDescriptor> indexes() {
             return textIdx == null ? Collections.<String, GridQueryIndexDescriptor>emptyMap() :
@@ -601,6 +625,16 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** */
+        @Override public String keyTypeName() {
+            return null;
+        }
+
+        /** */
+        @Override public String valueTypeName() {
+            return null;
+        }
+
+        /** */
         @Override public boolean valueTextIndex() {
             return textIdx == null;
         }


[43/52] ignite git commit: IGNITE-4243 Fixed build.

Posted by vo...@apache.org.
IGNITE-4243 Fixed build.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/44bb3614
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/44bb3614
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/44bb3614

Branch: refs/heads/master
Commit: 44bb361491589cac93899ce9b45f9ad1a535a950
Parents: e205214
Author: Andrey Novikov <an...@gridgain.com>
Authored: Tue Nov 29 09:51:28 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Tue Nov 29 09:51:28 2016 +0700

----------------------------------------------------------------------
 assembly/release-fabric-base.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/44bb3614/assembly/release-fabric-base.xml
----------------------------------------------------------------------
diff --git a/assembly/release-fabric-base.xml b/assembly/release-fabric-base.xml
index 7b3d8cf..c008083 100644
--- a/assembly/release-fabric-base.xml
+++ b/assembly/release-fabric-base.xml
@@ -203,6 +203,7 @@
             <includes>
                 <include>rest/**</include>
                 <include>memcached/**</include>
+                <include>redis/**</include>
                 <include>schema-import/src/**</include>
                 <include>schema-import/bin/**</include>
                 <include>schema-import/README.txt</include>


[22/52] ignite git commit: .NET: Remove unused code, sort example tests

Posted by vo...@apache.org.
.NET: Remove unused code, sort example tests


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/3f797b67
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3f797b67
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3f797b67

Branch: refs/heads/master
Commit: 3f797b67384f6237a51d358727bc406c5610d0cf
Parents: b04460c
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Nov 25 12:36:46 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Nov 25 12:36:46 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs        | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3f797b67/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs
index 663711c..4df012c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs
@@ -37,9 +37,6 @@ namespace Apache.Ignite.Core.Tests.Examples
         /** Config url */
         public string ConfigPath { get; private set; }
 
-        /** Source path */
-        public string SourceFilePath { get; private set; }
-
         /** Dll load flag */
         public bool NeedsTestDll { get; private set; }
 
@@ -75,7 +72,7 @@ namespace Apache.Ignite.Core.Tests.Examples
 
             Assert.IsTrue(sourceFiles.Any());
 
-            var types = examplesAsm.GetTypes().Where(x => x.GetMethod("Main") != null).ToArray();
+            var types = examplesAsm.GetTypes().Where(x => x.GetMethod("Main") != null).OrderBy(x => x.Name).ToArray();
 
             Assert.IsTrue(types.Any());
 
@@ -89,7 +86,6 @@ namespace Apache.Ignite.Core.Tests.Examples
 
                 yield return new Example
                 {
-                    SourceFilePath = sourceFile,
                     ConfigPath = GetConfigPath(sourceCode),
                     NeedsTestDll = sourceCode.Contains(examplesDllName),
                     _runAction = GetRunAction(type),


[04/52] ignite git commit: IGNITE-2294: Implemented DML.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 6da8758..f4ffbc7 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -68,6 +68,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridTopic;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.jdbc2.JdbcSqlFieldsQuery;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
@@ -79,6 +80,7 @@ import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
@@ -87,6 +89,7 @@ import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryIndexing;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOffheap;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap;
@@ -98,6 +101,7 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2TreeIndex;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
 import org.apache.ignite.internal.processors.query.h2.opt.GridLuceneIndex;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor;
@@ -129,6 +133,7 @@ import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.h2.api.ErrorCode;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.command.CommandInterface;
+import org.h2.command.Prepared;
 import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Index;
@@ -314,7 +319,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 c = DriverManager.getConnection(dbUrl);
             }
             catch (SQLException e) {
-                throw new IgniteException("Failed to initialize DB connection: " + dbUrl, e);
+                throw new IgniteSQLException("Failed to initialize DB connection: " + dbUrl, e);
             }
 
             conns.add(c);
@@ -327,6 +332,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private volatile GridKernalContext ctx;
 
     /** */
+    private final DmlStatementsProcessor dmlProc = new DmlStatementsProcessor(this);
+
+    /** */
     private final ConcurrentMap<String, GridH2Table> dataTables = new ConcurrentHashMap8<>();
 
     /** Statement cache. */
@@ -369,6 +377,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
+     * @return Logger.
+     */
+    IgniteLogger getLogger() {
+        return log;
+    }
+
+    /**
      * @param c Connection.
      * @param sql SQL.
      * @param useStmtCache If {@code true} uses statement cache.
@@ -410,6 +425,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             return c.prepareStatement(sql);
     }
 
+    /** {@inheritDoc} */
+    @Override public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException {
+        return prepareStatement(connectionForSpace(schema), sql, false);
+    }
+
     /**
      * Gets DB connection.
      *
@@ -437,7 +457,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 c.schema(schema);
             }
             catch (SQLException e) {
-                throw new IgniteCheckedException("Failed to set schema for DB connection for thread [schema=" +
+                throw new IgniteSQLException("Failed to set schema for DB connection for thread [schema=" +
                     schema + "]", e);
             }
             finally {
@@ -492,7 +512,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         catch (SQLException e) {
             onSqlException();
 
-            throw new IgniteCheckedException("Failed to execute statement: " + sql, e);
+            throw new IgniteSQLException("Failed to execute statement: " + sql, e);
         }
         finally {
             U.close(stmt, log);
@@ -732,8 +752,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         catch (SQLException e) {
             onSqlException();
 
-            throw new IgniteCheckedException("Failed to drop database index table [type=" + tbl.type().name() +
-                ", table=" + tbl.fullTableName() + "]", e);
+            throw new IgniteSQLException("Failed to drop database index table [type=" + tbl.type().name() +
+                ", table=" + tbl.fullTableName() + "]", IgniteQueryErrorCode.TABLE_DROP_FAILED, e);
         }
         finally {
             U.close(stmt, log);
@@ -776,9 +796,27 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         initLocalQueryContext(conn, enforceJoinOrder, filters);
 
+        Prepared p = null;
+
         try {
             final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
 
+            p = GridSqlQueryParser.prepared((JdbcPreparedStatement) stmt);
+
+            if (!p.isQuery()) {
+                GridH2QueryContext.clearThreadLocal();
+
+                SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry);
+
+                if (params != null)
+                    fldsQry.setArgs(params.toArray());
+
+                fldsQry.setEnforceJoinOrder(enforceJoinOrder);
+                fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
+
+                return dmlProc.updateLocalSqlFields(spaceName, stmt, fldsQry, filters, cancel);
+            }
+
             List<GridQueryFieldMetadata> meta;
 
             try {
@@ -797,7 +835,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             };
         }
         finally {
-            GridH2QueryContext.clearThreadLocal();
+            if (p == null || p.isQuery())
+                GridH2QueryContext.clearThreadLocal();
         }
     }
 
@@ -874,16 +913,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             throw new IgniteCheckedException("Failed to parse SQL query: " + sql, e);
         }
 
-        switch (commandType(stmt)) {
-            case CommandInterface.SELECT:
-            case CommandInterface.CALL:
-            case CommandInterface.EXPLAIN:
-            case CommandInterface.ANALYZE:
-                break;
-            default:
-                throw new IgniteCheckedException("Failed to execute non-query SQL statement: " + sql);
-        }
-
         bindParameters(stmt, params);
 
         return stmt;
@@ -1055,7 +1084,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         final TableDescriptor tbl = tableDescriptor(spaceName, type);
 
         if (tbl == null)
-            throw new CacheException("Failed to find SQL table for type: " + type.name());
+            throw new IgniteSQLException("Failed to find SQL table for type: " + type.name(),
+                IgniteQueryErrorCode.TABLE_NOT_FOUND);
 
         String sql = generateQuery(qry, tbl);
 
@@ -1100,7 +1130,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         TableDescriptor tblDesc = tableDescriptor(type, space);
 
         if (tblDesc == null)
-            throw new CacheException("Failed to find SQL table for type: " + type);
+            throw new IgniteSQLException("Failed to find SQL table for type: " + type,
+                IgniteQueryErrorCode.TABLE_NOT_FOUND);
 
         String sql;
 
@@ -1120,7 +1151,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if(qry.getTimeout() > 0)
             fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS);
 
-        final QueryCursor<List<?>> res = queryTwoStep(cctx, fqry);
+        final QueryCursor<List<?>> res = queryTwoStep(cctx, fqry, null);
 
         final Iterable<Cache.Entry<K, V>> converted = new Iterable<Cache.Entry<K, V>>() {
             @Override public Iterator<Cache.Entry<K, V>> iterator() {
@@ -1161,7 +1192,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, SqlFieldsQuery qry) {
+    @Override public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
+        GridQueryCancel cancel) {
         final String space = cctx.name();
         final String sqlQry = qry.getSql();
 
@@ -1214,7 +1246,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                             cachesCreated = true;
                         }
                         else
-                            throw new CacheException("Failed to parse query: " + sqlQry, e);
+                            throw new IgniteSQLException("Failed to parse query: " + sqlQry, e.getSQLState(),
+                                IgniteQueryErrorCode.PARSING);
                     }
                 }
             }
@@ -1222,6 +1255,22 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 GridH2QueryContext.clearThreadLocal();
             }
 
+            Prepared prepared = GridSqlQueryParser.prepared((JdbcPreparedStatement) stmt);
+
+            if (qry instanceof JdbcSqlFieldsQuery && ((JdbcSqlFieldsQuery) qry).isQuery() != prepared.isQuery())
+                throw new IgniteSQLException("Given statement type does not match that declared by JDBC driver",
+                    IgniteQueryErrorCode.STMT_TYPE_MISMATCH);
+
+            if (!prepared.isQuery()) {
+                try {
+                    return dmlProc.updateSqlFieldsTwoStep(cctx.namexx(), stmt, qry, cancel);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteSQLException("Failed to execute DML statement [qry=" + sqlQry + ", params=" +
+                        Arrays.deepToString(qry.getArgs()) + "]", e);
+                }
+            }
+
             try {
                 bindParameters(stmt, F.asList(qry.getArgs()));
 
@@ -1271,7 +1320,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     Arrays.deepToString(qry.getArgs()) + "]", e);
             }
             catch (SQLException e) {
-                throw new CacheException(e);
+                throw new IgniteSQLException(e);
             }
             finally {
                 U.close(stmt, log);
@@ -1283,7 +1332,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         twoStepQry.pageSize(qry.getPageSize());
 
-        GridQueryCancel cancel = new GridQueryCancel();
+        if (cancel == null)
+            cancel = new GridQueryCancel();
 
         QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
             runQueryTwoStep(cctx, twoStepQry, cctx.keepBinary(), enforceJoinOrder, qry.getTimeout(), cancel), cancel);
@@ -1433,7 +1483,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param escapeAll Escape flag.
      * @return Escaped name.
      */
-    private static String escapeName(String name, boolean escapeAll) {
+    public static String escapeName(String name, boolean escapeAll) {
         if (name == null) // It is possible only for a cache name.
             return ESC_STR;
 
@@ -2557,7 +2607,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Special field set iterator based on database result set.
      */
-    private static class FieldsIterator extends GridH2ResultSetIterator<List<?>> {
+    public static class FieldsIterator extends GridH2ResultSetIterator<List<?>> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2565,7 +2615,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
          * @param data Data.
          * @throws IgniteCheckedException If failed.
          */
-        protected FieldsIterator(ResultSet data) throws IgniteCheckedException {
+        public FieldsIterator(ResultSet data) throws IgniteCheckedException {
             super(data, false, true);
         }
 
@@ -2607,7 +2657,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Field descriptor.
      */
-    private static class SqlFieldMetadata implements GridQueryFieldMetadata {
+    static class SqlFieldMetadata implements GridQueryFieldMetadata {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -3012,6 +3062,21 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
 
         /** {@inheritDoc} */
+        @Override public void setColumnValue(Object key, Object val, Object colVal, int col) {
+            try {
+                props[col].setValue(key, val, colVal);
+            }
+            catch (IgniteCheckedException e) {
+                throw DbException.convert(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isColumnKeyProperty(int col) {
+            return props[col].key();
+        }
+
+        /** {@inheritDoc} */
         @Override public GridH2KeyValueRowOffheap createPointer(long ptr) {
             GridH2KeyValueRowOffheap row = schema.rowCache.get(ptr);
 
@@ -3033,6 +3098,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         @Override public boolean snapshotableIndex() {
             return snapshotableIdx;
         }
+
+        /** {@inheritDoc} */
+        @Override public boolean quoteAllIdentifiers() {
+            return schema.escapeAll();
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java
new file mode 100644
index 0000000..dc90fe9
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java
@@ -0,0 +1,27 @@
+/*
+ * 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.h2.dml;
+
+import org.apache.ignite.internal.util.lang.GridPlainClosure;
+
+/**
+ * Operand for fast UPDATE or DELETE (single item operation that does not involve any SELECT).
+ */
+public interface FastUpdateArgument extends GridPlainClosure<Object[], Object> {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
new file mode 100644
index 0000000..cb47704
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
@@ -0,0 +1,53 @@
+/*
+ * 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.h2.dml;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Arguments for fast, query-less UPDATE or DELETE - key and, optionally, value and new value.
+ */
+public final class FastUpdateArguments {
+    /** Operand to compute key. */
+    public final FastUpdateArgument key;
+
+    /** Operand to compute value. */
+    public final FastUpdateArgument val;
+
+    /** Operand to compute new value. */
+    public final FastUpdateArgument newVal;
+
+    /** */
+    public FastUpdateArguments(FastUpdateArgument key, FastUpdateArgument val, FastUpdateArgument newVal) {
+        assert key != null && key != NULL_ARGUMENT;
+        assert val != null;
+        assert newVal != null;
+
+        this.key = key;
+        this.val = val;
+        this.newVal = newVal;
+    }
+
+    /** Operand that always evaluates as {@code null}. */
+    public final static FastUpdateArgument NULL_ARGUMENT = new FastUpdateArgument() {
+        /** {@inheritDoc} */
+        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
+            return null;
+        }
+    };
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/KeyValueSupplier.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/KeyValueSupplier.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/KeyValueSupplier.java
new file mode 100644
index 0000000..709d5c9
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/KeyValueSupplier.java
@@ -0,0 +1,30 @@
+/*
+ * 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.h2.dml;
+
+import java.util.List;
+import org.apache.ignite.internal.util.lang.GridPlainClosure;
+
+/**
+ * Method to construct new instances of keys and values on SQL MERGE and INSERT,
+ * as well as to build new values during UPDATE - a function that takes a row selected from DB
+ * and then transforms it into new object.
+ */
+public interface KeyValueSupplier extends GridPlainClosure<List<?>, Object> {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java
new file mode 100644
index 0000000..0440648
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java
@@ -0,0 +1,36 @@
+/*
+ * 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.h2.dml;
+
+/**
+ * DML statement execution plan type - MERGE/INSERT from rows or subquery,
+ * or UPDATE/DELETE from subquery or literals/params based.
+ */
+public enum UpdateMode {
+    /** */
+    MERGE,
+
+    /** */
+    INSERT,
+
+    /** */
+    UPDATE,
+
+    /** */
+    DELETE,
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
new file mode 100644
index 0000000..5976f4c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
@@ -0,0 +1,121 @@
+/*
+ * 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.h2.dml;
+
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Update plan - where to take data to update cache from and how to construct new keys and values, if needed.
+ */
+public final class UpdatePlan {
+    /** Initial statement to drive the rest of the logic. */
+    public final UpdateMode mode;
+
+    /** Target table to be affected by initial DML statement. */
+    public final GridH2Table tbl;
+
+    /** Column names to set or update. */
+    public final String[] colNames;
+
+    /** Method to create key for INSERT or MERGE, ignored for UPDATE and DELETE. */
+    public final KeyValueSupplier keySupplier;
+
+    /** Method to create value to put to cache, ignored for DELETE. */
+    public final KeyValueSupplier valSupplier;
+
+    /** Index of key column, if it's explicitly mentioned in column list of MERGE or INSERT,
+     * ignored for UPDATE and DELETE. */
+    public final int keyColIdx;
+
+    /** Index of value column, if it's explicitly mentioned in column list. Ignored for UPDATE and DELETE. */
+    public final int valColIdx;
+
+    /** SELECT statement built upon initial DML statement. */
+    public final String selectQry;
+
+    /** Subquery flag - {@code true} if {@link #selectQry} is an actual subquery that retrieves data from some cache. */
+    public final boolean isLocSubqry;
+
+    /** Number of rows in rows based MERGE or INSERT. */
+    public final int rowsNum;
+
+    /** Arguments for fast UPDATE or DELETE. */
+    public final FastUpdateArguments fastUpdateArgs;
+
+    /** */
+    private UpdatePlan(UpdateMode mode, GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+        KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
+        int rowsNum, FastUpdateArguments fastUpdateArgs) {
+        this.colNames = colNames;
+        this.rowsNum = rowsNum;
+        assert mode != null;
+        assert tbl != null;
+
+        this.mode = mode;
+        this.tbl = tbl;
+        this.keySupplier = keySupplier;
+        this.valSupplier = valSupplier;
+        this.keyColIdx = keyColIdx;
+        this.valColIdx = valColIdx;
+        this.selectQry = selectQry;
+        this.isLocSubqry = isLocSubqry;
+        this.fastUpdateArgs = fastUpdateArgs;
+    }
+
+    /** */
+    public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+        KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
+        int rowsNum) {
+        assert !F.isEmpty(colNames);
+
+        return new UpdatePlan(UpdateMode.MERGE, tbl, colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+            selectQry, isLocSubqry, rowsNum, null);
+    }
+
+    /** */
+    public static UpdatePlan forInsert(GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+        KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry, int rowsNum) {
+        assert !F.isEmpty(colNames);
+
+        return new UpdatePlan(UpdateMode.INSERT, tbl, colNames, keySupplier, valSupplier, keyColIdx, valColIdx, selectQry,
+            isLocSubqry, rowsNum, null);
+    }
+
+    /** */
+    public static UpdatePlan forUpdate(GridH2Table tbl, String[] colNames, KeyValueSupplier valSupplier, int valColIdx,
+        String selectQry) {
+        assert !F.isEmpty(colNames);
+
+        return new UpdatePlan(UpdateMode.UPDATE, tbl, colNames, null, valSupplier, -1, valColIdx, selectQry,
+            false, 0, null);
+    }
+
+    /** */
+    public static UpdatePlan forDelete(GridH2Table tbl, String selectQry) {
+        return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, -1, -1, selectQry, false, 0, null);
+    }
+
+    /** */
+    public static UpdatePlan forFastUpdate(UpdateMode mode, GridH2Table tbl, FastUpdateArguments fastUpdateArgs) {
+        assert mode == UpdateMode.UPDATE || mode == UpdateMode.DELETE;
+
+        return new UpdatePlan(mode, tbl, null, null, null, -1, -1, null, false, 0, fastUpdateArgs);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..15c94c3
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -0,0 +1,502 @@
+/*
+ * 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.h2.dml;
+
+import java.lang.reflect.Constructor;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.h2.DmlStatementsProcessor;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+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.processors.query.h2.sql.DmlAstUtils;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlColumn;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDelete;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlInsert;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlMerge;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuery;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlSelect;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlTable;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlUnion;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlUpdate;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.h2.command.Prepared;
+import org.h2.table.Column;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.KEY_FIELD_NAME;
+import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.VAL_FIELD_NAME;
+
+/**
+ * Logic for building update plans performed by {@link DmlStatementsProcessor}.
+ */
+public final class UpdatePlanBuilder {
+    /** */
+    private UpdatePlanBuilder() {
+        // No-op.
+    }
+
+    /**
+     * Generate SELECT statements to retrieve data for modifications from and find fast UPDATE or DELETE args,
+     * if available.
+     *
+     * @param prepared H2's {@link Prepared}.
+     * @return Update plan.
+     */
+    public static UpdatePlan planForStatement(Prepared prepared,
+        @Nullable Integer errKeysPos) throws IgniteCheckedException {
+        assert !prepared.isQuery();
+
+        GridSqlStatement stmt = new GridSqlQueryParser().parse(prepared);
+
+        if (stmt instanceof GridSqlMerge || stmt instanceof GridSqlInsert)
+            return planForInsert(stmt);
+        else
+            return planForUpdate(stmt, errKeysPos);
+    }
+
+    /**
+     * Prepare update plan for INSERT or MERGE.
+     *
+     * @param stmt INSERT or MERGE statement.
+     * @return Update plan.
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    private static UpdatePlan planForInsert(GridSqlStatement stmt) throws IgniteCheckedException {
+        GridSqlQuery sel;
+
+        GridSqlElement target;
+
+        GridSqlColumn[] cols;
+
+        boolean isTableSubqry;
+
+        int rowsNum;
+
+        GridSqlTable tbl;
+
+        GridH2RowDescriptor desc;
+
+        if (stmt instanceof GridSqlInsert) {
+            GridSqlInsert ins = (GridSqlInsert) stmt;
+            target = ins.into();
+
+            tbl = gridTableForElement(target);
+            desc = tbl.dataTable().rowDescriptor();
+
+            cols = ins.columns();
+            sel = DmlAstUtils.selectForInsertOrMerge(cols, ins.rows(), ins.query(), desc);
+            isTableSubqry = (ins.query() != null);
+            rowsNum = isTableSubqry ? 0 : ins.rows().size();
+        }
+        else if (stmt instanceof GridSqlMerge) {
+            GridSqlMerge merge = (GridSqlMerge) stmt;
+
+            target = merge.into();
+
+            tbl = gridTableForElement(target);
+            desc = tbl.dataTable().rowDescriptor();
+
+            // This check also protects us from attempts to update key or its fields directly -
+            // when no key except cache key can be used, it will serve only for uniqueness checks,
+            // not for updates, and hence will allow putting new pairs only.
+            // We don't quote _key and _val column names on CREATE TABLE, so they are always uppercase here.
+            GridSqlColumn[] keys = merge.keys();
+            if (keys.length != 1 || IgniteH2Indexing.KEY_FIELD_NAME.equals(keys[0].columnName()))
+                throw new CacheException("SQL MERGE does not support arbitrary keys");
+
+            cols = merge.columns();
+            sel = DmlAstUtils.selectForInsertOrMerge(cols, merge.rows(), merge.query(), desc);
+            isTableSubqry = (merge.query() != null);
+            rowsNum = isTableSubqry ? 0 : merge.rows().size();
+        }
+        else throw new IgniteSQLException("Unexpected DML operation [cls=" + stmt.getClass().getName() + ']',
+                IgniteQueryErrorCode.UNEXPECTED_OPERATION);
+
+        // Let's set the flag only for subqueries that have their FROM specified.
+        isTableSubqry = (isTableSubqry && (sel instanceof GridSqlUnion ||
+            (sel instanceof GridSqlSelect && ((GridSqlSelect) sel).from() != null)));
+
+        int keyColIdx = -1;
+        int valColIdx = -1;
+
+        boolean hasKeyProps = false;
+        boolean hasValProps = false;
+
+        if (desc == null)
+            throw new IgniteSQLException("Row descriptor undefined for table '" + tbl.dataTable().getName() + "'",
+                IgniteQueryErrorCode.NULL_TABLE_DESCRIPTOR);
+
+        GridCacheContext<?, ?> cctx = desc.context();
+
+        String[] colNames = new String[cols.length];
+
+        for (int i = 0; i < cols.length; i++) {
+            colNames[i] = cols[i].columnName();
+
+            if (isKeyColumn(cols[i].columnName(), desc)) {
+                keyColIdx = i;
+                continue;
+            }
+
+            if (isValColumn(cols[i].columnName(), desc)) {
+                valColIdx = i;
+                continue;
+            }
+
+            GridQueryProperty prop = desc.type().property(cols[i].columnName());
+
+            assert prop != null : "Property '" + cols[i].columnName() + "' not found.";
+
+            if (prop.key())
+                hasKeyProps = true;
+            else
+                hasValProps = true;
+        }
+
+        KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, true);
+        KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false);
+
+        if (stmt instanceof GridSqlMerge)
+            return UpdatePlan.forMerge(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+                sel.getSQL(), !isTableSubqry, rowsNum);
+        else
+            return UpdatePlan.forInsert(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+                sel.getSQL(), !isTableSubqry, rowsNum);
+    }
+
+    /**
+     * Prepare update plan for UPDATE or DELETE.
+     *
+     * @param stmt UPDATE or DELETE statement.
+     * @param errKeysPos index to inject param for re-run keys at. Null if it's not a re-run plan.
+     * @return Update plan.
+     * @throws IgniteCheckedException if failed.
+     */
+    private static UpdatePlan planForUpdate(GridSqlStatement stmt, @Nullable Integer errKeysPos) throws IgniteCheckedException {
+        GridSqlElement target;
+
+        FastUpdateArguments fastUpdate;
+
+        UpdateMode mode;
+
+        if (stmt instanceof GridSqlUpdate) {
+            // Let's verify that user is not trying to mess with key's columns directly
+            verifyUpdateColumns(stmt);
+
+            GridSqlUpdate update = (GridSqlUpdate) stmt;
+            target = update.target();
+            fastUpdate = DmlAstUtils.getFastUpdateArgs(update);
+            mode = UpdateMode.UPDATE;
+        }
+        else if (stmt instanceof GridSqlDelete) {
+            GridSqlDelete del = (GridSqlDelete) stmt;
+            target = del.from();
+            fastUpdate = DmlAstUtils.getFastDeleteArgs(del);
+            mode = UpdateMode.DELETE;
+        }
+        else
+            throw new IgniteSQLException("Unexpected DML operation [cls=" + stmt.getClass().getName() + ']',
+                IgniteQueryErrorCode.UNEXPECTED_OPERATION);
+
+        GridSqlTable tbl = gridTableForElement(target);
+
+        GridH2Table gridTbl = tbl.dataTable();
+
+        GridH2RowDescriptor desc = gridTbl.rowDescriptor();
+
+        if (desc == null)
+            throw new IgniteSQLException("Row descriptor undefined for table '" + gridTbl.getName() + "'",
+                IgniteQueryErrorCode.NULL_TABLE_DESCRIPTOR);
+
+        if (fastUpdate != null)
+            return UpdatePlan.forFastUpdate(mode, gridTbl, fastUpdate);
+        else {
+            GridSqlSelect sel;
+
+            if (stmt instanceof GridSqlUpdate) {
+                boolean bin = desc.context().binaryMarshaller();
+
+                List<GridSqlColumn> updatedCols = ((GridSqlUpdate) stmt).cols();
+
+                int valColIdx = -1;
+
+                String[] colNames = new String[updatedCols.size()];
+
+                for (int i = 0; i < updatedCols.size(); i++) {
+                    colNames[i] = updatedCols.get(i).columnName();
+
+                    if (isValColumn(colNames[i], desc))
+                        valColIdx = i;
+                }
+
+                boolean hasNewVal = (valColIdx != -1);
+
+                // Statement updates distinct properties if it does not have _val in updated columns list
+                // or if its list of updated columns includes only _val, i.e. is single element.
+                boolean hasProps = !hasNewVal || updatedCols.size() > 1;
+
+                // Index of new _val in results of SELECT
+                if (hasNewVal)
+                    valColIdx += 2;
+
+                int newValColIdx;
+
+                if (!hasProps) // No distinct properties, only whole new value - let's take it
+                    newValColIdx = valColIdx;
+                else if (bin) // We update distinct columns in binary mode - let's choose correct index for the builder
+                    newValColIdx = (hasNewVal ? valColIdx : 1);
+                else // Distinct properties, non binary mode - let's instantiate.
+                    newValColIdx = -1;
+
+                // We want supplier to take present value only in case of binary mode as it will create
+                // whole new object as a result anyway, so we don't need to copy previous property values explicitly.
+                // Otherwise we always want it to instantiate new object whose properties we will later
+                // set to current values.
+                KeyValueSupplier newValSupplier = createSupplier(desc.context(), desc.type(), newValColIdx, hasProps, false);
+
+                sel = DmlAstUtils.selectForUpdate((GridSqlUpdate) stmt, errKeysPos);
+
+                return UpdatePlan.forUpdate(gridTbl, colNames, newValSupplier, valColIdx, sel.getSQL());
+            }
+            else {
+                sel = DmlAstUtils.selectForDelete((GridSqlDelete) stmt, errKeysPos);
+
+                return UpdatePlan.forDelete(gridTbl, sel.getSQL());
+            }
+        }
+    }
+
+    /**
+     * Detect appropriate method of instantiating key or value (take from param, create binary builder,
+     * invoke default ctor, or allocate).
+     *
+     * @param cctx Cache context.
+     * @param desc Table descriptor.
+     * @param colIdx Column index if key or value is present in columns list, {@code -1} if it's not.
+     * @param hasProps Whether column list affects individual properties of key or value.
+     * @param key Whether supplier should be created for key or for value.
+     * @return Closure returning key or value.
+     * @throws IgniteCheckedException
+     */
+    @SuppressWarnings({"ConstantConditions", "unchecked"})
+    private static KeyValueSupplier createSupplier(final GridCacheContext<?, ?> cctx, GridQueryTypeDescriptor desc,
+                                                   final int colIdx, boolean hasProps, final boolean key) throws IgniteCheckedException {
+        final String typeName = key ? desc.keyTypeName() : desc.valueTypeName();
+
+        //Try to find class for the key locally.
+        final Class<?> cls = key ? U.firstNotNull(U.classForName(desc.keyTypeName(), null), desc.keyClass())
+            : desc.valueClass();
+
+        boolean isSqlType = GridQueryProcessor.isSqlType(cls);
+
+        // If we don't need to construct anything from scratch, just return value from array.
+        if (isSqlType || !hasProps || !cctx.binaryMarshaller()) {
+            if (colIdx != -1)
+                return new KeyValueSupplier() {
+                    /** {@inheritDoc} */
+                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                        return arg.get(colIdx);
+                    }
+                };
+            else if (isSqlType)
+                // Non constructable keys and values (SQL types) must be present in the query explicitly.
+                throw new IgniteCheckedException((key ? "Key" : "Value") + " is missing from query");
+        }
+
+        if (cctx.binaryMarshaller()) {
+            if (colIdx != -1) {
+                // If we have key or value explicitly present in query, create new builder upon them...
+                return new KeyValueSupplier() {
+                    /** {@inheritDoc} */
+                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                        BinaryObject bin = cctx.grid().binary().toBinary(arg.get(colIdx));
+
+                        return cctx.grid().binary().builder(bin);
+                    }
+                };
+            }
+            else {
+                // ...and if we don't, just create a new builder.
+                return new KeyValueSupplier() {
+                    /** {@inheritDoc} */
+                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                        return cctx.grid().binary().builder(typeName);
+                    }
+                };
+            }
+        }
+        else {
+            Constructor<?> ctor;
+
+            try {
+                ctor = cls.getDeclaredConstructor();
+                ctor.setAccessible(true);
+            }
+            catch (NoSuchMethodException | SecurityException ignored) {
+                ctor = null;
+            }
+
+            if (ctor != null) {
+                final Constructor<?> ctor0 = ctor;
+
+                // Use default ctor, if it's present...
+                return new KeyValueSupplier() {
+                    /** {@inheritDoc} */
+                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                        try {
+                            return ctor0.newInstance();
+                        }
+                        catch (Exception e) {
+                            throw new IgniteCheckedException("Failed to invoke default ctor for " +
+                                (key ? "key" : "value"), e);
+                        }
+                    }
+                };
+            }
+            else {
+                // ...or allocate new instance with unsafe, if it's not
+                return new KeyValueSupplier() {
+                    /** {@inheritDoc} */
+                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                        try {
+                            return GridUnsafe.allocateInstance(cls);
+                        }
+                        catch (InstantiationException e) {
+                            throw new IgniteCheckedException("Failed to invoke default ctor for " +
+                                (key ? "key" : "value"), e);
+                        }
+                    }
+                };
+            }
+        }
+    }
+
+
+
+    /**
+     * @param target Expression to extract the table from.
+     * @return Back end table for this element.
+     */
+    private static GridSqlTable gridTableForElement(GridSqlElement target) {
+        Set<GridSqlTable> tbls = new HashSet<>();
+
+        DmlAstUtils.collectAllGridTablesInTarget(target, tbls);
+
+        if (tbls.size() != 1)
+            throw new IgniteSQLException("Failed to determine target table", IgniteQueryErrorCode.TABLE_NOT_FOUND);
+
+        return tbls.iterator().next();
+    }
+
+    /**
+     * Check that UPDATE statement affects no key columns.
+     *
+     * @param statement Statement.
+     */
+    private static void verifyUpdateColumns(GridSqlStatement statement) {
+        if (statement == null || !(statement instanceof GridSqlUpdate))
+            return;
+
+        GridSqlUpdate update = (GridSqlUpdate) statement;
+
+        GridSqlElement updTarget = update.target();
+
+        Set<GridSqlTable> tbls = new HashSet<>();
+
+        DmlAstUtils.collectAllGridTablesInTarget(updTarget, tbls);
+
+        if (tbls.size() != 1)
+            throw new IgniteSQLException("Failed to determine target table for UPDATE", IgniteQueryErrorCode.TABLE_NOT_FOUND);
+
+        GridSqlTable tbl = tbls.iterator().next();
+
+        GridH2Table gridTbl = tbl.dataTable();
+
+        if (updateAffectsKeyColumns(gridTbl, update.set().keySet()))
+            throw new IgniteSQLException("SQL UPDATE can't modify key or its fields directly",
+                IgniteQueryErrorCode.KEY_UPDATE);
+    }
+
+    /**
+     * Check if given set of modified columns intersects with the set of SQL properties of the key.
+     *
+     * @param gridTbl Table.
+     * @param affectedColNames Column names.
+     * @return {@code true} if any of given columns corresponds to the key or any of its properties.
+     */
+    private static boolean updateAffectsKeyColumns(GridH2Table gridTbl, Set<String> affectedColNames) {
+        GridH2RowDescriptor desc = gridTbl.rowDescriptor();
+
+        Column[] cols = gridTbl.getColumns();
+
+        // Check "_key" column itself - always has index of 0.
+        if (affectedColNames.contains(cols[0].getName()))
+            return true;
+
+        // Start off from i = 2 to skip indices of 0 an 1 corresponding to key and value respectively.
+        for (int i = 2; i < cols.length; i++)
+            if (affectedColNames.contains(cols[i].getName()) && desc.isColumnKeyProperty(i - 2))
+                return true;
+
+        return false;
+    }
+
+    /**
+     * Check that given column corresponds to the key with respect to case sensitivity, if needed (should be considered
+     * when the schema escapes all identifiers on table creation).
+     * @param colName Column name.
+     * @param desc Row descriptor.
+     * @return {@code true} if column name corresponds to _key with respect to case sensitivity depending on schema.
+     */
+    private static boolean isKeyColumn(String colName, GridH2RowDescriptor desc) {
+        if (desc.quoteAllIdentifiers())
+            return KEY_FIELD_NAME.equals(colName);
+        else
+            return KEY_FIELD_NAME.equalsIgnoreCase(colName);
+    }
+
+    /**
+     * Check that given column corresponds to the key with respect to case sensitivity, if needed (should be considered
+     * when the schema escapes all identifiers on table creation).
+     * @param colName Column name.
+     * @param desc Row descriptor.
+     * @return {@code true} if column name corresponds to _key with respect to case sensitivity depending on schema.
+     */
+    private static boolean isValColumn(String colName, GridH2RowDescriptor desc) {
+        if (desc.quoteAllIdentifiers())
+            return VAL_FIELD_NAME.equals(colName);
+        else
+            return VAL_FIELD_NAME.equalsIgnoreCase(colName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/package-info.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/package-info.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/package-info.java
new file mode 100644
index 0000000..653f1be
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Contains routines for executing DML statements on top of H2.
+ */
+package org.apache.ignite.internal.processors.query.h2.dml;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
index f519c30..3465ed7 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
@@ -107,6 +107,24 @@ public interface GridH2RowDescriptor extends GridOffHeapSmartPointerFactory<Grid
     public Object columnValue(Object key, Object val, int col);
 
     /**
+     * Gets column value by column index.
+     *
+     * @param key Key.
+     * @param val Value.
+     * @param colVal Value to set to column.
+     * @param col Column index.
+     */
+    public void setColumnValue(Object key, Object val, Object colVal, int col);
+
+    /**
+     * Determine whether a column corresponds to a property of key or to one of value.
+     *
+     * @param col Column index.
+     * @return {@code true} if given column corresponds to a key property, {@code false} otherwise
+     */
+    public boolean isColumnKeyProperty(int col);
+
+    /**
      * @return Unsafe memory.
      */
     public GridUnsafeMemory memory();
@@ -145,4 +163,9 @@ public interface GridH2RowDescriptor extends GridOffHeapSmartPointerFactory<Grid
      * @return {@code True} if index should support snapshots.
      */
     public boolean snapshotableIndex();
+
+    /**
+     * @return Escape all identifiers.
+     */
+    public boolean quoteAllIdentifiers();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 33aaf7b..914e0da 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -68,7 +68,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         IndexColumn.mapColumns(cols, tbl);
 
         initBaseIndex(tbl, 0, name, cols,
-            pk ? IndexType.createUnique(false, false) : IndexType.createNonUnique(false, false, false));
+            pk ? IndexType.createPrimaryKey(false, false) : IndexType.createNonUnique(false, false, false));
 
         final GridH2RowDescriptor desc = tbl.rowDescriptor();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
new file mode 100644
index 0000000..39b1b74
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
@@ -0,0 +1,599 @@
+/*
+ * 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.h2.sql;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow;
+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.lang.IgnitePair;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.h2.expression.Expression;
+import org.h2.table.Column;
+import org.h2.table.Table;
+import org.h2.util.IntArray;
+import org.h2.value.DataType;
+import org.h2.value.Value;
+import org.h2.value.ValueDate;
+import org.h2.value.ValueInt;
+import org.h2.value.ValueString;
+import org.h2.value.ValueTime;
+import org.h2.value.ValueTimestamp;
+import org.h2.value.ValueTimestampUtc;
+import org.jetbrains.annotations.Nullable;
+
+import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
+
+/**
+ * AST utils for DML
+ */
+public final class DmlAstUtils {
+    /**
+     * Empty ctor to prevent initialization.
+     */
+    private DmlAstUtils() {
+        // No-op.
+    }
+
+    /**
+     * Create SELECT on which subsequent INSERT or MERGE will be based.
+     *
+     * @param cols Columns to insert values into.
+     * @param rows Rows to create pseudo-SELECT upon.
+     * @param subQry Subquery to use rather than rows.
+     * @param desc Row descriptor.
+     * @return Subquery or pseudo-SELECT to evaluate inserted expressions.
+     */
+    public static GridSqlQuery selectForInsertOrMerge(GridSqlColumn[] cols, List<GridSqlElement[]> rows,
+        GridSqlQuery subQry, GridH2RowDescriptor desc) {
+        if (!F.isEmpty(rows)) {
+            assert !F.isEmpty(cols);
+
+            GridSqlSelect sel = new GridSqlSelect();
+
+            GridSqlFunction from = new GridSqlFunction(GridSqlFunctionType.TABLE);
+
+            sel.from(from);
+
+            GridSqlArray[] args = new GridSqlArray[cols.length];
+
+            for (int i = 0; i < cols.length; i++) {
+                GridSqlArray arr = new GridSqlArray(rows.size());
+
+                String colName = IgniteH2Indexing.escapeName(cols[i].columnName(), desc.quoteAllIdentifiers());
+
+                GridSqlAlias alias = new GridSqlAlias(colName, arr);
+
+                alias.resultType(cols[i].resultType());
+
+                from.addChild(alias);
+
+                args[i] = arr;
+
+                GridSqlColumn newCol = new GridSqlColumn(null, from, colName, "TABLE." + colName);
+
+                newCol.resultType(cols[i].resultType());
+
+                sel.addColumn(newCol, true);
+            }
+
+            for (GridSqlElement[] row : rows) {
+                assert cols.length == row.length;
+
+                for (int i = 0; i < row.length; i++)
+                    args[i].addChild(row[i]);
+            }
+
+            return sel;
+        }
+        else {
+            assert subQry != null;
+
+            return subQry;
+        }
+    }
+
+    /**
+     * Generate SQL SELECT based on DELETE's WHERE, LIMIT, etc.
+     *
+     * @param del Delete statement.
+     * @param keysParamIdx Index for .
+     * @return SELECT statement.
+     */
+    public static GridSqlSelect selectForDelete(GridSqlDelete del, @Nullable Integer keysParamIdx) {
+        GridSqlSelect mapQry = new GridSqlSelect();
+
+        mapQry.from(del.from());
+
+        Set<GridSqlTable> tbls = new HashSet<>();
+
+        collectAllGridTablesInTarget(del.from(), tbls);
+
+        assert tbls.size() == 1 : "Failed to determine target table for DELETE";
+
+        GridSqlTable tbl = tbls.iterator().next();
+
+        GridH2Table gridTbl = tbl.dataTable();
+
+        assert gridTbl != null : "Failed to determine target grid table for DELETE";
+
+        Column h2KeyCol = gridTbl.getColumn(GridH2AbstractKeyValueRow.KEY_COL);
+
+        Column h2ValCol = gridTbl.getColumn(GridH2AbstractKeyValueRow.VAL_COL);
+
+        GridSqlColumn keyCol = new GridSqlColumn(h2KeyCol, tbl, h2KeyCol.getName(), h2KeyCol.getSQL());
+        keyCol.resultType(GridSqlType.fromColumn(h2KeyCol));
+
+        GridSqlColumn valCol = new GridSqlColumn(h2ValCol, tbl, h2ValCol.getName(), h2ValCol.getSQL());
+        valCol.resultType(GridSqlType.fromColumn(h2ValCol));
+
+        mapQry.addColumn(keyCol, true);
+        mapQry.addColumn(valCol, true);
+
+        GridSqlElement where = del.where();
+        if (keysParamIdx != null)
+            where = injectKeysFilterParam(where, keyCol, keysParamIdx);
+
+        mapQry.where(where);
+        mapQry.limit(del.limit());
+
+        return mapQry;
+    }
+
+    /**
+     * @param update UPDATE statement.
+     * @return {@code null} if given statement directly updates {@code _val} column with a literal or param value
+     * and filters by single non expression key (and, optionally,  by single non expression value).
+     */
+    public static FastUpdateArguments getFastUpdateArgs(GridSqlUpdate update) {
+        IgnitePair<GridSqlElement> filter = findKeyValueEqualityCondition(update.where());
+
+        if (filter == null)
+            return null;
+
+        if (update.cols().size() != 1 ||
+            !IgniteH2Indexing.VAL_FIELD_NAME.equalsIgnoreCase(update.cols().get(0).columnName()))
+            return null;
+
+        GridSqlElement set = update.set().get(update.cols().get(0).columnName());
+
+        if (!(set instanceof GridSqlConst || set instanceof GridSqlParameter))
+            return null;
+
+        return new FastUpdateArguments(operandForElement(filter.getKey()), operandForElement(filter.getValue()),
+            operandForElement(set));
+    }
+
+    /**
+     * Create operand based on exact type of SQL element.
+     *
+     * @param el element.
+     * @return Operand.
+     */
+    private static FastUpdateArgument operandForElement(GridSqlElement el) {
+        assert el == null ^ (el instanceof GridSqlConst || el instanceof GridSqlParameter);
+
+        if (el == null)
+            return FastUpdateArguments.NULL_ARGUMENT;
+
+        if (el instanceof GridSqlConst)
+            return new ValueArgument(((GridSqlConst)el).value().getObject());
+        else
+            return new ParamArgument(((GridSqlParameter)el).index());
+    }
+
+    /**
+     * @param del DELETE statement.
+     * @return {@code true} if given statement filters by single non expression key.
+     */
+    public static FastUpdateArguments getFastDeleteArgs(GridSqlDelete del) {
+        IgnitePair<GridSqlElement> filter = findKeyValueEqualityCondition(del.where());
+
+        if (filter == null)
+            return null;
+
+        return new FastUpdateArguments(operandForElement(filter.getKey()), operandForElement(filter.getValue()),
+            FastUpdateArguments.NULL_ARGUMENT);
+    }
+
+    /**
+     * @param where Element to test.
+     * @return Whether given element corresponds to {@code WHERE _key = ?}, and key is a literal expressed
+     * in query or a query param.
+     */
+    private static IgnitePair<GridSqlElement> findKeyValueEqualityCondition(GridSqlElement where) {
+        if (where == null || !(where instanceof GridSqlOperation))
+            return null;
+
+        GridSqlOperation whereOp = (GridSqlOperation) where;
+
+        // Does this WHERE limit only by _key?
+        if (isKeyEqualityCondition(whereOp))
+            return new IgnitePair<>(whereOp.child(1), null);
+
+        // Or maybe it limits both by _key and _val?
+        if (whereOp.operationType() != GridSqlOperationType.AND)
+            return null;
+
+        GridSqlElement left = whereOp.child(0);
+
+        GridSqlElement right = whereOp.child(1);
+
+        if (!(left instanceof GridSqlOperation && right instanceof GridSqlOperation))
+            return null;
+
+        GridSqlOperation leftOp = (GridSqlOperation) left;
+
+        GridSqlOperation rightOp = (GridSqlOperation) right;
+
+        if (isKeyEqualityCondition(leftOp)) { // _key = ? and _val = ?
+            if (!isValueEqualityCondition(rightOp))
+                return null;
+
+            return new IgnitePair<>(leftOp.child(1), rightOp.child(1));
+        }
+        else if (isKeyEqualityCondition(rightOp)) { // _val = ? and _key = ?
+            if (!isValueEqualityCondition(leftOp))
+                return null;
+
+            return new IgnitePair<>(rightOp.child(1), leftOp.child(1));
+        }
+        else // Neither
+            return null;
+    }
+
+    /**
+     * @param op Operation.
+     * @param colName Column name to check.
+     * @return Whether this condition is of form {@code colName} = ?
+     */
+    private static boolean isEqualityCondition(GridSqlOperation op, String colName) {
+        if (op.operationType() != GridSqlOperationType.EQUAL)
+            return false;
+
+        GridSqlElement left = op.child(0);
+        GridSqlElement right = op.child(1);
+
+        return left instanceof GridSqlColumn &&
+            colName.equalsIgnoreCase(((GridSqlColumn) left).columnName()) &&
+            (right instanceof GridSqlConst || right instanceof GridSqlParameter);
+    }
+
+    /**
+     * @param op Operation.
+     * @return Whether this condition is of form _key = ?
+     */
+    private static boolean isKeyEqualityCondition(GridSqlOperation op) {
+        return isEqualityCondition(op, IgniteH2Indexing.KEY_FIELD_NAME);
+    }
+
+    /**
+     * @param op Operation.
+     * @return Whether this condition is of form _val = ?
+     */
+    private static boolean isValueEqualityCondition(GridSqlOperation op) {
+        return isEqualityCondition(op, IgniteH2Indexing.VAL_FIELD_NAME);
+    }
+
+
+    /**
+     * Generate SQL SELECT based on UPDATE's WHERE, LIMIT, etc.
+     *
+     * @param update Update statement.
+     * @param keysParamIdx Index of new param for the array of keys.
+     * @return SELECT statement.
+     */
+    public static GridSqlSelect selectForUpdate(GridSqlUpdate update, @Nullable Integer keysParamIdx) {
+        GridSqlSelect mapQry = new GridSqlSelect();
+
+        mapQry.from(update.target());
+
+        Set<GridSqlTable> tbls = new HashSet<>();
+
+        collectAllGridTablesInTarget(update.target(), tbls);
+
+        assert tbls.size() == 1 : "Failed to determine target table for UPDATE";
+
+        GridSqlTable tbl = tbls.iterator().next();
+
+        GridH2Table gridTbl = tbl.dataTable();
+
+        assert gridTbl != null : "Failed to determine target grid table for UPDATE";
+
+        Column h2KeyCol = gridTbl.getColumn(GridH2AbstractKeyValueRow.KEY_COL);
+
+        Column h2ValCol = gridTbl.getColumn(GridH2AbstractKeyValueRow.VAL_COL);
+
+        GridSqlColumn keyCol = new GridSqlColumn(h2KeyCol, tbl, h2KeyCol.getName(), h2KeyCol.getSQL());
+        keyCol.resultType(GridSqlType.fromColumn(h2KeyCol));
+
+        GridSqlColumn valCol = new GridSqlColumn(h2ValCol, tbl, h2ValCol.getName(), h2ValCol.getSQL());
+        valCol.resultType(GridSqlType.fromColumn(h2ValCol));
+
+        mapQry.addColumn(keyCol, true);
+        mapQry.addColumn(valCol, true);
+
+        for (GridSqlColumn c : update.cols()) {
+            String newColName = "_upd_" + c.columnName();
+            // We have to use aliases to cover cases when the user
+            // wants to update _val field directly (if it's a literal)
+            GridSqlAlias alias = new GridSqlAlias(newColName, elementOrDefault(update.set().get(c.columnName()), c), true);
+            alias.resultType(c.resultType());
+            mapQry.addColumn(alias, true);
+        }
+
+        GridSqlElement where = update.where();
+        if (keysParamIdx != null)
+            where = injectKeysFilterParam(where, keyCol, keysParamIdx);
+
+        mapQry.where(where);
+        mapQry.limit(update.limit());
+
+        return mapQry;
+    }
+
+    /**
+     * Do what we can to compute default value for this column (mimics H2 behavior).
+     * @see Table#getDefaultValue
+     * @see Column#validateConvertUpdateSequence
+     * @param el SQL element.
+     * @param col Column.
+     * @return {@link GridSqlConst#NULL}, if {@code el} is null, or {@code el} if
+     * it's not {@link GridSqlKeyword#DEFAULT}, or computed default value.
+     */
+    private static GridSqlElement elementOrDefault(GridSqlElement el, GridSqlColumn col) {
+        if (el == null)
+            return GridSqlConst.NULL;
+
+        if (el != GridSqlKeyword.DEFAULT)
+            return el;
+
+        Column h2Col = col.column();
+
+        Expression dfltExpr = h2Col.getDefaultExpression();
+
+        Value dfltVal;
+
+        try {
+            dfltVal = dfltExpr != null ? dfltExpr.getValue(null) : null;
+        }
+        catch (Exception e) {
+            throw new IgniteSQLException("Failed to evaluate default value for a column " + col.columnName());
+        }
+
+        if (dfltVal != null)
+            return new GridSqlConst(dfltVal);
+
+        int type = h2Col.getType();
+
+        DataType dt = DataType.getDataType(type);
+
+        if (dt.decimal)
+            dfltVal = ValueInt.get(0).convertTo(type);
+        else if (dt.type == Value.TIMESTAMP)
+            dfltVal = ValueTimestamp.fromMillis(U.currentTimeMillis());
+        else if (dt.type == Value.TIMESTAMP_UTC)
+            dfltVal = ValueTimestampUtc.fromMillis(U.currentTimeMillis());
+        else if (dt.type == Value.TIME)
+            dfltVal = ValueTime.fromNanos(0);
+        else if (dt.type == Value.DATE)
+            dfltVal = ValueDate.fromMillis(U.currentTimeMillis());
+        else
+            dfltVal = ValueString.get("").convertTo(type);
+
+        return new GridSqlConst(dfltVal);
+    }
+
+    /**
+     * Append additional condition to WHERE for it to select only specific keys.
+     *
+     * @param where Initial condition.
+     * @param keyCol Column to base the new condition on.
+     * @return New condition.
+     */
+    private static GridSqlElement injectKeysFilterParam(GridSqlElement where, GridSqlColumn keyCol, int paramIdx) {
+        GridSqlElement e = new GridSqlOperation(GridSqlOperationType.IN, keyCol, new GridSqlParameter(paramIdx));
+
+        if (where == null)
+            return e;
+        else
+            return new GridSqlOperation(GridSqlOperationType.AND, where, e);
+    }
+
+    /**
+     * @param qry Select.
+     * @param params Parameters.
+     * @param target Extracted parameters.
+     * @param paramIdxs Parameter indexes.
+     * @return Extracted parameters list.
+     */
+    private static List<Object> findParams(GridSqlQuery qry, Object[] params, ArrayList<Object> target,
+                                           IntArray paramIdxs) {
+        if (qry instanceof GridSqlSelect)
+            return findParams((GridSqlSelect)qry, params, target, paramIdxs);
+
+        GridSqlUnion union = (GridSqlUnion)qry;
+
+        findParams(union.left(), params, target, paramIdxs);
+        findParams(union.right(), params, target, paramIdxs);
+
+        findParams(qry.limit(), params, target, paramIdxs);
+        findParams(qry.offset(), params, target, paramIdxs);
+
+        return target;
+    }
+
+    /**
+     * @param qry Select.
+     * @param params Parameters.
+     * @param target Extracted parameters.
+     * @param paramIdxs Parameter indexes.
+     * @return Extracted parameters list.
+     */
+    private static List<Object> findParams(GridSqlSelect qry, Object[] params, ArrayList<Object> target,
+                                           IntArray paramIdxs) {
+        if (params.length == 0)
+            return target;
+
+        for (GridSqlElement el : qry.columns(false))
+            findParams(el, params, target, paramIdxs);
+
+        findParams(qry.from(), params, target, paramIdxs);
+        findParams(qry.where(), params, target, paramIdxs);
+
+        // Don't search in GROUP BY and HAVING since they expected to be in select list.
+
+        findParams(qry.limit(), params, target, paramIdxs);
+        findParams(qry.offset(), params, target, paramIdxs);
+
+        return target;
+    }
+
+    /**
+     * @param el Element.
+     * @param params Parameters.
+     * @param target Extracted parameters.
+     * @param paramIdxs Parameter indexes.
+     */
+    private static void findParams(@Nullable GridSqlElement el, Object[] params, ArrayList<Object> target,
+                                   IntArray paramIdxs) {
+        if (el == null)
+            return;
+
+        if (el instanceof GridSqlParameter) {
+            // H2 Supports queries like "select ?5" but first 4 non-existing parameters are need to be set to any value.
+            // Here we will set them to NULL.
+            final int idx = ((GridSqlParameter)el).index();
+
+            while (target.size() < idx)
+                target.add(null);
+
+            if (params.length <= idx)
+                throw new IgniteException("Invalid number of query parameters. " +
+                    "Cannot find " + idx + " parameter.");
+
+            Object param = params[idx];
+
+            if (idx == target.size())
+                target.add(param);
+            else
+                target.set(idx, param);
+
+            paramIdxs.add(idx);
+        }
+        else if (el instanceof GridSqlSubquery)
+            findParams(((GridSqlSubquery)el).select(), params, target, paramIdxs);
+        else
+            for (GridSqlElement child : el)
+                findParams(child, params, target, paramIdxs);
+    }
+
+    /**
+     * Processes all the tables and subqueries using the given closure.
+     *
+     * @param from FROM element.
+     * @param c Closure each found table and subquery will be passed to. If returns {@code true} the we need to stop.
+     * @return {@code true} If we have found.
+     */
+    private static boolean findTablesInFrom(GridSqlElement from, IgnitePredicate<GridSqlElement> c) {
+        if (from == null)
+            return false;
+
+        if (from instanceof GridSqlTable || from instanceof GridSqlSubquery)
+            return c.apply(from);
+
+        if (from instanceof GridSqlJoin) {
+            // Left and right.
+            if (findTablesInFrom(from.child(0), c))
+                return true;
+
+            if (findTablesInFrom(from.child(1), c))
+                return true;
+
+            // We don't process ON condition because it is not a joining part of from here.
+            return false;
+        }
+        else if (from instanceof GridSqlAlias)
+            return findTablesInFrom(from.child(), c);
+        else if (from instanceof GridSqlFunction)
+            return false;
+
+        throw new IllegalStateException(from.getClass().getName() + " : " + from.getSQL());
+    }
+
+    /**
+     * @param from From element.
+     * @param tbls Tables.
+     */
+    public static void collectAllGridTablesInTarget(GridSqlElement from, final Set<GridSqlTable> tbls) {
+        findTablesInFrom(from, new IgnitePredicate<GridSqlElement>() {
+            @Override public boolean apply(GridSqlElement el) {
+                if (el instanceof GridSqlTable)
+                    tbls.add((GridSqlTable)el);
+
+                return false;
+            }
+        });
+    }
+
+    /** Simple constant value based operand. */
+    private final static class ValueArgument implements FastUpdateArgument {
+        /** Value to return. */
+        private final Object val;
+
+        /** */
+        private ValueArgument(Object val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
+            return val;
+        }
+    }
+
+    /** Simple constant value based operand. */
+    private final static class ParamArgument implements FastUpdateArgument {
+        /** Value to return. */
+        private final int paramIdx;
+
+        /** */
+        private ParamArgument(int paramIdx) {
+            assert paramIdx >= 0;
+
+            this.paramIdx = paramIdx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
+            assert arg.length > paramIdx;
+
+            return arg[paramIdx];
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
index 69e98bf..caa2000 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 import org.h2.util.StatementBuilder;
 
 /**
@@ -32,6 +33,13 @@ public class GridSqlArray extends GridSqlElement {
         super(size == 0 ? Collections.<GridSqlElement>emptyList() : new ArrayList<GridSqlElement>(size));
     }
 
+    /**
+     * @param children Initial child list.
+     */
+    public GridSqlArray(List<GridSqlElement> children) {
+        super(children);
+    }
+
     /** {@inheritDoc} */
     @Override public String getSQL() {
         if (size() == 0)

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
index 976eb2c..bb7d338 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
@@ -18,14 +18,20 @@
 package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.util.Collections;
+import org.h2.expression.ValueExpression;
 import org.h2.value.Value;
 import org.h2.value.ValueBoolean;
+import org.h2.value.ValueNull;
 
 /**
  * Constant value.
  */
 public class GridSqlConst extends GridSqlElement implements GridSqlValue {
     /** */
+    public static final GridSqlElement NULL = new GridSqlConst(ValueNull.INSTANCE)
+        .resultType(GridSqlType.fromExpression(ValueExpression.getNull()));
+
+    /** */
     public static final GridSqlElement TRUE = new GridSqlConst(ValueBoolean.get(true))
         .resultType(GridSqlType.BOOLEAN);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDelete.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDelete.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDelete.java
new file mode 100644
index 0000000..225de58
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDelete.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.query.h2.sql;
+
+import org.h2.util.StatementBuilder;
+import org.h2.util.StringUtils;
+
+/** */
+public class GridSqlDelete extends GridSqlStatement {
+    /** */
+    private GridSqlElement from;
+
+    /** */
+    private GridSqlElement where;
+
+    /** */
+    public GridSqlDelete from(GridSqlElement from) {
+        this.from = from;
+        return this;
+    }
+
+    /** */
+    public GridSqlElement from() {
+        return from;
+    }
+
+    /** */
+    public GridSqlDelete where(GridSqlElement where) {
+        this.where = where;
+        return this;
+    }
+
+    /** */
+    public GridSqlElement where() {
+        return where;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StatementBuilder buff = new StatementBuilder(explain() ? "EXPLAIN " : "");
+        buff.append("DELETE")
+            .append("\nFROM ")
+            .append(from.getSQL());
+
+        if (where != null)
+            buff.append("\nWHERE ").append(StringUtils.unEnclose(where.getSQL()));
+
+        if (limit != null)
+            buff.append("\nLIMIT (").append(StringUtils.unEnclose(limit.getSQL())).append(')');
+
+        return buff.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
index 7cd7a6b..4e93196 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
@@ -94,7 +94,10 @@ public class GridSqlFunction extends GridSqlElement {
         if (schema != null)
             buff.append(Parser.quoteIdentifier(schema)).append('.');
 
-        buff.append(Parser.quoteIdentifier(name));
+        // We don't need to quote identifier as long as H2 never does so with function names when generating plan SQL.
+        // On the other hand, quoting identifiers that also serve as keywords (like CURRENT_DATE() and CURRENT_DATE)
+        // turns CURRENT_DATE() into "CURRENT_DATE"(), which is not good.
+        buff.append(name);
 
         if (type == CASE) {
             buff.append(' ').append(child().getSQL());

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlInsert.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlInsert.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlInsert.java
new file mode 100644
index 0000000..07b36e6
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlInsert.java
@@ -0,0 +1,149 @@
+/*
+ * 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.h2.sql;
+
+import java.util.List;
+import org.h2.util.StatementBuilder;
+
+/** */
+public class GridSqlInsert extends GridSqlStatement {
+    /** */
+    private GridSqlElement into;
+
+    /** */
+    private GridSqlColumn[] cols;
+
+    /** */
+    private List<GridSqlElement[]> rows;
+
+    /** Insert subquery. */
+    private GridSqlQuery qry;
+
+    /**
+     * Not supported, introduced for clarity and correct SQL generation.
+     * @see org.h2.command.dml.Insert#insertFromSelect
+     */
+    private boolean direct;
+
+    /**
+     * Not supported, introduced for clarity and correct SQL generation.
+     * @see org.h2.command.dml.Insert#sortedInsertMode
+     */
+    private boolean sorted;
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StatementBuilder buff = new StatementBuilder(explain() ? "EXPLAIN " : "");
+        buff.append("INSERT")
+            .append("\nINTO ")
+            .append(into.getSQL())
+            .append('(');
+
+        for (GridSqlColumn col : cols) {
+            buff.appendExceptFirst(", ");
+            buff.append('\n')
+                .append(col.getSQL());
+        }
+        buff.append("\n)\n");
+
+        if (direct)
+            buff.append("DIRECT ");
+
+        if (sorted)
+            buff.append("SORTED ");
+
+        if (!rows.isEmpty()) {
+            buff.append("VALUES\n");
+            StatementBuilder valuesBuff = new StatementBuilder();
+
+            for (GridSqlElement[] row : rows()) {
+                valuesBuff.appendExceptFirst(",\n");
+                StatementBuilder rowBuff = new StatementBuilder("(");
+                for (GridSqlElement e : row) {
+                    rowBuff.appendExceptFirst(", ");
+                    rowBuff.append(e != null ? e.getSQL() : "DEFAULT");
+                }
+                rowBuff.append(')');
+                valuesBuff.append(rowBuff.toString());
+            }
+            buff.append(valuesBuff.toString());
+        }
+        else
+            buff.append('\n')
+                .append(qry.getSQL());
+
+        return buff.toString();
+    }
+
+    /** */
+    public GridSqlElement into() {
+        return into;
+    }
+
+    /** */
+    public GridSqlInsert into(GridSqlElement from) {
+        this.into = from;
+        return this;
+    }
+
+    /** */
+    public List<GridSqlElement[]> rows() {
+        return rows;
+    }
+
+    /** */
+    public GridSqlInsert rows(List<GridSqlElement[]> rows) {
+        assert rows != null;
+        this.rows = rows;
+        return this;
+    }
+
+    /** */
+    public GridSqlQuery query() {
+        return qry;
+    }
+
+    /** */
+    public GridSqlInsert query(GridSqlQuery qry) {
+        this.qry = qry;
+        return this;
+    }
+
+    /** */
+    public GridSqlColumn[] columns() {
+        return cols;
+    }
+
+    /** */
+    public GridSqlInsert columns(GridSqlColumn[] cols) {
+        this.cols = cols;
+        return this;
+    }
+
+    /** */
+    public GridSqlInsert direct(boolean direct) {
+        this.direct = direct;
+        return this;
+    }
+
+    /** */
+    public GridSqlInsert sorted(boolean sorted) {
+        this.sorted = sorted;
+        return this;
+    }
+}


[48/52] ignite git commit: IGNITE-4347: ODBC: Fixed NPE in OdbcRequestHandler appearing when cache doesn't exist. This closes #1305.

Posted by vo...@apache.org.
IGNITE-4347: ODBC: Fixed NPE in OdbcRequestHandler appearing when cache doesn't exist. This closes #1305.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/af24a9e2
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/af24a9e2
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/af24a9e2

Branch: refs/heads/master
Commit: af24a9e2c3b61dff7c9116f1a3bea8c398b7ce78
Parents: 4ba8fcd
Author: isapego <ig...@gmail.com>
Authored: Fri Dec 2 12:35:13 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 2 12:35:13 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/odbc/OdbcRequestHandler.java | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/af24a9e2/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index 32fb9db..ea3ae8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
@@ -199,13 +198,13 @@ public class OdbcRequestHandler {
             qry.setDistributedJoins(distributedJoins);
             qry.setEnforceJoinOrder(enforceJoinOrder);
 
-            IgniteCache<Object, Object> cache = ctx.grid().cache(req.cacheName()).withKeepBinary();
+            IgniteCache<Object, Object> cache = ctx.grid().cache(req.cacheName());
 
             if (cache == null)
-                return new OdbcResponse(OdbcResponse.STATUS_FAILED,
-                    "Cache doesn't exist (did you configure it?): " + req.cacheName());
+                return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Cache doesn't exist (did you configure it?): " +
+                    req.cacheName());
 
-            QueryCursor qryCur = cache.query(qry);
+            QueryCursor qryCur = cache.withKeepBinary().query(qry);
 
             qryCursors.put(qryId, new IgniteBiTuple<QueryCursor, Iterator>(qryCur, null));
 


[34/52] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.8' into ignite-1.8

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.8' into ignite-1.8


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ac6d9105
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ac6d9105
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ac6d9105

Branch: refs/heads/master
Commit: ac6d910576f2b51298ab171a9ad87e71e0ade532
Parents: e7c0d45 4a4082a
Author: devozerov <vo...@gridgain.com>
Authored: Fri Nov 25 19:08:14 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 19:08:14 2016 +0300

----------------------------------------------------------------------
 .../dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java  | 5 ++++-
 .../dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec  | 4 ++--
 .../Datagrid/MultiTieredCacheExample.cs                      | 8 +++++---
 3 files changed, 11 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[16/52] ignite git commit: IGNITE-4299: ODBC Minor example fix. This closes #1277.

Posted by vo...@apache.org.
IGNITE-4299: ODBC Minor example fix. This closes #1277.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d5dc5815
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d5dc5815
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d5dc5815

Branch: refs/heads/master
Commit: d5dc5815416cc13f7b2b8a323fa5a39ec95ecb95
Parents: 4a758d1
Author: isapego <ig...@gmail.com>
Authored: Thu Nov 24 15:53:35 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Nov 24 15:53:35 2016 +0300

----------------------------------------------------------------------
 .../cpp/examples/putget-example/src/putget_example.cpp       | 2 +-
 .../cpp/examples/query-example/src/query_example.cpp         | 8 +-------
 2 files changed, 2 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d5dc5815/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp b/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
index 0b57886..8bf9c8c 100644
--- a/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
+++ b/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
@@ -117,7 +117,7 @@ int main()
     }
 
     std::cout << std::endl;
-    std::cout << ">>> Example finished, press any key to exit ..." << std::endl;
+    std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl;
     std::cout << std::endl;
 
     std::cin.get();

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5dc5815/modules/platforms/cpp/examples/query-example/src/query_example.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/query-example/src/query_example.cpp b/modules/platforms/cpp/examples/query-example/src/query_example.cpp
index 8b8d008..9524100 100644
--- a/modules/platforms/cpp/examples/query-example/src/query_example.cpp
+++ b/modules/platforms/cpp/examples/query-example/src/query_example.cpp
@@ -416,12 +416,6 @@ int main()
         // Populate cache.
         Initialize();
 
-        std::cout << std::endl;
-        std::cout << ">>> Ready" << std::endl;
-        std::cout << std::endl;
-
-        std::cin.get();
-
         // Example for SCAN-based query based on a predicate.
         DoScanQuery();
 
@@ -456,7 +450,7 @@ int main()
     }
 
     std::cout << std::endl;
-    std::cout << ">>> Example finished, press any key to exit ..." << std::endl;
+    std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl;
     std::cout << std::endl;
 
     std::cin.get();


[17/52] ignite git commit: IGNITE-4297 .NET: Update NuGet scripts to clean up jars in bin folder

Posted by vo...@apache.org.
IGNITE-4297 .NET: Update NuGet scripts to clean up jars in bin folder


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/c4badeef
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c4badeef
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c4badeef

Branch: refs/heads/master
Commit: c4badeef809851c1ab890e2d6358834f698e94ad
Parents: d5dc581
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Nov 24 16:32:02 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Nov 24 16:33:45 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core/NuGet/Install.ps1       | 14 ++++++++++++++
 .../dotnet/Apache.Ignite.Core/NuGet/Uninstall.ps1     | 14 ++++++++++++++
 2 files changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c4badeef/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/Install.ps1
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/Install.ps1 b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/Install.ps1
index d4e97bb..e7cdad4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/Install.ps1
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/Install.ps1
@@ -24,4 +24,18 @@ if (!$currentPostBuildCmd.Contains($IgnitePostBuildCmd)) {
     $project.Properties.Item("PostBuildEvent").Value += $IgnitePostBuildCmd
 }
 
+# Save
+$project.Save()
+
+# Remove bin\Libs folders with old jars
+$project.ConfigurationManager | % { 
+    $projPath = $project.Properties.Item("FullPath").Value
+    $binDir = ($_.Properties | Where Name -match OutputPath).Value
+
+    $binPath = Join-Path $projPath $binDir
+    $libsPath = Join-Path $binPath "Libs"
+
+    Remove-Item -Force -Recurse -ErrorAction SilentlyContinue $libsPath
+}
+
 Write-Host "Welcome to Apache Ignite.NET!"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4badeef/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/Uninstall.ps1
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/Uninstall.ps1 b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/Uninstall.ps1
index f81f41c..634c78e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/Uninstall.ps1
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/Uninstall.ps1
@@ -19,3 +19,17 @@ $currentPostBuildCmd = $project.Properties.Item("PostBuildEvent").Value
 
 # Remove our post build command from it (if it's there)
 $project.Properties.Item("PostBuildEvent").Value = $currentPostBuildCmd.Replace($IgnitePostBuildCmd, "")
+
+# Save
+$project.Save()
+
+# Remove bin\Libs folders with jars
+$project.ConfigurationManager | % { 
+    $projPath = $project.Properties.Item("FullPath").Value
+    $binDir = ($_.Properties | Where Name -match OutputPath).Value
+
+    $binPath = Join-Path $projPath $binDir
+    $libsPath = Join-Path $binPath "Libs"
+
+    Remove-Item -Force -Recurse -ErrorAction SilentlyContinue $libsPath
+}
\ No newline at end of file


[28/52] ignite git commit: Release notes template.

Posted by vo...@apache.org.
Release notes template.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a6cb18a9
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a6cb18a9
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a6cb18a9

Branch: refs/heads/master
Commit: a6cb18a9375f3f0b9c6cb7def0380999a78138c5
Parents: 7b8d530
Author: devozerov <vo...@gridgain.com>
Authored: Fri Nov 25 15:07:30 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 15:07:30 2016 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a6cb18a9/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index d8a13e7..5f5d190 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -1,6 +1,10 @@
 Apache Ignite Release Notes
 ===========================
 
+Apache Ignite In-Memory Data Fabric 1.8
+---------------------------------------
+TBD
+
 Apache Ignite In-Memory Data Fabric 1.7
 ---------------------------------------
 Ignite:


[41/52] ignite git commit: IGNITE-4243 Fixed link in Redis examples.

Posted by vo...@apache.org.
IGNITE-4243 Fixed link in Redis examples.

(cherry picked from commit 7af7a99)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a53fd38c
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a53fd38c
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a53fd38c

Branch: refs/heads/master
Commit: a53fd38c0bdb65c454affa947b2edb7b694973c4
Parents: 1073b2c
Author: Andrey Novikov <an...@gridgain.com>
Authored: Mon Nov 28 17:03:52 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Mon Nov 28 17:56:50 2016 +0700

----------------------------------------------------------------------
 examples/redis/redis-example.php | 2 +-
 examples/redis/redis-example.py  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a53fd38c/examples/redis/redis-example.php
----------------------------------------------------------------------
diff --git a/examples/redis/redis-example.php b/examples/redis/redis-example.php
index 0054f16..f015b5c 100644
--- a/examples/redis/redis-example.php
+++ b/examples/redis/redis-example.php
@@ -22,7 +22,7 @@
  * To execute this script, you need to have Predis extension installed and Ignite running.
  * See https://github.com/nrk/predis for Predis details.
  *
- * See https://apacheignite.readme.io/ for more details on Redis integration.
+ * See https://apacheignite.readme.io/docs/redis for more details on Redis integration.
  */
 
 // Load the library.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a53fd38c/examples/redis/redis-example.py
----------------------------------------------------------------------
diff --git a/examples/redis/redis-example.py b/examples/redis/redis-example.py
index 15b847b..f6c4f36 100644
--- a/examples/redis/redis-example.py
+++ b/examples/redis/redis-example.py
@@ -20,7 +20,7 @@ import redis
 To execute this, you will have redis-py installed and Ignite running.
 See https://github.com/andymccurdy/redis-py for the details on redis-py.
 
-See https://apacheignite.readme.io/ for more details on Redis integration.
+See https://apacheignite.readme.io/docs/redis for more details on Redis integration.
 '''
 
 r = redis.StrictRedis(host='localhost', port=11211, db=0)


[23/52] ignite git commit: IGNITE-2294: Fixed incorrect merge of BinaryObjectOffheapImpl.typeId().

Posted by vo...@apache.org.
IGNITE-2294: Fixed incorrect merge of BinaryObjectOffheapImpl.typeId().


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/0822dc23
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/0822dc23
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/0822dc23

Branch: refs/heads/master
Commit: 0822dc2370e75427078875bcd67051eb9cd44779
Parents: 7b35ba7
Author: devozerov <vo...@gridgain.com>
Authored: Fri Nov 25 12:38:13 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 12:38:13 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0822dc23/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
index 354ac11..f9a31db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
@@ -96,7 +96,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
         if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) {
             int off = start + GridBinaryMarshaller.DFLT_HDR_LEN;
 
-            String clsName = BinaryUtils.doReadClassName(new BinaryOffheapInputStream(off, size));
+            String clsName = BinaryUtils.doReadClassName(new BinaryOffheapInputStream(ptr + off, size));
 
             typeId = ctx.typeId(clsName);
         }


[30/52] ignite git commit: .NET: Update ASP.NET nuget package info

Posted by vo...@apache.org.
.NET: Update ASP.NET nuget package info


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/2fccde72
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/2fccde72
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/2fccde72

Branch: refs/heads/master
Commit: 2fccde7227704cb04abb310b650ac7060db726c0
Parents: dda4fc9
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Nov 25 16:14:43 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Nov 25 16:14:43 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2fccde72/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec
index 3b3b5ac..49f2fe8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec
@@ -39,8 +39,8 @@ Creating NuGet package:
         <iconUrl>https://ignite.apache.org/images/logo_ignite_32_32.png</iconUrl>
         <requireLicenseAcceptance>false</requireLicenseAcceptance>
         <description>
-Apache Ignite ASP.NET Output Cache Provider: caches page output in a distributed in-memory cache.
-Ideal for web farms: share cached data between servers effortlessly.
+Output Cache Provider: caches page output in a distributed in-memory cache.
+Session State Store Provider: stores session state data in a distributed in-memory cache.
             
 More info: https://apacheignite-net.readme.io/
         </description>


[38/52] ignite git commit: IGNITE-4320: Minor fixes inside DML engine. This closes #1292.

Posted by vo...@apache.org.
IGNITE-4320: Minor fixes inside DML engine. This closes #1292.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b0127d3c
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b0127d3c
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b0127d3c

Branch: refs/heads/master
Commit: b0127d3c57601c475d186dd3becbc17829a657ce
Parents: 0b7c62d
Author: Alexander Paschenko <al...@gmail.com>
Authored: Mon Nov 28 13:24:41 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Nov 28 13:24:41 2016 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        | 143 +++++++++----------
 .../query/h2/dml/UpdatePlanBuilder.java         |  16 +--
 2 files changed, 74 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b0127d3c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 469e36c..d57f95f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
@@ -464,22 +465,6 @@ public class DmlStatementsProcessor {
 
         long res = 0;
 
-        CacheOperationContext opCtx = cctx.operationContextPerCall();
-
-        // Force keepBinary for operation context to avoid binary deserialization inside entry processor
-        if (cctx.binaryMarshaller()) {
-            CacheOperationContext newOpCtx = null;
-
-            if (opCtx == null)
-                // Mimics behavior of GridCacheAdapter#keepBinary and GridCacheProxyImpl#keepBinary
-                newOpCtx = new CacheOperationContext(false, null, true, null, false, null);
-            else if (!opCtx.isKeepBinary())
-                newOpCtx = opCtx.keepBinary();
-
-            if (newOpCtx != null)
-                cctx.operationContextPerCall(newOpCtx);
-        }
-
         Map<Object, EntryProcessor<Object, Object, Boolean>> rows = new LinkedHashMap<>();
 
         // Keys that failed to UPDATE due to concurrent updates.
@@ -487,97 +472,101 @@ public class DmlStatementsProcessor {
 
         SQLException resEx = null;
 
-        try {
-            Iterator<List<?>> it = cursor.iterator();
+        Iterator<List<?>> it = cursor.iterator();
 
-            while (it.hasNext()) {
-                List<?> e = it.next();
-                Object key = e.get(0);
-                Object val = (hasNewVal ? e.get(valColIdx) : e.get(1));
+        while (it.hasNext()) {
+            List<?> e = it.next();
+            Object key = e.get(0);
+            Object val = (hasNewVal ? e.get(valColIdx) : e.get(1));
 
-                Object newVal;
+            Object newVal;
 
-                Map<String, Object> newColVals = new HashMap<>();
+            Map<String, Object> newColVals = new HashMap<>();
 
-                for (int i = 0; i < plan.colNames.length; i++) {
-                    if (hasNewVal && i == valColIdx - 2)
-                        continue;
+            for (int i = 0; i < plan.colNames.length; i++) {
+                if (hasNewVal && i == valColIdx - 2)
+                    continue;
 
-                    newColVals.put(plan.colNames[i], e.get(i + 2));
-                }
+                newColVals.put(plan.colNames[i], e.get(i + 2));
+            }
 
-                newVal = plan.valSupplier.apply(e);
+            newVal = plan.valSupplier.apply(e);
 
-                if (bin && !(val instanceof BinaryObject))
-                    val = cctx.grid().binary().toBinary(val);
+            if (bin && !(val instanceof BinaryObject))
+                val = cctx.grid().binary().toBinary(val);
 
-                // Skip key and value - that's why we start off with 2nd column
-                for (int i = 0; i < plan.tbl.getColumns().length - 2; i++) {
-                    Column c = plan.tbl.getColumn(i + 2);
+            // Skip key and value - that's why we start off with 2nd column
+            for (int i = 0; i < plan.tbl.getColumns().length - 2; i++) {
+                Column c = plan.tbl.getColumn(i + 2);
 
-                    boolean hasNewColVal = newColVals.containsKey(c.getName());
+                GridQueryProperty prop = desc.type().property(c.getName());
 
-                    // Binary objects get old field values from the Builder, so we can skip what we're not updating
-                    if (bin && !hasNewColVal)
-                        continue;
+                if (prop.key())
+                    continue; // Don't get values of key's columns - we won't use them anyway
 
-                    Object colVal = hasNewColVal ? newColVals.get(c.getName()) : desc.columnValue(key, val, i);
+                boolean hasNewColVal = newColVals.containsKey(c.getName());
 
-                    desc.setColumnValue(key, newVal, colVal, i);
-                }
+                // Binary objects get old field values from the Builder, so we can skip what we're not updating
+                if (bin && !hasNewColVal)
+                    continue;
 
-                if (bin && hasProps) {
-                    assert newVal instanceof BinaryObjectBuilder;
+                // Column values that have been explicitly specified have priority over field values in old or new _val
+                // If no value given for the column, then we expect to find it in value, and not in key - hence null arg.
+                Object colVal = hasNewColVal ? newColVals.get(c.getName()) : prop.value(null, val);
 
-                    newVal = ((BinaryObjectBuilder) newVal).build();
-                }
+                // UPDATE currently does not allow to modify key or its fields, so we must be safe to pass null as key.
+                desc.setColumnValue(null, newVal, colVal, i);
+            }
 
-                Object srcVal = e.get(1);
+            if (bin && hasProps) {
+                assert newVal instanceof BinaryObjectBuilder;
 
-                if (bin && !(srcVal instanceof BinaryObject))
-                    srcVal = cctx.grid().binary().toBinary(srcVal);
+                newVal = ((BinaryObjectBuilder) newVal).build();
+            }
 
-                rows.put(key, new ModifyingEntryProcessor(srcVal, new EntryValueUpdater(newVal)));
+            Object srcVal = e.get(1);
 
-                if ((pageSize > 0 && rows.size() == pageSize) || (!it.hasNext())) {
-                    PageProcessingResult pageRes = processPage(cctx, rows);
+            if (bin && !(srcVal instanceof BinaryObject))
+                srcVal = cctx.grid().binary().toBinary(srcVal);
 
-                    res += pageRes.cnt;
+            rows.put(key, new ModifyingEntryProcessor(srcVal, new EntryValueUpdater(newVal)));
 
-                    failedKeys.addAll(F.asList(pageRes.errKeys));
+            if ((pageSize > 0 && rows.size() == pageSize) || (!it.hasNext())) {
+                PageProcessingResult pageRes = processPage(cctx, rows);
 
-                    if (pageRes.ex != null) {
-                        if (resEx == null)
-                            resEx = pageRes.ex;
-                        else
-                            resEx.setNextException(pageRes.ex);
-                    }
+                res += pageRes.cnt;
 
-                    if (it.hasNext())
-                        rows.clear(); // No need to clear after the last batch.
+                failedKeys.addAll(F.asList(pageRes.errKeys));
+
+                if (pageRes.ex != null) {
+                    if (resEx == null)
+                        resEx = pageRes.ex;
+                    else
+                        resEx.setNextException(pageRes.ex);
                 }
-            }
 
-            if (resEx != null) {
-                if (!F.isEmpty(failedKeys)) {
-                    // Don't go for a re-run if processing of some keys yielded exceptions and report keys that
-                    // had been modified concurrently right away.
-                    String msg = "Failed to UPDATE some keys because they had been modified concurrently " +
-                        "[keys=" + failedKeys + ']';
+                if (it.hasNext())
+                    rows.clear(); // No need to clear after the last batch.
+            }
+        }
 
-                    SQLException dupEx = createJdbcSqlException(msg, IgniteQueryErrorCode.CONCURRENT_UPDATE);
+        if (resEx != null) {
+            if (!F.isEmpty(failedKeys)) {
+                // Don't go for a re-run if processing of some keys yielded exceptions and report keys that
+                // had been modified concurrently right away.
+                String msg = "Failed to UPDATE some keys because they had been modified concurrently " +
+                    "[keys=" + failedKeys + ']';
 
-                    dupEx.setNextException(resEx);
+                SQLException dupEx = createJdbcSqlException(msg, IgniteQueryErrorCode.CONCURRENT_UPDATE);
 
-                    resEx = dupEx;
-                }
+                dupEx.setNextException(resEx);
 
-                throw new IgniteSQLException(resEx);
+                resEx = dupEx;
             }
+
+            throw new IgniteSQLException(resEx);
         }
-        finally {
-            cctx.operationContextPerCall(opCtx);
-        }
+
 
         return new UpdateResult(res, failedKeys.toArray());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0127d3c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
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 15c94c3..549b901 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
@@ -99,7 +99,7 @@ public final class UpdatePlanBuilder {
 
         GridSqlColumn[] cols;
 
-        boolean isTableSubqry;
+        boolean isTwoStepSubqry;
 
         int rowsNum;
 
@@ -116,8 +116,8 @@ public final class UpdatePlanBuilder {
 
             cols = ins.columns();
             sel = DmlAstUtils.selectForInsertOrMerge(cols, ins.rows(), ins.query(), desc);
-            isTableSubqry = (ins.query() != null);
-            rowsNum = isTableSubqry ? 0 : ins.rows().size();
+            isTwoStepSubqry = (ins.query() != null);
+            rowsNum = isTwoStepSubqry ? 0 : ins.rows().size();
         }
         else if (stmt instanceof GridSqlMerge) {
             GridSqlMerge merge = (GridSqlMerge) stmt;
@@ -137,14 +137,14 @@ public final class UpdatePlanBuilder {
 
             cols = merge.columns();
             sel = DmlAstUtils.selectForInsertOrMerge(cols, merge.rows(), merge.query(), desc);
-            isTableSubqry = (merge.query() != null);
-            rowsNum = isTableSubqry ? 0 : merge.rows().size();
+            isTwoStepSubqry = (merge.query() != null);
+            rowsNum = isTwoStepSubqry ? 0 : merge.rows().size();
         }
         else throw new IgniteSQLException("Unexpected DML operation [cls=" + stmt.getClass().getName() + ']',
                 IgniteQueryErrorCode.UNEXPECTED_OPERATION);
 
         // Let's set the flag only for subqueries that have their FROM specified.
-        isTableSubqry = (isTableSubqry && (sel instanceof GridSqlUnion ||
+        isTwoStepSubqry = (isTwoStepSubqry && (sel instanceof GridSqlUnion ||
             (sel instanceof GridSqlSelect && ((GridSqlSelect) sel).from() != null)));
 
         int keyColIdx = -1;
@@ -189,10 +189,10 @@ public final class UpdatePlanBuilder {
 
         if (stmt instanceof GridSqlMerge)
             return UpdatePlan.forMerge(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
-                sel.getSQL(), !isTableSubqry, rowsNum);
+                sel.getSQL(), !isTwoStepSubqry, rowsNum);
         else
             return UpdatePlan.forInsert(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
-                sel.getSQL(), !isTableSubqry, rowsNum);
+                sel.getSQL(), !isTwoStepSubqry, rowsNum);
     }
 
     /**


[20/52] ignite git commit: IGNITE-4270: Fixed system property name.

Posted by vo...@apache.org.
IGNITE-4270: Fixed system property name.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7b35ba73
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7b35ba73
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7b35ba73

Branch: refs/heads/master
Commit: 7b35ba730edbb91a30a59b1efdc22eceea7298e1
Parents: deee3b9
Author: devozerov <vo...@gridgain.com>
Authored: Fri Nov 25 11:22:05 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 11:22:05 2016 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/IgniteSystemProperties.java    | 2 +-
 .../src/main/java/org/apache/ignite/internal/util/GridUnsafe.java  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7b35ba73/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 043c95a..16fc759 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -491,7 +491,7 @@ public final class IgniteSystemProperties {
      * <p>
      * Defaults to {@code} false, meaning that unaligned access will be performed only on x86 architecture.
      */
-    public static final String IGNITE_UNALIGNED_MEMORY_ACCESS = "IGNITE_UNALIGNED_MEMORY_ACCESS";
+    public static final String IGNITE_MEMORY_UNALIGNED_ACCESS = "IGNITE_MEMORY_UNALIGNED_ACCESS";
 
     /**
      * Enforces singleton.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b35ba73/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
index 1c492ef..35b793a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
@@ -1209,7 +1209,7 @@ public abstract class GridUnsafe {
         boolean res = arch.equals("i386") || arch.equals("x86") || arch.equals("amd64") || arch.equals("x86_64");
 
         if (!res)
-            res = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_UNALIGNED_MEMORY_ACCESS, false);
+            res = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_MEMORY_UNALIGNED_ACCESS, false);
 
         return res;
     }


[42/52] ignite git commit: IGNITE-4017: Added DML example. This closes #1293.

Posted by vo...@apache.org.
IGNITE-4017: Added DML example. This closes #1293.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/e205214a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/e205214a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/e205214a

Branch: refs/heads/master
Commit: e205214ab4680f3b873a5cd4cb41ce877d2d3f17
Parents: a53fd38
Author: Alexander Paschenko <al...@gmail.com>
Authored: Mon Nov 28 15:51:24 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Nov 28 15:51:24 2016 +0300

----------------------------------------------------------------------
 .../examples/datagrid/CacheQueryDmlExample.java | 163 +++++++++++++++++++
 .../ignite/examples/CacheExamplesSelfTest.java  |   8 +
 2 files changed, 171 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e205214a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryDmlExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryDmlExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryDmlExample.java
new file mode 100644
index 0000000..21027d0
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryDmlExample.java
@@ -0,0 +1,163 @@
+/*
+ * 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.examples.datagrid;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.examples.model.Organization;
+import org.apache.ignite.examples.model.Person;
+
+import java.util.List;
+
+/**
+ * Example to showcase DML capabilities of Ignite's SQL engine.
+ */
+public class CacheQueryDmlExample {
+    /** Organizations cache name. */
+    private static final String ORG_CACHE = CacheQueryDmlExample.class.getSimpleName() + "Organizations";
+
+    /** Persons cache name. */
+    private static final String PERSON_CACHE = CacheQueryDmlExample.class.getSimpleName() + "Persons";
+
+    /**
+     * Executes example.
+     *
+     * @param args Command line arguments, none required.
+     * @throws Exception If example execution failed.
+     */
+    @SuppressWarnings({"unused", "ThrowFromFinallyBlock"})
+    public static void main(String[] args) throws Exception {
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            print("Cache query DML example started.");
+
+            CacheConfiguration<Long, Organization> orgCacheCfg = new CacheConfiguration<>(ORG_CACHE);
+            orgCacheCfg.setIndexedTypes(Long.class, Organization.class);
+
+            CacheConfiguration<Long, Person> personCacheCfg = new CacheConfiguration<>(PERSON_CACHE);
+            personCacheCfg.setIndexedTypes(Long.class, Person.class);
+
+            // Auto-close cache at the end of the example.
+            try (
+                IgniteCache<Long, Organization> orgCache = ignite.getOrCreateCache(orgCacheCfg);
+                IgniteCache<Long, Person> personCache = ignite.getOrCreateCache(personCacheCfg)
+            ) {
+                insert(orgCache, personCache);
+                select(personCache, "Insert data");
+
+                update(personCache);
+                select(personCache, "Update salary for Master degrees");
+
+                delete(personCache);
+                select(personCache, "Delete non-Apache employees");
+            }
+            finally {
+                // Distributed cache could be removed from cluster only by #destroyCache() call.
+                ignite.destroyCache(PERSON_CACHE);
+                ignite.destroyCache(ORG_CACHE);
+            }
+
+            print("Cache query DML example finished.");
+        }
+    }
+
+    /**
+     * Populate cache with test data.
+     *
+     * @param orgCache Organization cache,
+     * @param personCache Person cache.
+     */
+    private static void insert(IgniteCache<Long, Organization> orgCache, IgniteCache<Long, Person> personCache) {
+        // Insert organizations.
+        SqlFieldsQuery qry = new SqlFieldsQuery("insert into Organization (_key, id, name) values (?, ?, ?)");
+
+        orgCache.query(qry.setArgs(1L, 1L, "ASF"));
+        orgCache.query(qry.setArgs(2L, 2L, "Eclipse"));
+
+        // Insert persons.
+        qry = new SqlFieldsQuery(
+            "insert into Person (_key, id, orgId, firstName, lastName, salary, resume) values (?, ?, ?, ?, ?, ?, ?)");
+
+        personCache.query(qry.setArgs(1L, 1L, 1L, "John", "Doe", 4000, "Master"));
+        personCache.query(qry.setArgs(2L, 2L, 1L, "Jane", "Roe", 2000, "Bachelor"));
+        personCache.query(qry.setArgs(3L, 3L, 2L, "Mary", "Major", 5000, "Master"));
+        personCache.query(qry.setArgs(4L, 4L, 2L, "Richard", "Miles", 3000, "Bachelor"));
+    }
+
+    /**
+     * Example of conditional UPDATE query: raise salary by 10% to everyone who has Master degree.
+     *
+     * @param personCache Person cache.
+     */
+    private static void update(IgniteCache<Long, Person> personCache) {
+        String sql =
+            "update Person set salary = salary * 1.1 " +
+            "where resume = ?";
+
+        personCache.query(new SqlFieldsQuery(sql).setArgs("Master"));
+    }
+
+    /**
+     * Example of conditional DELETE query: delete non-Apache employees.
+     *
+     * @param personCache Person cache.
+     */
+    private static void delete(IgniteCache<Long, Person> personCache) {
+        String sql =
+            "delete from Person " +
+            "where id in (" +
+                "select p.id " +
+                "from Person p, \"" + ORG_CACHE + "\".Organization as o " +
+                "where o.name != ? and p.orgId = o.id" +
+            ")";
+
+        personCache.query(new SqlFieldsQuery(sql).setArgs("ASF")).getAll();
+    }
+
+    /**
+     * Query current data.
+     *
+     * @param personCache Person cache.
+     * @param msg Message.
+     */
+    private static void select(IgniteCache<Long, Person> personCache, String msg) {
+        String sql =
+            "select p.id, concat(p.firstName, ' ', p.lastName), o.name, p.resume, p.salary " +
+            "from Person as p, \"" + ORG_CACHE + "\".Organization as o " +
+            "where p.orgId = o.id";
+
+        List<List<?>> res = personCache.query(new SqlFieldsQuery(sql).setDistributedJoins(true)).getAll();
+
+        print(msg);
+
+        for (Object next : res)
+            System.out.println(">>>     " + next);
+    }
+
+    /**
+     * Prints message.
+     *
+     * @param msg Message to print before all objects are printed.
+     */
+    private static void print(String msg) {
+        System.out.println();
+        System.out.println(">>> " + msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e205214a/examples/src/test/java/org/apache/ignite/examples/CacheExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/examples/CacheExamplesSelfTest.java b/examples/src/test/java/org/apache/ignite/examples/CacheExamplesSelfTest.java
index 43b05b5..4bec419 100644
--- a/examples/src/test/java/org/apache/ignite/examples/CacheExamplesSelfTest.java
+++ b/examples/src/test/java/org/apache/ignite/examples/CacheExamplesSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.examples.datagrid.CacheApiExample;
 import org.apache.ignite.examples.datagrid.CacheContinuousQueryExample;
 import org.apache.ignite.examples.datagrid.CacheDataStreamerExample;
 import org.apache.ignite.examples.datagrid.CachePutGetExample;
+import org.apache.ignite.examples.datagrid.CacheQueryDmlExample;
 import org.apache.ignite.examples.datagrid.CacheQueryExample;
 import org.apache.ignite.examples.datagrid.CacheTransactionExample;
 import org.apache.ignite.examples.datagrid.starschema.CacheStarSchemaExample;
@@ -129,6 +130,13 @@ public class CacheExamplesSelfTest extends GridAbstractExamplesTest {
     /**
      * @throws Exception If failed.
      */
+    public void testCacheQueryDmlExample() throws Exception {
+        CacheQueryDmlExample.main(EMPTY_ARGS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testCacheApiExample() throws Exception {
         CacheApiExample.main(EMPTY_ARGS);
     }


[46/52] ignite git commit: IGNITE-4321 Minor update of README.txt in ignite-cassandra-serializers module.

Posted by vo...@apache.org.
IGNITE-4321 Minor update of README.txt in ignite-cassandra-serializers module.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/f2d8d078
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/f2d8d078
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/f2d8d078

Branch: refs/heads/master
Commit: f2d8d078597ee46fef40b07a8d263be0a76d0385
Parents: ca28001
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Nov 30 17:17:36 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Nov 30 17:17:36 2016 +0700

----------------------------------------------------------------------
 modules/cassandra/serializers/README.txt | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f2d8d078/modules/cassandra/serializers/README.txt
----------------------------------------------------------------------
diff --git a/modules/cassandra/serializers/README.txt b/modules/cassandra/serializers/README.txt
index 01948ec..aaa309f 100644
--- a/modules/cassandra/serializers/README.txt
+++ b/modules/cassandra/serializers/README.txt
@@ -4,9 +4,10 @@ Apache Ignite Cassandra Serializers Module
 Apache Ignite Cassandra Serializers module provides additional serializers to store objects as BLOBs in Cassandra. The
 module could be used as an addition to Ignite Cassandra Store module.
 
-To enable Cassandra Serializers module when starting a standalone node, move 'optional/ignite-cassandra-serializers' folder to
-'libs' folder before running 'ignite.{sh|bat}' script. The content of the module folder will
-be added to classpath in this case.
+To enable Cassandra Serializers module when starting a standalone node, move 'optional/ignite-cassandra-serializers'
+folder to 'libs' folder before running 'ignite.{sh|bat}' script. The content of the module folder will be added to
+classpath in this case. Note, copying folder 'optional/ignite-cassandra-serializers' requires copying
+'optional/ignite-cassandra-store' folder.
 
 Importing Cassandra Serializers Module In Maven Project
 -------------------------------------


[18/52] ignite git commit: IGNITE-4280 Test fix for IgniteBinaryCacheQueryTestSuite. This closes #1275.

Posted by vo...@apache.org.
IGNITE-4280 Test fix for IgniteBinaryCacheQueryTestSuite. This closes #1275.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/fabb77d1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/fabb77d1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/fabb77d1

Branch: refs/heads/master
Commit: fabb77d17859a0a6318c2fe11182ab753e2198e0
Parents: c4badee
Author: Alexander Paschenko <al...@gmail.com>
Authored: Thu Nov 24 18:51:07 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Nov 24 18:51:07 2016 +0300

----------------------------------------------------------------------
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |  2 ++
 .../org/apache/ignite/cache/QueryEntity.java    |  2 +-
 .../processors/query/GridQueryProcessor.java    | 23 ++++++++++----------
 .../odbc-example/config/example-odbc.xml        |  8 +++++++
 .../odbc-test/config/queries-test-noodbc.xml    |  4 ++++
 .../cpp/odbc-test/config/queries-test.xml       |  4 ++++
 6 files changed, 30 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fabb77d1/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
index 4a97aef..1590067 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
@@ -118,6 +118,8 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac
         e.addQueryField("firstName", String.class.getName(), null);
         e.addQueryField("lastName", String.class.getName(), null);
 
+        e.setKeyFields(Collections.<String>emptySet());
+
         ccfg.setQueryEntities(Collections.singletonList(e));
 
         return cfg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/fabb77d1/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
index fc3b921..c8abdfa 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
@@ -45,7 +45,7 @@ public class QueryEntity implements Serializable {
     private LinkedHashMap<String, String> fields = new LinkedHashMap<>();
 
     /** Set of field names that belong to the key. */
-    private Set<String> keyFields = new HashSet<>();
+    private Set<String> keyFields;
 
     /** Aliases. */
     private Map<String, String> aliases = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/fabb77d1/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index fe773c1..0fc6044 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -44,7 +44,6 @@ import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.binary.BinaryField;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
@@ -1425,7 +1424,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             aliases = Collections.emptyMap();
 
         for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null, d);
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null);
 
             d.addProperty(prop, false);
 
@@ -1437,7 +1436,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null, d);
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null);
 
             d.addProperty(prop, false);
 
@@ -1449,7 +1448,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (String txtIdx : meta.getTextFields()) {
-            BinaryProperty prop = buildBinaryProperty(txtIdx, String.class, aliases, null, d);
+            BinaryProperty prop = buildBinaryProperty(txtIdx, String.class, aliases, null);
 
             d.addProperty(prop, false);
 
@@ -1468,7 +1467,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                 for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
                     BinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(), aliases,
-                        null, d);
+                        null);
 
                     d.addProperty(prop, false);
 
@@ -1482,7 +1481,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null, d);
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null);
 
             if (!d.props.containsKey(prop.name()))
                 d.addProperty(prop, false);
@@ -1509,7 +1508,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             Boolean isKeyField = (hasKeyFields ? keyFields.contains(entry.getKey()) : null);
 
             BinaryProperty prop = buildBinaryProperty(entry.getKey(), U.classForName(entry.getValue(), Object.class, true),
-                aliases, isKeyField, d);
+                aliases, isKeyField);
 
             d.addProperty(prop, false);
         }
@@ -1609,12 +1608,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      *      nested fields.
      * @param resType Result type.
      * @param aliases Aliases.
-     * @param isKeyField
-     *@param d Type descriptor.  @return Binary property.
+     * @param isKeyField Key ownership flag, as defined in {@link QueryEntity#keyFields}: {@code true} if field belongs
+     *      to key, {@code false} if it belongs to value, {@code null} if QueryEntity#keyFields is null.
+     * @return Binary property.
      */
     private BinaryProperty buildBinaryProperty(String pathStr, Class<?> resType, Map<String, String> aliases,
-        @Nullable Boolean isKeyField, TypeDescriptor d) throws IgniteCheckedException {
-
+        @Nullable Boolean isKeyField) throws IgniteCheckedException {
         String[] path = pathStr.split("\\.");
 
         BinaryProperty res = null;
@@ -2190,7 +2189,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             if (isKeyProp0 == 0)
                 throw new IllegalStateException("Ownership flag not set for binary property. Have you set 'keyFields'" +
-                    " property of QueryEntity in configuration XML?");
+                    " property of QueryEntity in programmatic or XML configuration?");
 
             return isKeyProp0 == 1;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fabb77d1/modules/platforms/cpp/examples/odbc-example/config/example-odbc.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/odbc-example/config/example-odbc.xml b/modules/platforms/cpp/examples/odbc-example/config/example-odbc.xml
index 864f950..e19075e 100644
--- a/modules/platforms/cpp/examples/odbc-example/config/example-odbc.xml
+++ b/modules/platforms/cpp/examples/odbc-example/config/example-odbc.xml
@@ -56,6 +56,10 @@
                                     </map>
                                 </property>
 
+                                <property name="keyFields">
+                                    <list></list>
+                                </property>
+
                                 <property name="indexes">
                                     <list>
                                         <bean class="org.apache.ignite.cache.QueryIndex">
@@ -85,6 +89,10 @@
                                         <entry key="name" value="java.lang.String"/>
                                     </map>
                                 </property>
+
+                                <property name="keyFields">
+                                    <list></list>
+                                </property>
                             </bean>
                         </list>
                     </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/fabb77d1/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
index db19669..82173de 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
@@ -61,6 +61,10 @@
                                     </map>
                                 </property>
 
+                                <property name="keyFields">
+                                    <list></list>
+                                </property>
+
                                 <property name="indexes">
                                     <list>
                                         <bean class="org.apache.ignite.cache.QueryIndex">

http://git-wip-us.apache.org/repos/asf/ignite/blob/fabb77d1/modules/platforms/cpp/odbc-test/config/queries-test.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test.xml b/modules/platforms/cpp/odbc-test/config/queries-test.xml
index 26e6341..f511c7d 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-test.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-test.xml
@@ -67,6 +67,10 @@
                                     </map>
                                 </property>
 
+                                <property name="keyFields">
+                                    <list></list>
+                                </property>
+
                                 <property name="indexes">
                                     <list>
                                         <bean class="org.apache.ignite.cache.QueryIndex">


[52/52] ignite git commit: Merge branch 'ignite-1.8'

Posted by vo...@apache.org.
Merge branch 'ignite-1.8'

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisDelCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisExistsCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/server/GridRedisDbSizeCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisAppendCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetRangeCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetSetCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMGetCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMSetCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetRangeCommandHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisStrlenCommandHandler.java


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/6d348aed
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/6d348aed
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/6d348aed

Branch: refs/heads/master
Commit: 6d348aedaff492758111c17460d93fa904b90b9e
Parents: 6e8c35b 97a6515
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 5 15:37:15 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 5 15:37:15 2016 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |   22 +
 assembly/dependencies-fabric-lgpl.xml           |    1 +
 assembly/dependencies-fabric.xml                |    1 +
 assembly/release-fabric-base.xml                |    1 +
 .../examples/datagrid/CacheQueryDmlExample.java |  163 +++
 .../CacheExamplesMultiNodeSelfTest.java         |    7 +
 .../ignite/examples/CacheExamplesSelfTest.java  |    8 +
 modules/cassandra/pom.xml                       |   16 +
 modules/cassandra/serializers/README.txt        |    7 +-
 .../clients/src/test/config/jdbc-bin-config.xml |   54 +
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |  263 +++++
 .../JdbcAbstractUpdateStatementSelfTest.java    |   37 +
 ...BinaryMarshallerInsertStatementSelfTest.java |   37 +
 ...cBinaryMarshallerMergeStatementSelfTest.java |   37 +
 .../jdbc2/JdbcDeleteStatementSelfTest.java      |   49 +
 .../jdbc2/JdbcInsertStatementSelfTest.java      |  122 ++
 .../jdbc2/JdbcMergeStatementSelfTest.java       |   91 ++
 .../internal/jdbc2/JdbcNoDefaultCacheTest.java  |   40 +-
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |   18 +
 .../tcp/redis/RedisProtocolSelfTest.java        |  101 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    5 +
 .../java/org/apache/ignite/IgniteCache.java     |    2 +
 .../apache/ignite/IgniteSystemProperties.java   |    2 +-
 .../binary/BinaryAbstractIdentityResolver.java  |   53 +
 .../binary/BinaryArrayIdentityResolver.java     |  224 ++++
 .../binary/BinaryFieldIdentityResolver.java     |  307 +++++
 .../ignite/binary/BinaryIdentityResolver.java   |   42 +
 .../ignite/binary/BinaryTypeConfiguration.java  |   27 +-
 .../org/apache/ignite/cache/QueryEntity.java    |   27 +
 .../ignite/cache/query/SqlFieldsQuery.java      |    2 +-
 .../configuration/CacheConfiguration.java       |   19 +-
 .../internal/binary/BinaryClassDescriptor.java  |   15 +
 .../ignite/internal/binary/BinaryContext.java   |   76 +-
 .../ignite/internal/binary/BinaryFieldImpl.java |   10 +-
 .../internal/binary/BinaryObjectExImpl.java     |   90 +-
 .../internal/binary/BinaryObjectImpl.java       |   48 +-
 .../binary/BinaryObjectOffheapImpl.java         |   42 +-
 .../internal/binary/BinaryPrimitives.java       |   24 +
 .../binary/BinarySerializedFieldComparator.java |  343 ++++++
 .../ignite/internal/binary/BinaryUtils.java     |    2 +-
 .../internal/binary/BinaryWriterExImpl.java     |   47 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |    6 +
 .../streams/BinaryAbstractInputStream.java      |    5 +
 .../streams/BinaryAbstractOutputStream.java     |    5 +
 .../binary/streams/BinaryHeapInputStream.java   |    5 +
 .../binary/streams/BinaryHeapOutputStream.java  |    5 +
 .../streams/BinaryOffheapInputStream.java       |   10 +
 .../streams/BinaryOffheapOutputStream.java      |    4 +-
 .../internal/binary/streams/BinaryStream.java   |   12 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |   17 +
 .../internal/jdbc2/JdbcPreparedStatement.java   |   71 +-
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |    5 +-
 .../ignite/internal/jdbc2/JdbcQueryTaskV2.java  |  406 +++++++
 .../ignite/internal/jdbc2/JdbcResultSet.java    |   72 +-
 .../internal/jdbc2/JdbcSqlFieldsQuery.java      |   49 +
 .../ignite/internal/jdbc2/JdbcStatement.java    |  213 +++-
 .../apache/ignite/internal/jdbc2/JdbcUtils.java |   25 +-
 .../processors/cache/QueryCursorImpl.java       |   31 +-
 ...GridNearAtomicSingleUpdateInvokeRequest.java |    1 +
 .../cache/query/GridCacheTwoStepQuery.java      |  253 ----
 .../cache/query/IgniteQueryErrorCode.java       |   91 ++
 .../cache/transactions/IgniteTxManager.java     |    7 +-
 .../processors/odbc/OdbcMessageParser.java      |   16 +
 .../processors/odbc/OdbcNioListener.java        |    2 +-
 .../odbc/OdbcQueryGetParamsMetaRequest.java     |   60 +
 .../odbc/OdbcQueryGetParamsMetaResult.java      |   40 +
 .../internal/processors/odbc/OdbcRequest.java   |    3 +
 .../processors/odbc/OdbcRequestHandler.java     |  151 ++-
 .../internal/processors/odbc/OdbcUtils.java     |    4 +-
 .../memory/PlatformInputStreamImpl.java         |   10 +
 .../memory/PlatformOutputStreamImpl.java        |   12 +
 .../processors/query/GridQueryIndexing.java     |   23 +-
 .../processors/query/GridQueryProcessor.java    |  589 ++++++++--
 .../processors/query/GridQueryProperty.java     |   20 +
 .../query/GridQueryTypeDescriptor.java          |   25 +
 .../processors/query/IgniteSQLException.java    |   89 ++
 .../string/GridRedisGetCommandHandler.java      |   27 +-
 .../string/GridRedisIncrDecrCommandHandler.java |   63 +-
 .../string/GridRedisSetCommandHandler.java      |   24 +-
 .../tcp/redis/GridRedisNioListener.java         |    4 +-
 .../apache/ignite/internal/util/GridUnsafe.java |    2 +-
 .../util/lang/IgniteSingletonIterator.java      |   56 +
 .../BinaryArrayIdentityResolverSelfTest.java    |  300 +++++
 .../BinaryFieldIdentityResolverSelfTest.java    |  333 ++++++
 ...ryIdentityResolverConfigurationSelfTest.java |  138 +++
 .../BinarySerialiedFieldComparatorSelfTest.java |  568 +++++++++
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  260 ++++-
 .../IgniteBinaryObjectsTestSuite.java           |   10 +
 .../cache/query/GridCacheTwoStepQuery.java      |  253 ++++
 .../query/h2/DmlStatementsProcessor.java        | 1083 ++++++++++++++++++
 .../query/h2/GridH2ResultSetIterator.java       |    3 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  130 ++-
 .../query/h2/dml/FastUpdateArgument.java        |   27 +
 .../query/h2/dml/FastUpdateArguments.java       |   53 +
 .../query/h2/dml/KeyValueSupplier.java          |   30 +
 .../processors/query/h2/dml/UpdateMode.java     |   36 +
 .../processors/query/h2/dml/UpdatePlan.java     |  128 +++
 .../query/h2/dml/UpdatePlanBuilder.java         |  486 ++++++++
 .../processors/query/h2/dml/package-info.java   |   22 +
 .../query/h2/opt/GridH2RowDescriptor.java       |   23 +
 .../query/h2/opt/GridH2TreeIndex.java           |    2 +-
 .../processors/query/h2/sql/DmlAstUtils.java    |  616 ++++++++++
 .../processors/query/h2/sql/GridSqlArray.java   |    8 +
 .../processors/query/h2/sql/GridSqlConst.java   |    6 +
 .../processors/query/h2/sql/GridSqlDelete.java  |   68 ++
 .../query/h2/sql/GridSqlFunction.java           |    5 +-
 .../processors/query/h2/sql/GridSqlInsert.java  |  149 +++
 .../processors/query/h2/sql/GridSqlKeyword.java |   46 +
 .../processors/query/h2/sql/GridSqlMerge.java   |  143 +++
 .../processors/query/h2/sql/GridSqlQuery.java   |   44 +-
 .../query/h2/sql/GridSqlQueryParser.java        |  323 +++++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |    6 +-
 .../processors/query/h2/sql/GridSqlSelect.java  |    3 +-
 .../query/h2/sql/GridSqlStatement.java          |   64 ++
 .../processors/query/h2/sql/GridSqlUpdate.java  |  105 ++
 .../h2/twostep/GridReduceQueryExecutor.java     |   28 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |  559 +++++++++
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |  243 ++++
 .../IgniteCacheDeleteSqlQuerySelfTest.java      |  106 ++
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  203 ++++
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  153 +++
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |  472 ++++++++
 .../IgniteCacheAtomicFieldsQuerySelfTest.java   |   21 -
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   40 +-
 .../query/h2/sql/GridQueryParsingTest.java      |  109 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    9 +
 .../cpp/core-test/config/cache-query.xml        |   10 +
 .../odbc-example/config/example-odbc.xml        |   38 +-
 .../project/vs/odbc-example.vcxproj             |    7 +-
 .../project/vs/odbc-example.vcxproj.filters     |    8 +
 .../examples/odbc-example/src/odbc_example.cpp  |  514 +++++++--
 .../query-example/src/query_example.cpp         |    2 +-
 .../odbc-test/config/queries-test-noodbc.xml    |    4 +
 .../cpp/odbc-test/config/queries-test.xml       |    4 +
 .../src/application_data_buffer_test.cpp        |    2 +-
 .../cpp/odbc-test/src/queries_test.cpp          |  336 ++++++
 .../platforms/cpp/odbc/include/ignite/odbc.h    |   14 +-
 .../ignite/odbc/app/application_data_buffer.h   |    4 +-
 .../odbc/include/ignite/odbc/app/parameter.h    |    2 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |    3 +
 .../cpp/odbc/include/ignite/odbc/message.h      |  138 ++-
 .../odbc/include/ignite/odbc/query/data_query.h |   12 +-
 .../cpp/odbc/include/ignite/odbc/query/query.h  |   44 +-
 .../cpp/odbc/include/ignite/odbc/statement.h    |   83 +-
 .../cpp/odbc/include/ignite/odbc/type_traits.h  |    2 +-
 .../cpp/odbc/include/ignite/odbc/utility.h      |   11 +-
 .../odbc/src/app/application_data_buffer.cpp    |   57 +-
 .../platforms/cpp/odbc/src/app/parameter.cpp    |    3 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |    4 +-
 modules/platforms/cpp/odbc/src/entry_points.cpp |   32 +-
 modules/platforms/cpp/odbc/src/odbc.cpp         |   25 +-
 .../odbc/src/query/column_metadata_query.cpp    |    2 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |    6 +-
 .../cpp/odbc/src/query/foreign_keys_query.cpp   |    2 +-
 .../cpp/odbc/src/query/primary_keys_query.cpp   |    2 +-
 .../odbc/src/query/special_columns_query.cpp    |    2 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |    2 +-
 .../cpp/odbc/src/query/type_info_query.cpp      |    2 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |  230 +++-
 modules/platforms/cpp/odbc/src/utility.cpp      |   22 +-
 .../Apache.Ignite.AspNet.nuspec                 |    4 +-
 .../Examples/Example.cs                         |    6 +-
 .../Examples/ExamplesTest.cs                    |    7 +-
 .../Process/IgniteProcess.cs                    |   40 +-
 .../Datagrid/MultiTieredCacheExample.cs         |    8 +-
 .../config/benchmark-bin-identity.properties    |   94 ++
 .../config/benchmark-multicast.properties       |   15 +
 .../config/benchmark-sql-dml.properties         |   72 ++
 modules/yardstick/config/ignite-base-config.xml |   73 +-
 .../config/ignite-bin-multicast-config.xml      |   86 ++
 .../cache/IgniteBinaryIdentityBenchmark.java    |  108 ++
 .../cache/IgniteBinaryIdentityGetBenchmark.java |   34 +
 .../cache/IgniteBinaryIdentityPutBenchmark.java |   35 +
 .../IgniteFieldsBinaryIdentityGetBenchmark.java |   30 +
 .../IgniteFieldsBinaryIdentityPutBenchmark.java |   30 +
 .../IgniteLegacyBinaryIdentityGetBenchmark.java |   30 +
 .../IgniteLegacyBinaryIdentityPutBenchmark.java |   30 +
 ...IgnitePutIfAbsentIndexedValue1Benchmark.java |   45 +
 .../IgniteReplaceIndexedValue1Benchmark.java    |   79 ++
 .../cache/dml/IgniteSqlDeleteBenchmark.java     |   83 ++
 .../dml/IgniteSqlDeleteFilteredBenchmark.java   |   88 ++
 .../IgniteSqlInsertIndexedValue1Benchmark.java  |   48 +
 .../IgniteSqlInsertIndexedValue2Benchmark.java  |   48 +
 .../IgniteSqlInsertIndexedValue8Benchmark.java  |   48 +
 .../cache/dml/IgniteSqlMergeAllBenchmark.java   |   82 ++
 .../cache/dml/IgniteSqlMergeBenchmark.java      |   42 +
 .../IgniteSqlMergeIndexedValue1Benchmark.java   |   43 +
 .../IgniteSqlMergeIndexedValue2Benchmark.java   |   43 +
 .../IgniteSqlMergeIndexedValue8Benchmark.java   |   43 +
 .../cache/dml/IgniteSqlMergeQueryBenchmark.java |  116 ++
 .../cache/dml/IgniteSqlUpdateBenchmark.java     |   82 ++
 .../dml/IgniteSqlUpdateFilteredBenchmark.java   |   88 ++
 .../yardstick/cache/model/SampleValue.java      |    2 +
 193 files changed, 14608 insertions(+), 1112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6d348aed/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6d348aed/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6d348aed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
index ded66af,aac4170..cc7008f
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
@@@ -44,14 -46,21 +46,21 @@@ public class GridRedisGetCommandHandle
          GET
      );
  
+     /** Grid context. */
+     private final GridKernalContext ctx;
+ 
      /**
 -     * Constructor.
 +     * Handler constructor.
       *
 -     * @param log Logger.
 -     * @param hnd Handler.
 +     * @param log Logger to use.
 +     * @param hnd Rest handler.
+      * @param ctx Context.
       */
-     public GridRedisGetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+     public GridRedisGetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd,
+         GridKernalContext ctx) {
          super(log, hnd);
+ 
+         this.ctx = ctx;
      }
  
      /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6d348aed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6d348aed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
index 9e53055,2eea0e1..8a7adf4
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisSetCommandHandler.java
@@@ -52,14 -55,21 +55,21 @@@ public class GridRedisSetCommandHandle
      /** Value position in Redis message. */
      private static final int VAL_POS = 2;
  
+     /** Grid context. */
+     private final GridKernalContext ctx;
+ 
      /**
 -     * Constructor.
 +     * Handler constructor.
       *
 -     * @param log Logger.
 -     * @param hnd Handler.
 +     * @param log Logger to use.
 +     * @param hnd Rest handler.
+      * @param ctx Context.
       */
-     public GridRedisSetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+     public GridRedisSetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd,
+         GridKernalContext ctx) {
          super(log, hnd);
+ 
+         this.ctx = ctx;
      }
  
      /** {@inheritDoc} */


[02/52] ignite git commit: IGNITE-2294: Implemented DML.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------
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 d13af12..537ccdf 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
@@ -40,8 +40,11 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.h2.command.Prepared;
 import org.h2.command.dml.Query;
+import org.h2.command.dml.Update;
 import org.h2.engine.Session;
+import org.h2.expression.Expression;
 import org.h2.jdbc.JdbcConnection;
+import org.h2.util.StringUtils;
 
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
@@ -106,7 +109,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testAllExamples() throws Exception {
+    public void testParseSelectAndUnion() throws Exception {
         checkQuery("select 42");
         checkQuery("select ()");
         checkQuery("select (1)");
@@ -120,6 +123,8 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
         checkQuery("select (select 1)");
         checkQuery("select (select 1, select ?)");
         checkQuery("select ((select 1), select ? + ?)");
+        checkQuery("select CURRENT_DATE");
+        checkQuery("select CURRENT_DATE()");
 
         checkQuery("select extract(year from ?)");
         checkQuery("select convert(?, timestamp)");
@@ -263,6 +268,106 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
         checkQuery("(select 2 a) union all (select 1) order by a desc nulls first limit ? offset ?");
     }
 
+    /** */
+    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(DEFAULT)");
+        checkQuery("merge into Person(name) values(DEFAULT), (null)");
+        checkQuery("merge into Person(name, parentName) values(DEFAULT, null), (?, ?)");
+        checkQuery("merge into Person(old, name) values(5, 'John',), (6, 'Jack')");
+        checkQuery("merge into Person(old, name) values(5 * 3, DEFAULT,)");
+        checkQuery("merge into Person(old, name) values(ABS(-8), 'Max')");
+        checkQuery("merge into Person(old, name) values(5, 'Jane'), (DEFAULT, DEFAULT), (6, 'Jill')");
+        checkQuery("merge into Person(old, name, parentName) values(8 * 7, DEFAULT, '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'))), DEFAULT)");
+        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, DEFAULT, DEFAULT)");
+        checkQuery("merge into Person(old, name) select ASCII(parentName), INSERT(parentName, 4, 4, 'Max') from " +
+            "Person where date='20110312'");
+
+        /* 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 Address limit ? offset ?");
+    }
+
+    /** */
+    public void testParseInsert() throws Exception {
+        /* Plain rows w/functions, operators, defaults, and placeholders. */
+        checkQuery("insert into Person(old, name) values(5, 'John')");
+        checkQuery("insert into Person(name) values(DEFAULT)");
+        checkQuery("insert into Person default values");
+        checkQuery("insert into Person() values()");
+        checkQuery("insert into Person(name) values(DEFAULT), (null)");
+        checkQuery("insert into Person(name) values(DEFAULT),");
+        checkQuery("insert into Person(name, parentName) values(DEFAULT, null), (?, ?)");
+        checkQuery("insert into Person(old, name) values(5, 'John',), (6, 'Jack')");
+        checkQuery("insert into Person(old, name) values(5 * 3, DEFAULT,)");
+        checkQuery("insert into Person(old, name) values(ABS(-8), 'Max')");
+        checkQuery("insert into Person(old, name) values(5, 'Jane'), (DEFAULT, DEFAULT), (6, 'Jill')");
+        checkQuery("insert into Person(old, name, parentName) values(8 * 7, DEFAULT, 'Unknown')");
+        checkQuery("insert 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'))), DEFAULT),");
+        checkQuery("insert into Person(date, old, name, parentName, addrId) values " +
+            "('20160112', 1233, 'Ivan Ivanov', 'Peter Ivanov', 123)");
+        checkQuery("insert into Person(date, old, name, parentName, addrId) values " +
+            "(CURRENT_DATE(), RAND(), ASCII('Hi'), INSERT('Leo Tolstoy', 4, 4, 'Max'), ASCII('HI'))");
+        checkQuery("insert into Person(date, old, name, parentName, addrId) values " +
+            "(TRUNCATE(TIMESTAMP '2015-12-31 23:59:59'), POWER(3,12), NULL, DEFAULT, DEFAULT)");
+        checkQuery("insert into Person SET old = 5, name = 'John'");
+        checkQuery("insert into Person SET name = CONCAT('Fyodor', null, UPPER(CONCAT(SQRT(?), 'dostoevsky'))), old = " +
+            "select (5, 6)");
+        checkQuery("insert into Person(old, name) select ASCII(parentName), INSERT(parentName, 4, 4, 'Max') from " +
+            "Person where date='20110312'");
+
+        /* Subqueries. */
+        checkQuery("insert into Person(old, name) select old, parentName from Person");
+        checkQuery("insert into Person(old, name) direct sorted select old, parentName from Person");
+        checkQuery("insert into Person(old, name) sorted select old, parentName from Person where old > 5");
+        checkQuery("insert into Person(old, name) select 5, 'John'");
+        checkQuery("insert into Person(old, name) select p1.old, 'Name' from person p1 join person p2 on " +
+            "p2.name = p1.parentName where p2.old > 30");
+        checkQuery("insert into Person(old) select 5 from Person UNION select street from Address limit ? offset ?");
+    }
+
+    /** */
+    public void testParseDelete() throws Exception {
+        checkQuery("delete from Person");
+        checkQuery("delete from Person p where p.old > ?");
+        checkQuery("delete from Person where old in (select (40, 41, 42))");
+        checkQuery("delete top 5 from Person where old in (select (40, 41, 42))");
+        checkQuery("delete top ? from Person where old > 5 and length(name) < ?");
+        checkQuery("delete from Person where name in ('Ivan', 'Peter') limit 20");
+        checkQuery("delete from Person where name in ('Ivan', ?) limit ?");
+    }
+
+    /** */
+    public void testParseUpdate() throws Exception {
+        checkQuery("update Person set name='Peter'");
+        checkQuery("update Person per set name='Peter', old = 5");
+        checkQuery("update Person p set name='Peter' limit 20");
+        checkQuery("update Person p set name='Peter', old = length('zzz') limit 20");
+        checkQuery("update Person p set name=DEFAULT, old = null limit ?");
+        checkQuery("update Person p set name=? where old >= ? and old < ? limit ?");
+        checkQuery("update Person p set name=(select a.Street from Address a where a.id=p.addrId), old = (select 42)" +
+            " where old = sqrt(?)");
+        checkQuery("update Person p set (name, old) = (select 'Peter', 42)");
+        checkQuery("update Person p set (name, old) = (select street, id from Address where id > 5 and id <= ?)");
+    }
+
     /**
      *
      */
@@ -315,7 +420,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     private void checkQuery(String qry) throws Exception {
         Prepared prepared = parse(qry);
 
-        GridSqlQuery gQry = new GridSqlQueryParser().parse(prepared);
+        GridSqlStatement gQry = new GridSqlQueryParser().parse(prepared);
 
         String res = gQry.getSQL();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 7f98d0a..783f831 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgume
 import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheCrossCacheJoinRandomTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDeleteSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCollocatedAndNotTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCustomAffinityMapper;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinPartitionedAndReplicatedTest;
@@ -37,9 +38,11 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinQue
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinNoIndexTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheInsertSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheJoinPartitionedAndReplicatedTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheJoinQueryWithAffinityKeyTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheMergeSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapEvictQueryTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapIndexScanTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
@@ -54,6 +57,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreaded
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCrossCachesJoinsQueryTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicQuerySelfTest;
@@ -148,6 +152,11 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheCrossCacheJoinRandomTest.class);
         suite.addTestSuite(IgniteCacheDistributedJoinCustomAffinityMapper.class);
 
+        suite.addTestSuite(IgniteCacheMergeSqlQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheInsertSqlQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheUpdateSqlQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheDeleteSqlQuerySelfTest.class);
+
         suite.addTestSuite(IgniteBinaryObjectQueryArgumentsTest.class);
         suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapTest.class);
         suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapLocalTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/config/benchmark-bin-identity.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-bin-identity.properties b/modules/yardstick/config/benchmark-bin-identity.properties
new file mode 100644
index 0000000..3d754c5
--- /dev/null
+++ b/modules/yardstick/config/benchmark-bin-identity.properties
@@ -0,0 +1,94 @@
+# 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.
+
+#
+# Contains all multicast benchmarks
+#
+
+now0=`date +'%H%M%S'`
+
+# JVM options.
+JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+JVM_OPTS=${JVM_OPTS}" \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
+-XX:+UseTLAB \
+-XX:NewSize=128m \
+-XX:MaxNewSize=128m \
+-XX:MaxTenuringThreshold=0 \
+-XX:SurvivorRatio=1024 \
+-XX:+UseCMSInitiatingOccupancyOnly \
+-XX:CMSInitiatingOccupancyFraction=60 \
+"
+
+#Ignite version
+ver="RELEASE-"
+
+# List of default probes.
+# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
+BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe,DStatProbe
+
+# Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
+
+# Restart servers for each benchmark.
+RESTART_SERVERS=true
+
+# Probe point writer class name.
+# BENCHMARK_WRITER=
+
+# Comma-separated list of the hosts to run BenchmarkServers on.
+SERVER_HOSTS=localhost,localhost,localhost
+
+# Comma-separated list of the hosts to run BenchmarkDrivers on.
+DRIVER_HOSTS=localhost
+
+# Remote username.
+# REMOTE_USER=
+
+# Number of nodes, used to wait for the specified number of nodes to start.
+nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
+
+# Backups count.
+b=1
+
+# Warmup.
+w=60
+
+# Duration.
+d=300
+
+# Threads count.
+t=64
+
+# Sync mode.
+sm=PRIMARY_SYNC
+
+# Jobs.
+j=10
+
+# Run configuration which contains all benchmarks.
+# Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute).
+CONFIGS="\
+-cfg ${SCRIPT_DIR}/../config/ignite-bin-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteFieldsBinaryIdentityGetBenchmark -sn IgniteNode -ds ${ver}fields-bin-id-atomic-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-bin-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteFieldsBinaryIdentityPutBenchmark -sn IgniteNode -ds ${ver}fields-bin-id-atomic-put-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-bin-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteLegacyBinaryIdentityGetBenchmark -sn IgniteNode -ds ${ver}legacy-bin-id-atomic-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-bin-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteLegacyBinaryIdentityPutBenchmark -sn IgniteNode -ds ${ver}legacy-bin-id-atomic-put-1-backup\
+"

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/config/benchmark-multicast.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties
index 0428c73..3b31745 100644
--- a/modules/yardstick/config/benchmark-multicast.properties
+++ b/modules/yardstick/config/benchmark-multicast.properties
@@ -117,6 +117,21 @@ CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllBenchmark -sn IgniteNode -ds ${ver}atomic-putAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-putAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-putAllSerializable-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgniteSqlMergeAllBenchmark -sn IgniteNode -ds ${ver}sql-merge-all-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteReplaceIndexedValue1Benchmark -sn IgniteNode -ds ${ver}replace-indexed1-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutIfAbsentIndexedValue1Benchmark -sn IgniteNode -ds ${ver}put-if-absent-indexed1-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlMergeBenchmark -sn IgniteNode -ds ${ver}sql-merge-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlMergeQueryBenchmark -sn IgniteNode -ds ${ver}sql-merge-query-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlMergeIndexedValue1Benchmark -sn IgniteNode -ds ${ver}sql-merge-indexed1-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlMergeIndexedValue2Benchmark -sn IgniteNode -ds ${ver}sql-merge-indexed2-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlMergeIndexedValue8Benchmark -sn IgniteNode -ds ${ver}sql-merge-indexed8-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlInsertIndexedValue1Benchmark -sn IgniteNode -ds ${ver}sql-insert-indexed1-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlInsertIndexedValue2Benchmark -sn IgniteNode -ds ${ver}sql-insert-indexed2-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlInsertIndexedValue8Benchmark -sn IgniteNode -ds ${ver}sql-insert-indexed8-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -r 300000 -dn IgniteSqlDeleteBenchmark -sn IgniteNode -ds ${ver}sql-delete-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlDeleteFilteredBenchmark -sn IgniteNode -ds ${ver}sql-delete-filtered-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlUpdateBenchmark -sn IgniteNode -ds ${ver}sql-update-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlUpdateFilteredBenchmark -sn IgniteNode -ds ${ver}sql-update-filtered-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC  -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-optimistic-getAllPutAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC  -dn IgniteGetEntriesPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-optimistic-getEntriesPutAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc PESSIMISTIC -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-pessimistic-getAllPutAll-1-backup,\

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/config/benchmark-sql-dml.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-sql-dml.properties b/modules/yardstick/config/benchmark-sql-dml.properties
new file mode 100644
index 0000000..2ce2e1f
--- /dev/null
+++ b/modules/yardstick/config/benchmark-sql-dml.properties
@@ -0,0 +1,72 @@
+# 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.
+
+#
+# Contains benchmarks for SQL queries.
+#
+
+# JVM options.
+# JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
+  -XX:+UseParNewGC \
+  -XX:+UseConcMarkSweepGC \
+  -XX:+UseTLAB \
+  -XX:NewSize=128m \
+  -XX:MaxNewSize=128m \
+  -XX:MaxTenuringThreshold=0 \
+  -XX:SurvivorRatio=1024 \
+  -XX:+UseCMSInitiatingOccupancyOnly \
+  -XX:CMSInitiatingOccupancyFraction=60 \
+"
+
+# List of default probes.
+# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
+BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+
+# Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
+
+# Probe point writer class name.
+# BENCHMARK_WRITER=
+
+# Comma-separated list of the hosts to run BenchmarkServers on. 2 nodes on local host are enabled by default.
+SERVER_HOSTS=localhost,localhost
+
+# Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+DRIVER_HOSTS=localhost
+
+# Remote username.
+# REMOTE_USER=
+
+# Number of nodes, used to wait for the specified number of nodes to start.
+nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
+
+# Run configuration.
+# Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute).
+
+#
+
+CONFIGS="\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 2 -dn IgniteSqlMergeAllBenchmark -sn IgniteNode -ds sql-merge-all-bs-2-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 6 -dn IgniteSqlMergeAllBenchmark -sn IgniteNode -ds sql-merge-all-bs-6-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 10 -dn IgniteSqlMergeAllBenchmark -sn IgniteNode -ds sql-merge-all-bs-10-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlMergeBenchmark -sn IgniteNode -ds sql-merge-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlMergeQueryBenchmark -sn IgniteNode -ds sql-merge-query-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlMergeIndexedValue1Benchmark -sn IgniteNode -ds sql-merge-indexed1-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlMergeIndexedValue2Benchmark -sn IgniteNode -ds sql-merge-indexed2-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlMergeIndexedValue8Benchmark -sn IgniteNode -ds sql-merge-indexed8-1-backup\
+"

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/config/ignite-base-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-base-config.xml b/modules/yardstick/config/ignite-base-config.xml
index 346f507..615cb42 100644
--- a/modules/yardstick/config/ignite-base-config.xml
+++ b/modules/yardstick/config/ignite-base-config.xml
@@ -127,7 +127,14 @@
                 </bean>
 
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="query"/>
+                    <property name="name" value="atomic-index-with-eviction"/>
+
+                    <property name="evictionPolicy">
+                        <bean class="org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy">
+                            <!-- default range (1000000) x (1 + default backups number (1)) / default nodes number (4) -->
+                            <constructor-arg value="500000"/>
+                        </bean>
+                    </property>
 
                     <property name="cacheMode" value="PARTITIONED"/>
 
@@ -135,38 +142,40 @@
 
                     <property name="swapEnabled" value="false"/>
 
-                    <property name="typeMetadata">
+                    <property name="indexedTypes">
                         <list>
-                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
-                                <property name="keyType" value="java.lang.Integer"/>
-                                <property name="valueType" value="org.apache.ignite.yardstick.cache.model.Organization"/>
-
-                                <property name="ascendingFields">
-                                    <map>
-                                        <entry key="id" value="java.lang.Integer"/>
-                                        <entry key="name" value="java.lang.String"/>
-                                    </map>
-                                </property>
-                            </bean>
-                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
-                                <property name="keyType" value="java.lang.Integer"/>
-                                <property name="valueType" value="org.apache.ignite.yardstick.cache.model.Person"/>
-
-                                <property name="ascendingFields">
-                                    <map>
-                                        <entry key="id" value="java.lang.Integer"/>
-                                        <entry key="orgId" value="java.lang.Integer"/>
-                                        <entry key="salary" value="java.lang.Double"/>
-                                    </map>
-                                </property>
-
-                                <property name="queryFields">
-                                    <map>
-                                        <entry key="firstName" value="java.lang.String"/>
-                                        <entry key="lastName" value="java.lang.String"/>
-                                    </map>
-                                </property>
-                            </bean>
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person1</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person2</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person8</value>
+                        </list>
+                    </property>
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="query"/>
+
+                    <property name="cacheMode" value="PARTITIONED"/>
+
+                    <property name="atomicityMode" value="ATOMIC"/>
+
+                    <property name="indexedTypes">
+                        <list>
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Organization</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.SampleValue</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>java.lang.Integer</value>
                         </list>
                     </property>
                 </bean>

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/config/ignite-bin-multicast-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-bin-multicast-config.xml b/modules/yardstick/config/ignite-bin-multicast-config.xml
new file mode 100644
index 0000000..e7eb6a1
--- /dev/null
+++ b/modules/yardstick/config/ignite-bin-multicast-config.xml
@@ -0,0 +1,86 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    Ignite Spring configuration file to startup grid.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.5.xsd">
+    <import resource="ignite-base-config.xml"/>
+
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" parent="base-ignite.cfg">
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder"/>
+                </property>
+            </bean>
+        </property>
+
+        <property name="marshaller">
+            <bean class="org.apache.ignite.internal.binary.BinaryMarshaller"/>
+        </property>
+
+        <property name="cacheKeyConfiguration">
+            <list>
+                <bean class="org.apache.ignite.cache.CacheKeyConfiguration">
+                    <property name="typeName" value="BinaryKeyWithLegacyIdentity"/>
+
+                    <property name="affinityKeyFieldName" value="f3" />
+                </bean>
+
+                <bean class="org.apache.ignite.cache.CacheKeyConfiguration">
+                    <property name="typeName" value="BinaryKeyWithFieldsIdentity"/>
+
+                    <property name="affinityKeyFieldName" value="f3" />
+                </bean>
+            </list>
+        </property>
+
+        <property name="binaryConfiguration">
+            <bean class="org.apache.ignite.configuration.BinaryConfiguration">
+                <property name="typeConfigurations">
+                    <list>
+                        <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">
+                            <property name="typeName" value="BinaryKeyWithLegacyIdentity"/>
+                        </bean>
+
+                        <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">
+                            <property name="typeName" value="BinaryKeyWithFieldsIdentity"/>
+
+                            <property name="identity">
+                                <bean class="org.apache.ignite.binary.BinaryFieldIdentityResolver">
+                                    <property name="fieldNames">
+                                        <list>
+                                            <value>f1</value>
+                                            <value>f2</value>
+                                            <value>f3</value>
+                                        </list>
+                                    </property>
+                                </bean>
+                            </property>
+                        </bean>
+                    </list>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityBenchmark.java
new file mode 100644
index 0000000..a2e29dd
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityBenchmark.java
@@ -0,0 +1,108 @@
+/*
+ * 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.yardstick.cache;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.yardstickframework.BenchmarkUtils.println;
+
+/**
+ * Ignite benchmark that performs get operations.
+ */
+abstract class IgniteBinaryIdentityBenchmark extends IgniteCacheAbstractBenchmark<BinaryObject, SampleValue> {
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        println(cfg, "Populating data...");
+
+        long start = System.nanoTime();
+
+        try (IgniteDataStreamer<BinaryObject, Object> dataLdr = ignite().dataStreamer(cache.getName())) {
+            for (int i = 0; i < args.range() && !Thread.currentThread().isInterrupted();) {
+                dataLdr.addData(createKey(i), new SampleValue(i));
+
+                if (++i % 100000 == 0)
+                    println(cfg, "Items populated: " + i);
+            }
+        }
+
+        println(cfg, "Finished populating data in " + ((System.nanoTime() - start) / 1_000_000) + " ms.");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<BinaryObject, SampleValue> cache() {
+        return ignite().cache("atomic").withKeepBinary();
+    }
+
+    /**
+     * @param key Base key value.
+     * @return Binary key.
+     */
+    abstract BinaryObject createKey(int key);
+
+    /**
+     * @param key Key field value.
+     * @return Binary object without hash code explicitly set at build time.
+     */
+    BinaryObject createFieldsIdentityBinaryKey(int key) {
+        BinaryObjectBuilder bldr = ignite().binary().builder("BinaryKeyWithFieldsIdentity");
+
+        setBuilderFields(bldr, key);
+
+        return bldr.build();
+    }
+
+    /**
+     * @param key Key field value.
+     * @return Binary object with hash code explicitly set at build time.
+     */
+    @SuppressWarnings("ResultOfMethodCallIgnored")
+    BinaryObject createLegacyIdentityBinaryKey(int key) {
+        BinaryObjectBuilder bldr = ignite().binary().builder("BinaryKeyWithLegacyIdentity");
+
+        setBuilderFields(bldr, key);
+
+        int hash = 0;
+
+        hash = 31 * hash + 1;
+        hash = 31 * hash + ("SomeString".hashCode());
+        hash = 31 * hash + Long.valueOf(key).hashCode();
+
+        bldr.hashCode(hash);
+
+        return bldr.build();
+    }
+
+    /**
+     * @param builder Builder.
+     * @param key Key field value.
+     */
+    private static void setBuilderFields(BinaryObjectBuilder builder, int key) {
+        builder.setField("f1", 1);
+
+        builder.setField("f2", "SomeString");
+
+        builder.setField("f3", (long) key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityGetBenchmark.java
new file mode 100644
index 0000000..1af1846
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityGetBenchmark.java
@@ -0,0 +1,34 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.Map;
+
+/**
+ * Test GETs with binary hashed key.
+ */
+public abstract class IgniteBinaryIdentityGetBenchmark extends IgniteBinaryIdentityBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.get(createKey(key));
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityPutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityPutBenchmark.java
new file mode 100644
index 0000000..58895a0
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityPutBenchmark.java
@@ -0,0 +1,35 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.Map;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+
+/**
+ * Test PUTs with binary hashed key.
+ */
+public abstract class IgniteBinaryIdentityPutBenchmark extends IgniteBinaryIdentityBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.put(createKey(key), new SampleValue(key));
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityGetBenchmark.java
new file mode 100644
index 0000000..333b245
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityGetBenchmark.java
@@ -0,0 +1,30 @@
+/*
+ * 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.yardstick.cache;
+
+import org.apache.ignite.binary.BinaryObject;
+
+/**
+ * Test GETs with binary hashed key.
+ */
+public class IgniteFieldsBinaryIdentityGetBenchmark extends IgniteBinaryIdentityGetBenchmark {
+    /** {@inheritDoc} */
+    @Override BinaryObject createKey(int key) {
+        return createFieldsIdentityBinaryKey(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityPutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityPutBenchmark.java
new file mode 100644
index 0000000..c905845
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityPutBenchmark.java
@@ -0,0 +1,30 @@
+/*
+ * 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.yardstick.cache;
+
+import org.apache.ignite.binary.BinaryObject;
+
+/**
+ * Test PUTs with binary hashed key.
+ */
+public class IgniteFieldsBinaryIdentityPutBenchmark extends IgniteBinaryIdentityPutBenchmark {
+    /** {@inheritDoc} */
+    @Override BinaryObject createKey(int key) {
+        return createFieldsIdentityBinaryKey(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityGetBenchmark.java
new file mode 100644
index 0000000..1b36b24
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityGetBenchmark.java
@@ -0,0 +1,30 @@
+/*
+ * 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.yardstick.cache;
+
+import org.apache.ignite.binary.BinaryObject;
+
+/**
+ * Test GETs with binary hashed key.
+ */
+public class IgniteLegacyBinaryIdentityGetBenchmark extends IgniteBinaryIdentityGetBenchmark {
+    /** {@inheritDoc} */
+    @Override BinaryObject createKey(int key) {
+        return createLegacyIdentityBinaryKey(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityPutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityPutBenchmark.java
new file mode 100644
index 0000000..246fc28
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityPutBenchmark.java
@@ -0,0 +1,30 @@
+/*
+ * 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.yardstick.cache;
+
+import org.apache.ignite.binary.BinaryObject;
+
+/**
+ * Test PUTs with binary hashed key.
+ */
+public class IgniteLegacyBinaryIdentityPutBenchmark extends IgniteBinaryIdentityPutBenchmark {
+    /** {@inheritDoc} */
+    @Override BinaryObject createKey(int key) {
+        return createLegacyIdentityBinaryKey(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIfAbsentIndexedValue1Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIfAbsentIndexedValue1Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIfAbsentIndexedValue1Benchmark.java
new file mode 100644
index 0000000..aea909a
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIfAbsentIndexedValue1Benchmark.java
@@ -0,0 +1,45 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.yardstick.cache.model.Person1;
+
+/**
+ * Ignite benchmark that performs putIfAbsent operations for entity with indexed fields.
+ */
+public class IgnitePutIfAbsentIndexedValue1Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private final AtomicInteger insCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = insCnt.getAndIncrement();
+
+        cache.putIfAbsent(key, new Person1(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index-with-eviction");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteReplaceIndexedValue1Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteReplaceIndexedValue1Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteReplaceIndexedValue1Benchmark.java
new file mode 100644
index 0000000..cc50c84
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteReplaceIndexedValue1Benchmark.java
@@ -0,0 +1,79 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.yardstick.cache.model.Person1;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ * Ignite benchmark that performs SQL UPDATE operations.
+ */
+public class IgniteReplaceIndexedValue1Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** {@inheritDoc} */
+    @Override public void setUp(final BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        final AtomicInteger i = new AtomicInteger();
+
+        Collection<Thread> setupThreads = new ArrayList<>(cfg.threads());
+
+        for (int j = 0; j < cfg.threads(); j++) {
+            Thread t = new Thread() {
+                /** {@inheritDoc} */
+                @Override public void run() {
+                    int k;
+
+                    while ((k = i.getAndIncrement()) < args.range()) {
+                        cache().put(k, new Person1(k));
+                        if (++k % 100000 == 0)
+                            BenchmarkUtils.println(cfg, "REPLACE setUp: have successfully put " + k + " items");
+                    }
+                }
+            };
+
+            setupThreads.add(t);
+
+            t.start();
+        }
+
+        for (Thread t : setupThreads)
+            t.join();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        cache.replace(rnd.nextInt(args.range()), new Person1(rnd.nextInt(args.range())));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteBenchmark.java
new file mode 100644
index 0000000..864211c
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteBenchmark.java
@@ -0,0 +1,83 @@
+/*
+ * 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.yardstick.cache.dml;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person1;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ * Ignite benchmark that performs SQL DELETE operations.
+ */
+public class IgniteSqlDeleteBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private final ConcurrentLinkedQueue<Integer> keys = new ConcurrentLinkedQueue<>();
+
+    /** {@inheritDoc} */
+    @Override public void setUp(final BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        final AtomicInteger i = new AtomicInteger();
+
+        Collection<Thread> setupThreads = new ArrayList<>(cfg.threads());
+
+        for (int j = 0; j < cfg.threads(); j++) {
+            Thread t = new Thread() {
+                /** {@inheritDoc} */
+                @Override public void run() {
+                    int k;
+
+                    while ((k = i.getAndIncrement()) < args.range()) {
+                        cache().put(k, new Person1(k));
+                        keys.add(k);
+                        if (k % 100000 == 0)
+                            BenchmarkUtils.println(cfg, "DELETE setUp: have successfully put " + k + " items");
+                    }
+                }
+            };
+
+            setupThreads.add(t);
+
+            t.start();
+        }
+
+        for (Thread t : setupThreads)
+            t.join();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        cache.query(new SqlFieldsQuery("delete from Person1 where _key = ?").setArgs(keys.remove()));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteFilteredBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteFilteredBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteFilteredBenchmark.java
new file mode 100644
index 0000000..218e8dc
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteFilteredBenchmark.java
@@ -0,0 +1,88 @@
+/*
+ * 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.yardstick.cache.dml;
+
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.yardstickframework.BenchmarkUtils.println;
+
+/**
+ * Ignite benchmark that performs put and SQL DELETE operations.
+ */
+public class IgniteSqlDeleteFilteredBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private AtomicInteger putCnt = new AtomicInteger();
+
+    /** */
+    private AtomicInteger delCnt = new AtomicInteger();
+
+    /** */
+    private AtomicInteger delItemsCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        if (rnd.nextBoolean()) {
+            double salary = rnd.nextDouble() * args.range() * 1000;
+
+            double maxSalary = salary + 1000;
+
+            int res = (Integer) cache().query(new SqlFieldsQuery("delete from Person where salary >= ? and salary <= ?")
+                .setArgs(salary, maxSalary)).getAll().get(0).get(0);
+
+            delItemsCnt.getAndAdd(res);
+
+            delCnt.getAndIncrement();
+        }
+        else {
+            int i = rnd.nextInt(args.range());
+
+            cache.put(i, new Person(i, "firstName" + i, "lastName" + i, i * 1000));
+
+            putCnt.getAndIncrement();
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("query");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() throws Exception {
+        println(cfg, "Finished SQL DELETE query benchmark [putCnt=" + putCnt.get() + ", delCnt=" + delCnt.get() +
+            ", delItemsCnt=" + delItemsCnt.get() + ']');
+
+        super.tearDown();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue1Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue1Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue1Benchmark.java
new file mode 100644
index 0000000..63a585d
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue1Benchmark.java
@@ -0,0 +1,48 @@
+/*
+ * 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.yardstick.cache.dml;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person1;
+
+/**
+ * Ignite benchmark that performs SQL INSERT operations for entity with 1 indexed field.
+ */
+public class IgniteSqlInsertIndexedValue1Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private final AtomicInteger insCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = insCnt.getAndIncrement();
+
+        cache.query(new SqlFieldsQuery("insert into Person1(_key, _val) values (?, ?)")
+            .setArgs(key, new Person1(key)));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index-with-eviction");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue2Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue2Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue2Benchmark.java
new file mode 100644
index 0000000..b099951
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue2Benchmark.java
@@ -0,0 +1,48 @@
+/*
+ * 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.yardstick.cache.dml;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person2;
+
+/**
+ * Ignite benchmark that performs SQL INSERT operations for entity with 2 indexed fields.
+ */
+public class IgniteSqlInsertIndexedValue2Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private final AtomicInteger insCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = insCnt.getAndIncrement();
+
+        cache.query(new SqlFieldsQuery("insert into Person2(_key, _val) values (?, ?)")
+            .setArgs(key, new Person2(key)));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index-with-eviction");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue8Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue8Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue8Benchmark.java
new file mode 100644
index 0000000..3193f38
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue8Benchmark.java
@@ -0,0 +1,48 @@
+/*
+ * 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.yardstick.cache.dml;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person8;
+
+/**
+ * Ignite benchmark that performs SQL INSERT operations for entity with 8 indexed fields.
+ */
+public class IgniteSqlInsertIndexedValue8Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private final AtomicInteger insCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = insCnt.incrementAndGet();
+
+        cache.query(new SqlFieldsQuery("insert into Person8(_key, _val) values (?, ?)")
+            .setArgs(key, new Person8(key)));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index-with-eviction");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeAllBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeAllBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeAllBenchmark.java
new file mode 100644
index 0000000..dc8c105
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeAllBenchmark.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.yardstick.cache.dml;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.yardstickframework.BenchmarkConfiguration;
+
+/**
+ * Ignite benchmark that performs putAll operations via SQL MERGE.
+ */
+public class IgniteSqlMergeAllBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** Affinity mapper. */
+    private Affinity<Integer> aff;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        aff = ignite().affinity("query");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        Object[] vals = new Object[args.batch() * 2];
+
+        ClusterNode node = args.collocated() ? aff.mapKeyToNode(nextRandom(args.range())) : null;
+
+        StringBuilder qry = new StringBuilder("merge into Integer(_key, _val) values ");
+
+        int j = 0;
+
+        for (int i = 0; i < args.batch(); ) {
+            int key = nextRandom(args.range());
+
+            if (args.collocated() && !aff.isPrimary(node, key))
+                continue;
+
+            ++i;
+
+            // Put two args, for key and value.
+            vals[j++] = key;
+            vals[j++] = key;
+
+            qry.append("(?, ?),");
+        }
+
+        Arrays.sort(vals);
+
+        // Trim trailing comma.
+        qry = qry.deleteCharAt(qry.length() - 1);
+
+        cache.query(new SqlFieldsQuery(qry.toString()).setArgs(vals));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("query");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeBenchmark.java
new file mode 100644
index 0000000..3b93403
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeBenchmark.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.yardstick.cache.dml;
+
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+
+/**
+ * Ignite benchmark that performs SQL MERGE operations.
+ */
+public class IgniteSqlMergeBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.query(new SqlFieldsQuery("merge into SampleValue(_key, id) values(?, ?)").setArgs(key, key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("query");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue1Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue1Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue1Benchmark.java
new file mode 100644
index 0000000..022e7c8
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue1Benchmark.java
@@ -0,0 +1,43 @@
+/*
+ * 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.yardstick.cache.dml;
+
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person1;
+
+/**
+ * Ignite benchmark that performs SQL MERGE operations for entity with indexed fields.
+ */
+public class IgniteSqlMergeIndexedValue1Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.query(new SqlFieldsQuery("merge into Person1(_key, _val) values (?, ?)").setArgs(key, new Person1(key)));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue2Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue2Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue2Benchmark.java
new file mode 100644
index 0000000..08d3071
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue2Benchmark.java
@@ -0,0 +1,43 @@
+/*
+ * 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.yardstick.cache.dml;
+
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person2;
+
+/**
+ * Ignite benchmark that performs SQL MERGE operations for entity with indexed fields.
+ */
+public class IgniteSqlMergeIndexedValue2Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.query(new SqlFieldsQuery("merge into Person2(_key, _val) values (?, ?)").setArgs(key, new Person2(key)));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue8Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue8Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue8Benchmark.java
new file mode 100644
index 0000000..ba253b7
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeIndexedValue8Benchmark.java
@@ -0,0 +1,43 @@
+/*
+ * 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.yardstick.cache.dml;
+
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person8;
+
+/**
+ * Ignite benchmark that performs SQL MERGE operations for entity with indexed fields.
+ */
+public class IgniteSqlMergeIndexedValue8Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.query(new SqlFieldsQuery("merge into Person8(_key, _val) values (?, ?)").setArgs(key, new Person8(key)));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}


[49/52] ignite git commit: IGNITE-4343: Fixed update logic in DmlStatementsProcessor. This closes #1304.

Posted by vo...@apache.org.
IGNITE-4343: Fixed update logic in DmlStatementsProcessor. This closes #1304.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b2692ada
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b2692ada
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b2692ada

Branch: refs/heads/master
Commit: b2692adace32b7308ce53d5f93bd56c506eeed36
Parents: af24a9e
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Dec 2 12:48:08 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 2 12:48:08 2016 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        | 20 ++++++++++++++++++--
 1 file changed, 18 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b2692ada/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index d57f95f..c2c8726 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -492,6 +492,9 @@ public class DmlStatementsProcessor {
 
             newVal = plan.valSupplier.apply(e);
 
+            if (newVal == null)
+                throw new IgniteSQLException("New value for UPDATE must not be null", IgniteQueryErrorCode.NULL_VALUE);
+
             if (bin && !(val instanceof BinaryObject))
                 val = cctx.grid().binary().toBinary(val);
 
@@ -858,7 +861,7 @@ public class DmlStatementsProcessor {
 
         /** {@inheritDoc} */
         @Override public Boolean process(MutableEntry<Object, Object> entry, Object... arguments) throws EntryProcessorException {
-            if (entry.getValue() != null)
+            if (entry.exists())
                 return false;
 
             entry.setValue(val);
@@ -878,17 +881,28 @@ public class DmlStatementsProcessor {
 
         /** */
         private ModifyingEntryProcessor(Object val, IgniteInClosure<MutableEntry<Object, Object>> entryModifier) {
+            assert val != null;
+
             this.val = val;
             this.entryModifier = entryModifier;
         }
 
         /** {@inheritDoc} */
         @Override public Boolean process(MutableEntry<Object, Object> entry, Object... arguments) throws EntryProcessorException {
+            if (!entry.exists())
+                return null; // Someone got ahead of us and removed this entry, let's skip it.
+
+            Object entryVal = entry.getValue();
+
+            if (entryVal == null)
+                return null;
+
             // Something happened to the cache while we were performing map-reduce.
-            if (!F.eq(entry.getValue(), val))
+            if (!F.eq(entryVal, val))
                 return false;
 
             entryModifier.apply(entry);
+
             return null; // To leave out only erroneous keys - nulls are skipped on results' processing.
         }
     }
@@ -910,6 +924,8 @@ public class DmlStatementsProcessor {
 
         /** */
         private EntryValueUpdater(Object val) {
+            assert val != null;
+
             this.val = val;
         }
 


[11/52] ignite git commit: IGNITE-4016: ODBC: Example now uses DML. This closes #1265.

Posted by vo...@apache.org.
IGNITE-4016: ODBC: Example now uses DML. This closes #1265.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/83363d8e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/83363d8e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/83363d8e

Branch: refs/heads/master
Commit: 83363d8e0cab93b62f674ce0215082364881254b
Parents: f4524a6
Author: isapego <ig...@gmail.com>
Authored: Wed Nov 23 18:07:06 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Nov 23 18:07:06 2016 +0300

----------------------------------------------------------------------
 .../odbc-example/config/example-odbc.xml        |  30 +-
 .../project/vs/odbc-example.vcxproj             |   7 +-
 .../project/vs/odbc-example.vcxproj.filters     |   8 +
 .../examples/odbc-example/src/odbc_example.cpp  | 514 ++++++++++++++++---
 4 files changed, 466 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/83363d8e/modules/platforms/cpp/examples/odbc-example/config/example-odbc.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/odbc-example/config/example-odbc.xml b/modules/platforms/cpp/examples/odbc-example/config/example-odbc.xml
index db4c390..864f950 100644
--- a/modules/platforms/cpp/examples/odbc-example/config/example-odbc.xml
+++ b/modules/platforms/cpp/examples/odbc-example/config/example-odbc.xml
@@ -39,26 +39,30 @@
                     <property name="cacheMode" value="PARTITIONED"/>
                     <property name="atomicityMode" value="TRANSACTIONAL"/>
                     <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-            
-                    <!-- Configure type metadata to enable queries. -->
-                    <property name="typeMetadata">
+
+                    <property name="queryEntities">
                         <list>
-                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                            <bean class="org.apache.ignite.cache.QueryEntity">
                                 <property name="keyType" value="java.lang.Long"/>
                                 <property name="valueType" value="Person"/>
-                                <property name="ascendingFields">
+
+                                <property name="fields">
                                     <map>
                                         <entry key="orgId" value="java.lang.Long"/>
-                                    </map>
-                                </property>
-                                <property name="queryFields">
-                                    <map>
                                         <entry key="firstName" value="java.lang.String"/>
                                         <entry key="lastName" value="java.lang.String"/>
                                         <entry key="resume" value="java.lang.String"/>
                                         <entry key="salary" value="java.lang.Double"/>
                                     </map>
                                 </property>
+
+                                <property name="indexes">
+                                    <list>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="orgId"/>
+                                        </bean>
+                                    </list>
+                                </property>
                             </bean>
                         </list>
                     </property>
@@ -70,13 +74,13 @@
                     <property name="atomicityMode" value="TRANSACTIONAL"/>
                     <property name="writeSynchronizationMode" value="FULL_SYNC"/>
             
-                    <!-- Configure type metadata to enable queries. -->
-                    <property name="typeMetadata">
+                    <property name="queryEntities">
                         <list>
-                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                            <bean class="org.apache.ignite.cache.QueryEntity">
                                 <property name="keyType" value="java.lang.Long"/>
                                 <property name="valueType" value="Organization"/>
-                                <property name="ascendingFields">
+
+                                <property name="fields">
                                     <map>
                                         <entry key="name" value="java.lang.String"/>
                                     </map>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83363d8e/modules/platforms/cpp/examples/odbc-example/project/vs/odbc-example.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/odbc-example/project/vs/odbc-example.vcxproj b/modules/platforms/cpp/examples/odbc-example/project/vs/odbc-example.vcxproj
index e7bdd08..9f7cf3a 100644
--- a/modules/platforms/cpp/examples/odbc-example/project/vs/odbc-example.vcxproj
+++ b/modules/platforms/cpp/examples/odbc-example/project/vs/odbc-example.vcxproj
@@ -54,7 +54,7 @@
       <Optimization>MaxSpeed</Optimization>
       <FunctionLevelLinking>true</FunctionLevelLinking>
       <IntrinsicFunctions>true</IntrinsicFunctions>
-      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;_CRT_SECURE_NO_WARNINGS;%(PreprocessorDefinitions)</PreprocessorDefinitions>
       <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;..\..\..\include;..\..\..\..\common\os\win\include;..\..\..\..\common\include;..\..\..\..\jni\os\win\include;..\..\..\..\jni\include;..\..\..\..\binary\include;..\..\..\..\core\os\win\include;..\..\..\..\core\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
     </ClCompile>
     <Link>
@@ -78,7 +78,7 @@ copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.co
       <Optimization>MaxSpeed</Optimization>
       <FunctionLevelLinking>true</FunctionLevelLinking>
       <IntrinsicFunctions>true</IntrinsicFunctions>
-      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;_CRT_SECURE_NO_WARNINGS;%(PreprocessorDefinitions)</PreprocessorDefinitions>
       <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;..\..\..\include;..\..\..\..\common\os\win\include;..\..\..\..\common\include;..\..\..\..\jni\os\win\include;..\..\..\..\jni\include;..\..\..\..\binary\include;..\..\..\..\core\os\win\include;..\..\..\..\core\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
     </ClCompile>
     <Link>
@@ -102,6 +102,9 @@ copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.co
     <ClInclude Include="..\..\..\include\ignite\examples\organization.h" />
     <ClInclude Include="..\..\..\include\ignite\examples\person.h" />
   </ItemGroup>
+  <ItemGroup>
+    <None Include="..\..\config\example-odbc.xml" />
+  </ItemGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
   <ImportGroup Label="ExtensionTargets">
   </ImportGroup>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83363d8e/modules/platforms/cpp/examples/odbc-example/project/vs/odbc-example.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/odbc-example/project/vs/odbc-example.vcxproj.filters b/modules/platforms/cpp/examples/odbc-example/project/vs/odbc-example.vcxproj.filters
index 44fff7c..aa4427f 100644
--- a/modules/platforms/cpp/examples/odbc-example/project/vs/odbc-example.vcxproj.filters
+++ b/modules/platforms/cpp/examples/odbc-example/project/vs/odbc-example.vcxproj.filters
@@ -8,6 +8,9 @@
     <Filter Include="Header Files">
       <UniqueIdentifier>{6400d7ba-6390-4cdb-aa25-9525f8a71444}</UniqueIdentifier>
     </Filter>
+    <Filter Include="Config">
+      <UniqueIdentifier>{d054b826-ae5e-4a0c-833f-ae0b975f57d8}</UniqueIdentifier>
+    </Filter>
   </ItemGroup>
   <ItemGroup>
     <ClCompile Include="..\..\src\odbc_example.cpp">
@@ -25,4 +28,9 @@
       <Filter>Header Files</Filter>
     </ClInclude>
   </ItemGroup>
+  <ItemGroup>
+    <None Include="..\..\config\example-odbc.xml">
+      <Filter>Config</Filter>
+    </None>
+  </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/83363d8e/modules/platforms/cpp/examples/odbc-example/src/odbc_example.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/odbc-example/src/odbc_example.cpp b/modules/platforms/cpp/examples/odbc-example/src/odbc_example.cpp
index 39648d2..38e7426 100644
--- a/modules/platforms/cpp/examples/odbc-example/src/odbc_example.cpp
+++ b/modules/platforms/cpp/examples/odbc-example/src/odbc_example.cpp
@@ -35,7 +35,6 @@
 #include "ignite/examples/organization.h"
 
 using namespace ignite;
-using namespace cache;
 
 using namespace examples;
 
@@ -48,6 +47,16 @@ using namespace examples;
  *
  * After all pre-requirements are fulfilled just build project as described
  * in README and run resulting file.
+ *
+ * Note, that all fields which used in queries must be listed in config file
+ * under queryEntities property of the caches. You can find config file in
+ * config directory: cpp/examples/odbc-example/config/example-odbc.xml
+ *
+ * In addition to all the fields listed under QueryEntity bean, each table
+ * have two special predefined fields: _key and _val, which represent links
+ * to whole key and value objects. In some queries _key column is used. Key
+ * in our case works like an ID for the row and it should always present in
+ * INSERT statements.
  */
 
 /** Read buffer size. */
@@ -118,101 +127,424 @@ std::string GetOdbcErrorMessage(SQLSMALLINT handleType, SQLHANDLE handle)
 }
 
 /**
+ * Extract error from ODBC handle and throw it as IgniteError.
+ *
+ * @param handleType Type of the handle.
+ * @param handle Handle.
+ * @param msg Error message.
+ */
+void ThrowOdbcError(SQLSMALLINT handleType, SQLHANDLE handle, std::string msg)
+{
+    std::stringstream builder;
+
+    builder << msg << ": " << GetOdbcErrorMessage(handleType, handle);
+
+    std::string errorMsg = builder.str();
+
+    throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, errorMsg.c_str());
+}
+
+/**
  * Fetch cache data using ODBC interface.
  */
-void GetDataWithOdbc(const std::string& query)
+void GetDataWithOdbc(SQLHDBC dbc, const std::string& query)
 {
-    SQLHENV env;
+    SQLHSTMT stmt;
 
-    // Allocate an environment handle
-    SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
+    // Allocate a statement handle
+    SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
 
-    // We want ODBC 3 support
-    SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast<void*>(SQL_OV_ODBC3), 0);
+    std::vector<SQLCHAR> buf(query.begin(), query.end());
 
-    SQLHDBC dbc;
+    SQLRETURN ret = SQLExecDirect(stmt, &buf[0], static_cast<SQLSMALLINT>(buf.size()));
 
-    // Allocate a connection handle
-    SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc);
+    if (SQL_SUCCEEDED(ret))
+        PrintOdbcResultSet(stmt);
+    else
+        std::cerr << "Failed to execute query: " << GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt) << std::endl;
 
-    // Combining connect string
-    std::string connectStr = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=Person;";
+    // Releasing statement handle.
+    SQLFreeHandle(SQL_HANDLE_STMT, stmt);
+}
 
-    SQLCHAR outstr[ODBC_BUFFER_SIZE];
-    SQLSMALLINT outstrlen;
+/**
+ * Populate Person cache with sample data.
+ * 
+ * @param dbc Database connection.
+ */
+void PopulatePerson(SQLHDBC dbc)
+{
+    SQLHSTMT stmt;
 
-    // Connecting to ODBC server.
-    SQLRETURN ret = SQLDriverConnect(dbc, NULL, reinterpret_cast<SQLCHAR*>(&connectStr[0]),
-        static_cast<SQLSMALLINT>(connectStr.size()), outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE);
+    // Allocate a statement handle
+    SQLRETURN ret = SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
 
     if (!SQL_SUCCEEDED(ret))
+        ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to allocate statement handle");
+
+    try
     {
-        std::cerr << "Failed to connect: " << GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc) << std::endl;
+        SQLCHAR query[] =
+            "INSERT INTO Person (_key, orgId, firstName, lastName, resume, salary) "
+            "VALUES (?, ?, ?, ?, ?, ?)";
+
+        ret = SQLPrepare(stmt, query, static_cast<SQLSMALLINT>(sizeof(query)));
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to prepare query");
+
+        // Binding columns.
+
+        int64_t key = 0;
+        int64_t orgId = 0;
+        char firstName[1024] = { 0 };
+        SQLLEN firstNameLen = SQL_NTS;
+        char lastName[1024] = { 0 };
+        SQLLEN lastNameLen = SQL_NTS;
+        char resume[1024] = { 0 };
+        SQLLEN resumeLen = SQL_NTS;
+        double salary = 0.0;
+
+        ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to bind parameter");
+
+        ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &orgId, 0, 0);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to bind parameter");
+
+        ret = SQLBindParameter(stmt, 3, SQL_PARAM_INPUT, SQL_C_CHAR, SQL_VARCHAR,
+            sizeof(firstName), sizeof(firstName), firstName, 0, &firstNameLen);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to bind parameter");
+
+        ret = SQLBindParameter(stmt, 4, SQL_PARAM_INPUT, SQL_C_CHAR, SQL_VARCHAR,
+            sizeof(lastName), sizeof(lastName), lastName, 0, &lastNameLen);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to bind parameter");
+
+        ret = SQLBindParameter(stmt, 5, SQL_PARAM_INPUT, SQL_C_CHAR, SQL_VARCHAR,
+            sizeof(resume), sizeof(resume), resume, 0, &resumeLen);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to bind parameter");
+
+        ret = SQLBindParameter(stmt, 6, SQL_PARAM_INPUT, SQL_C_DOUBLE, SQL_DOUBLE, 0, 0, &salary, 0, 0);
 
-        return;
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to bind parameter");
+
+        // Filling cache.
+
+        key = 1;
+        orgId = 1;
+        strncpy(firstName, "John", sizeof(firstName));
+        strncpy(lastName, "Doe", sizeof(lastName));
+        strncpy(resume, "Master Degree.", sizeof(resume));
+        salary = 2200.0;
+
+        ret = SQLExecute(stmt);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute prepared statement");
+
+        ret = SQLMoreResults(stmt);
+
+        if (ret != SQL_NO_DATA)
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "No more data expected");
+
+        ++key;
+        orgId = 1;
+        strncpy(firstName, "Jane", sizeof(firstName));
+        strncpy(lastName, "Doe", sizeof(lastName));
+        strncpy(resume, "Bachelor Degree.", sizeof(resume));
+        salary = 1300.0;
+
+        ret = SQLExecute(stmt);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute prepared statement");
+
+
+        ret = SQLMoreResults(stmt);
+
+        if (ret != SQL_NO_DATA)
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "No more data expected");
+
+        ++key;
+        orgId = 2;
+        strncpy(firstName, "John", sizeof(firstName));
+        strncpy(lastName, "Smith", sizeof(lastName));
+        strncpy(resume, "Bachelor Degree.", sizeof(resume));
+        salary = 1700.0;
+
+        ret = SQLExecute(stmt);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute prepared statement");
+
+        ret = SQLMoreResults(stmt);
+
+        if (ret != SQL_NO_DATA)
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "No more data expected");
+
+        ++key;
+        orgId = 2;
+        strncpy(firstName, "Jane", sizeof(firstName));
+        strncpy(lastName, "Smith", sizeof(lastName));
+        strncpy(resume, "Master Degree.", sizeof(resume));
+        salary = 2500.0;
+
+        ret = SQLExecute(stmt);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute prepared statement");
+
+        ret = SQLMoreResults(stmt);
+
+        if (ret != SQL_NO_DATA)
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "No more data expected");
+
+        ++key;
+        orgId = 2;
+        strncpy(firstName, "John", sizeof(firstName));
+        strncpy(lastName, "Roe", sizeof(lastName));
+        strncpy(resume, "Bachelor Degree.", sizeof(resume));
+        salary = 1500.0;
+
+        ret = SQLExecute(stmt);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute prepared statement");
+
+        ret = SQLMoreResults(stmt);
+
+        if (ret != SQL_NO_DATA)
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "No more data expected");
+
+        ++key;
+        orgId = 2;
+        strncpy(firstName, "Jane", sizeof(firstName));
+        strncpy(lastName, "Roe", sizeof(lastName));
+        strncpy(resume, "Bachelor Degree.", sizeof(resume));
+        salary = 1000.0;
+
+        ret = SQLExecute(stmt);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute prepared statement");
+
+        ret = SQLMoreResults(stmt);
+
+        if (ret != SQL_NO_DATA)
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "No more data expected");
+
+        ++key;
+        orgId = 1;
+        strncpy(firstName, "Richard", sizeof(firstName));
+        strncpy(lastName, "Miles", sizeof(lastName));
+        strncpy(resume, "Master Degree.", sizeof(resume));
+        salary = 2400.0;
+
+        ret = SQLExecute(stmt);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute prepared statement");
+
+        ret = SQLMoreResults(stmt);
+
+        if (ret != SQL_NO_DATA)
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "No more data expected");
+
+        ++key;
+        orgId = 2;
+        strncpy(firstName, "Mary", sizeof(firstName));
+        strncpy(lastName, "Major", sizeof(lastName));
+        strncpy(resume, "Bachelor Degree.", sizeof(resume));
+        salary = 900.0;
+
+        ret = SQLExecute(stmt);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute prepared statement");
+    }
+    catch(...)
+    {
+        // Releasing statement handle.
+        SQLFreeHandle(SQL_HANDLE_STMT, stmt);
+
+        // Re-throwing expection.
+        throw;
     }
 
+    // Releasing statement handle.
+    SQLFreeHandle(SQL_HANDLE_STMT, stmt);
+}
+
+/**
+ * Populate Organization cache with sample data.
+ * 
+ * @param dbc Database connection.
+ */
+void PopulateOrganization(SQLHDBC dbc)
+{
     SQLHSTMT stmt;
 
     // Allocate a statement handle
-    SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
+    SQLRETURN ret = SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
 
-    std::vector<SQLCHAR> buf(query.begin(), query.end());
+    if (!SQL_SUCCEEDED(ret))
+        ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to allocate statement handle");
 
-    ret = SQLExecDirect(stmt, &buf[0], static_cast<SQLSMALLINT>(buf.size()));
+    try
+    {
+        SQLCHAR query1[] = "INSERT INTO \"Organization\".Organization (_key, name) VALUES (1L, 'Microsoft')";
 
-    if (SQL_SUCCEEDED(ret))
-        PrintOdbcResultSet(stmt);
-    else
-        std::cerr << "Failed to execute query: " << GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt) << std::endl;
+        ret = SQLExecDirect(stmt, query1, static_cast<SQLSMALLINT>(sizeof(query1)));
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute query");
+
+        SQLFreeStmt(stmt, SQL_CLOSE);
+
+        SQLCHAR query2[] = "INSERT INTO \"Organization\".Organization (_key, name) VALUES (2L, 'Red Cross')";
+
+        ret = SQLExecDirect(stmt, query2, static_cast<SQLSMALLINT>(sizeof(query2)));
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute query");
+    }
+    catch (...)
+    {
+        // Releasing statement handle.
+        SQLFreeHandle(SQL_HANDLE_STMT, stmt);
+
+        // Re-throwing expection.
+        throw;
+    }
 
     // Releasing statement handle.
     SQLFreeHandle(SQL_HANDLE_STMT, stmt);
+}
+
+/**
+ * Adjust salary for specified employee.
+ *
+ * @param dbc Database connection.
+ * @param key Person key.
+ * @param salary New salary.
+ */
+void AdjustSalary(SQLHDBC dbc, int64_t key, double salary)
+{
+    SQLHSTMT stmt;
+
+    // Allocate a statement handle
+    SQLRETURN ret = SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
+
+    if (!SQL_SUCCEEDED(ret))
+        ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to allocate statement handle");
+
+    try
+    {
+        SQLCHAR query[] = "UPDATE Person SET salary=? WHERE _key=?";
+
+        ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_DOUBLE, SQL_DOUBLE, 0, 0, &salary, 0, 0);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to bind parameter");
+
+        ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to bind parameter");
+
+        ret = SQLExecDirect(stmt, query, static_cast<SQLSMALLINT>(sizeof(query)));
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute query");
+    }
+    catch (...)
+    {
+        // Releasing statement handle.
+        SQLFreeHandle(SQL_HANDLE_STMT, stmt);
 
-    // Disconneting from the server.
-    SQLDisconnect(dbc);
+        // Re-throwing expection.
+        throw;
+    }
 
-    // Releasing allocated handles.
-    SQLFreeHandle(SQL_HANDLE_DBC, dbc);
-    SQLFreeHandle(SQL_HANDLE_ENV, env);
+    // Releasing statement handle.
+    SQLFreeHandle(SQL_HANDLE_STMT, stmt);
 }
 
 /**
- * Populate Person cache with sample data.
- * 
- * @param cache Cache instance.
+ * Remove specified person.
+ *
+ * @param dbc Database connection.
+ * @param key Person key.
  */
-void Populate(Cache<int64_t, Person>& cache)
+void DeletePerson(SQLHDBC dbc, int64_t key)
 {
-    std::map<int64_t, Person> persons;
-
-    int64_t key = 0;
-    persons[++key] = Person(1, "John", "Doe", "Master Degree.", 2200.0);
-    persons[++key] = Person(1, "Jane", "Doe", "Bachelor Degree.", 1300.0);
-    persons[++key] = Person(2, "John", "Smith", "Bachelor Degree.", 1700.0);
-    persons[++key] = Person(2, "Jane", "Smith", "Master Degree.", 2500.0);
-    persons[++key] = Person(2, "John", "Roe", "Bachelor Degree.", 1500.0);
-    persons[++key] = Person(2, "Jane", "Roe", "Bachelor Degree.", 1000.0);
-    persons[++key] = Person(1, "Richard", "Miles", "Master Degree.", 2400.0);
-    persons[++key] = Person(2, "Mary", "Major", "Bachelor Degree.", 900.0);
-
-    cache.PutAll(persons);
+    SQLHSTMT stmt;
+
+    // Allocate a statement handle
+    SQLRETURN ret = SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
+
+    if (!SQL_SUCCEEDED(ret))
+        ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to allocate statement handle");
+
+    try
+    {
+        SQLCHAR query[] = "DELETE FROM Person WHERE _key=?";
+
+        ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to bind parameter");
+
+        ret = SQLExecDirect(stmt, query, static_cast<SQLSMALLINT>(sizeof(query)));
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_STMT, stmt, "Failed to execute query");
+    }
+    catch (...)
+    {
+        // Releasing statement handle.
+        SQLFreeHandle(SQL_HANDLE_STMT, stmt);
+
+        // Re-throwing expection.
+        throw;
+    }
+
+    // Releasing statement handle.
+    SQLFreeHandle(SQL_HANDLE_STMT, stmt);
 }
 
 /**
- * Populate Organization cache with sample data.
+ * Query tables.
  * 
- * @param cache Cache instance.
+ * @param dbc Database connection.
  */
-void Populate(Cache<int64_t, Organization>& cache)
+void QueryData(SQLHDBC dbc)
 {
-    std::map<int64_t, Organization> orgs;
+    std::cout << std::endl;
+    std::cout << ">>> Getting list of persons:" << std::endl;
 
-    int64_t key = 0;
-    orgs[++key] = Organization("Microsoft", Address("1096 Eddy Street, San Francisco, CA", 94109));
-    orgs[++key] = Organization("Red Cross", Address("184 Fidler Drive, San Antonio, TX", 78205));
+    GetDataWithOdbc(dbc, "SELECT firstName, lastName, resume, salary FROM Person");
+
+    std::cout << std::endl;
+    std::cout << ">>> Getting average salary by degree:" << std::endl;
+
+    GetDataWithOdbc(dbc, "SELECT resume, AVG(salary) FROM Person GROUP BY resume");
+
+    std::cout << std::endl;
+    std::cout << ">>> Getting people with organizations:" << std::endl;
 
-    cache.PutAll(orgs);
+    GetDataWithOdbc(dbc, "SELECT firstName, lastName, Organization.name FROM Person "
+        "INNER JOIN \"Organization\".Organization ON Person.orgId = Organization._KEY");
 }
 
 /**
@@ -225,7 +557,7 @@ int main()
     IgniteConfiguration cfg;
 
     cfg.jvmInitMem = 512;
-    cfg.jvmMaxMem = 512;
+    cfg.jvmMaxMem = 1024;
 
     cfg.springCfgPath = "platforms/cpp/examples/odbc-example/config/example-odbc.xml";
 
@@ -234,39 +566,65 @@ int main()
         // Start a node.
         Ignite grid = Ignition::Start(cfg);
 
-        std::cout << std::endl;
-        std::cout << ">>> Cache ODBC example started." << std::endl;
-        std::cout << std::endl;
+        SQLHENV env;
 
-        // Get Person cache instance.
-        Cache<int64_t, Person> personCache = grid.GetCache<int64_t, Person>("Person");
+        // Allocate an environment handle
+        SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
 
-        // Get Organization cache instance.
-        Cache<int64_t, Organization> orgCache = grid.GetCache<int64_t, Organization>("Organization");
+        // We want ODBC 3 support
+        SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast<void*>(SQL_OV_ODBC3), 0);
 
-        // Clear caches.
-        personCache.Clear();
-        orgCache.Clear();
+        SQLHDBC dbc;
 
-        // Populate caches.
-        Populate(personCache);
-        Populate(orgCache);
+        // Allocate a connection handle
+        SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc);
+
+        // Combining connect string
+        std::string connectStr = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=Person;";
+
+        SQLCHAR outstr[ODBC_BUFFER_SIZE];
+        SQLSMALLINT outstrlen;
+
+        // Connecting to ODBC server.
+        SQLRETURN ret = SQLDriverConnect(dbc, NULL, reinterpret_cast<SQLCHAR*>(&connectStr[0]),
+            static_cast<SQLSMALLINT>(connectStr.size()), outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE);
+
+        if (!SQL_SUCCEEDED(ret))
+            ThrowOdbcError(SQL_HANDLE_DBC, dbc, "Failed to connect");
 
         std::cout << std::endl;
-        std::cout << ">>> Getting list of persons:" << std::endl;
+        std::cout << ">>> Cache ODBC example started." << std::endl;
+        std::cout << std::endl;
+
+        // Populate caches.
+        PopulatePerson(dbc);
+        PopulateOrganization(dbc);
 
-        GetDataWithOdbc("SELECT firstName, lastName, resume, salary FROM Person");
+        QueryData(dbc);
 
         std::cout << std::endl;
-        std::cout << ">>> Getting average salary by degree:" << std::endl;
+        std::cout << std::endl;
+        std::cout << ">>> Adjusted salary for Mary Major. Querying again." << std::endl;
+
+        AdjustSalary(dbc, 8, 1200.0);
 
-        GetDataWithOdbc("SELECT resume, AVG(salary) FROM Person GROUP BY resume");
+        QueryData(dbc);
 
         std::cout << std::endl;
-        std::cout << ">>> Getting people with organizations:" << std::endl;
+        std::cout << std::endl;
+        std::cout << ">>> Removing several employees. Querying again." << std::endl;
+
+        DeletePerson(dbc, 4);
+        DeletePerson(dbc, 5);
+
+        QueryData(dbc);
+
+        // Disconneting from the server.
+        SQLDisconnect(dbc);
 
-        GetDataWithOdbc("SELECT firstName, lastName, Organization.name FROM Person "
-            "INNER JOIN \"Organization\".Organization ON Person.orgId = Organization._KEY");
+        // Releasing allocated handles.
+        SQLFreeHandle(SQL_HANDLE_DBC, dbc);
+        SQLFreeHandle(SQL_HANDLE_ENV, env);
 
         // Stop node.
         Ignition::StopAll(false);
@@ -277,7 +635,7 @@ int main()
     }
 
     std::cout << std::endl;
-    std::cout << ">>> Example finished, press any key to exit ..." << std::endl;
+    std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl;
     std::cout << std::endl;
 
     std::cin.get();


[13/52] ignite git commit: Merge branch 'ignite-1.7.4' into 'ignite-1.8'.

Posted by vo...@apache.org.
Merge branch 'ignite-1.7.4' into 'ignite-1.8'.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/e43ea5ee
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/e43ea5ee
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/e43ea5ee

Branch: refs/heads/master
Commit: e43ea5ee9b6982b2b2086fd2a678882d24c1654c
Parents: 4d24325 bf33025
Author: Andrey Novikov <an...@gridgain.com>
Authored: Thu Nov 24 17:12:07 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Thu Nov 24 17:12:07 2016 +0700

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   7 +
 .../apache/ignite/internal/util/GridUnsafe.java |   9 +-
 modules/web-console/backend/app/agent.js        |  32 ++-
 modules/web-console/backend/app/browser.js      |  13 ++
 modules/web-console/backend/app/mongo.js        |   6 +
 .../backend/config/settings.json.sample         |   7 -
 modules/web-console/backend/index.js            |   6 +-
 modules/web-console/backend/middlewares/host.js |   7 +-
 modules/web-console/backend/routes/agent.js     |   4 +-
 modules/web-console/backend/services/agents.js  |   2 +-
 modules/web-console/backend/services/caches.js  |   2 +
 .../docker/compose/backend/.dockerignore        |   2 +
 .../docker/compose/backend/Dockerfile           |   6 +-
 .../web-console/docker/compose/backend/build.sh |   2 +-
 .../docker/compose/docker-compose.yml           |   5 -
 .../docker/compose/frontend/DockerfileBuild     |   4 +-
 .../docker/compose/frontend/build.sh            |   2 +-
 .../compose/frontend/nginx/web-console.conf     |   9 +
 .../web-console/docker/standalone/.dockerignore |   5 +
 .../web-console/docker/standalone/Dockerfile    |  10 +-
 modules/web-console/docker/standalone/build.sh  |   2 +-
 .../docker/standalone/docker-compose.yml        |   7 +-
 .../docker/standalone/nginx/web-console.conf    |   9 +
 .../frontend/app/data/pom-dependencies.json     |   2 +-
 .../directives/ui-ace-pojos/ui-ace-pojos.jade   |   2 +-
 .../frontend/app/filters/duration.filter.js     |   3 +
 .../helpers/jade/form/form-field-checkbox.jade  |   2 +-
 .../helpers/jade/form/form-field-datalist.jade  |   2 +-
 .../helpers/jade/form/form-field-dropdown.jade  |   2 +-
 .../helpers/jade/form/form-field-number.jade    |   2 +-
 .../helpers/jade/form/form-field-password.jade  |   2 +-
 .../app/helpers/jade/form/form-field-text.jade  |   2 +-
 .../frontend/app/helpers/jade/mixins.jade       |  24 +-
 .../modules/configuration/generator/Beans.js    |   6 +
 .../generator/ConfigurationGenerator.js         | 165 ++++++-------
 .../generator/JavaTransformer.service.js        |  15 +-
 .../generator/Properties.service.js             |  19 ++
 .../generator/SpringTransformer.service.js      |  22 +-
 .../generator/defaults/cache.provider.js        |   8 +
 .../generator/defaults/cluster.provider.js      |   2 +-
 .../app/modules/form/field/tooltip.directive.js |   2 +-
 .../app/modules/form/group/tooltip.directive.js |   2 +-
 .../app/modules/form/panel/field.directive.js   |   4 +-
 .../app/modules/sql/scan-filter-input.jade      |   2 +-
 .../frontend/app/modules/sql/sql.controller.js  |   6 +-
 .../configuration/caches/client-near-cache.jade |   2 +-
 .../configuration/caches/concurrency.jade       |   2 +-
 .../states/configuration/caches/general.jade    |   2 +-
 .../states/configuration/caches/memory.jade     |   2 +-
 .../configuration/caches/near-cache-client.jade |   2 +-
 .../configuration/caches/near-cache-server.jade |   2 +-
 .../configuration/caches/node-filter.jade       |   2 +-
 .../states/configuration/caches/query.jade      |   2 +-
 .../states/configuration/caches/rebalance.jade  |   2 +-
 .../states/configuration/caches/statistics.jade |   2 +-
 .../states/configuration/caches/store.jade      |  20 +-
 .../states/configuration/clusters/atomic.jade   |   2 +-
 .../configuration/clusters/attributes.jade      |   2 +-
 .../states/configuration/clusters/binary.jade   |   2 +-
 .../configuration/clusters/cache-key-cfg.jade   |   2 +-
 .../configuration/clusters/checkpoint.jade      |   8 +-
 .../configuration/clusters/checkpoint/fs.jade   |   2 +-
 .../configuration/clusters/checkpoint/jdbc.jade |  41 ++--
 .../configuration/clusters/checkpoint/s3.jade   | 229 ++++++++++---------
 .../configuration/clusters/collision.jade       |  12 +-
 .../clusters/collision/custom.jade              |   2 +-
 .../clusters/collision/fifo-queue.jade          |   2 +-
 .../clusters/collision/job-stealing.jade        |   2 +-
 .../clusters/collision/priority-queue.jade      |   2 +-
 .../configuration/clusters/communication.jade   |   2 +-
 .../configuration/clusters/connector.jade       |   2 +-
 .../configuration/clusters/deployment.jade      |   2 +-
 .../configuration/clusters/discovery.jade       |   2 +-
 .../states/configuration/clusters/events.jade   |   2 +-
 .../states/configuration/clusters/failover.jade |   2 +-
 .../states/configuration/clusters/general.jade  |   2 +-
 .../clusters/general/discovery/cloud.jade       |   2 +-
 .../clusters/general/discovery/google.jade      |   2 +-
 .../clusters/general/discovery/jdbc.jade        |   3 +-
 .../clusters/general/discovery/multicast.jade   |   2 +-
 .../clusters/general/discovery/s3.jade          |   2 +-
 .../clusters/general/discovery/shared.jade      |   2 +-
 .../clusters/general/discovery/vm.jade          |   2 +-
 .../clusters/general/discovery/zookeeper.jade   |   2 +-
 .../bounded-exponential-backoff.jade            |   2 +-
 .../discovery/zookeeper/retrypolicy/custom.jade |   2 +-
 .../retrypolicy/exponential-backoff.jade        |   2 +-
 .../zookeeper/retrypolicy/forever.jade          |   2 +-
 .../zookeeper/retrypolicy/n-times.jade          |   2 +-
 .../zookeeper/retrypolicy/one-time.jade         |   2 +-
 .../zookeeper/retrypolicy/until-elapsed.jade    |   2 +-
 .../states/configuration/clusters/igfs.jade     |   2 +-
 .../configuration/clusters/load-balancing.jade  |   2 +-
 .../states/configuration/clusters/logger.jade   |   2 +-
 .../configuration/clusters/logger/custom.jade   |   2 +-
 .../configuration/clusters/logger/log4j.jade    |   2 +-
 .../configuration/clusters/logger/log4j2.jade   |   2 +-
 .../configuration/clusters/marshaller.jade      |   2 +-
 .../states/configuration/clusters/metrics.jade  |   2 +-
 .../states/configuration/clusters/odbc.jade     |   2 +-
 .../states/configuration/clusters/ssl.jade      |   2 +-
 .../states/configuration/clusters/swap.jade     |   2 +-
 .../states/configuration/clusters/thread.jade   |   2 +-
 .../states/configuration/clusters/time.jade     |   2 +-
 .../configuration/clusters/transactions.jade    |   2 +-
 .../states/configuration/domains/general.jade   |   2 +-
 .../states/configuration/domains/query.jade     |  16 +-
 .../states/configuration/domains/store.jade     |  12 +-
 .../modules/states/configuration/igfs/dual.jade |   2 +-
 .../states/configuration/igfs/fragmentizer.jade |   2 +-
 .../states/configuration/igfs/general.jade      |   2 +-
 .../modules/states/configuration/igfs/ipc.jade  |   2 +-
 .../modules/states/configuration/igfs/misc.jade |   6 +-
 .../states/configuration/igfs/secondary.jade    |   2 +-
 .../configuration/summary/summary.controller.js |  12 +-
 .../frontend/controllers/clusters-controller.js |  10 +
 .../frontend/controllers/domains-controller.js  |   8 +-
 .../frontend/gulpfile.babel.js/tasks/jade.js    |   4 +-
 .../frontend/gulpfile.babel.js/tasks/test.js    |  92 --------
 .../gulpfile.babel.js/webpack/common.js         |   5 +-
 .../webpack/environments/development.js         |   5 +
 modules/web-console/frontend/package.json       |   2 +-
 .../stylesheets/_font-awesome-custom.scss       |  18 ++
 .../frontend/public/stylesheets/style.scss      |  13 +-
 .../frontend/views/configuration/caches.jade    |  22 +-
 .../frontend/views/configuration/clusters.jade  |  50 ++--
 .../views/configuration/domains-import.jade     |   4 +-
 .../frontend/views/configuration/domains.jade   |   8 +-
 .../frontend/views/configuration/igfs.jade      |  14 +-
 .../frontend/views/configuration/summary.jade   |   2 +-
 .../frontend/views/includes/header.jade         |  15 +-
 .../frontend/views/settings/profile.jade        |   2 +-
 modules/web-console/frontend/views/sql/sql.jade |   2 +-
 .../views/templates/agent-download.jade         |   2 +-
 .../frontend/views/templates/batch-confirm.jade |   2 +-
 .../frontend/views/templates/clone.jade         |   2 +-
 .../frontend/views/templates/confirm.jade       |   2 +-
 modules/web-console/web-agent/README.txt        |   4 +-
 .../web-agent/bin/ignite-web-agent.bat          |   7 +-
 .../web-agent/bin/ignite-web-agent.sh           |   6 +-
 .../console/agent/AgentConfiguration.java       |   5 +-
 .../ignite/console/agent/AgentLauncher.java     |   6 +-
 .../ignite/console/demo/AgentClusterDemo.java   |   1 +
 143 files changed, 682 insertions(+), 564 deletions(-)
----------------------------------------------------------------------



[09/52] ignite git commit: Merge branch 'ignite-1.7.4' into ignite-1.8

Posted by vo...@apache.org.
Merge branch 'ignite-1.7.4' into ignite-1.8


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/4ba13869
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4ba13869
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4ba13869

Branch: refs/heads/master
Commit: 4ba1386987cdd7ee26c14a6d599d6c6a970ca08a
Parents: 86d143b b038730
Author: devozerov <vo...@gridgain.com>
Authored: Wed Nov 23 17:56:50 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Nov 23 17:56:50 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcRequestHandler.java     | 32 ++++++++++++++------
 .../src/impl/binary/binary_reader_impl.cpp      |  3 +-
 modules/platforms/cpp/odbc/src/connection.cpp   | 21 +++++++------
 3 files changed, 34 insertions(+), 22 deletions(-)
----------------------------------------------------------------------



[07/52] ignite git commit: IGNITE-2294: Implemented DML.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
index e15e770..b80f573 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
@@ -23,12 +23,13 @@ import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.Map;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryArrayIdentityResolver;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryIdentityResolver;
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl;
-import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
@@ -50,7 +51,7 @@ public abstract class BinaryObjectExImpl implements BinaryObjectEx {
     /**
      * @return {@code True} if object is array based.
      */
-    protected abstract boolean hasArray();
+    public abstract boolean hasArray();
 
     /**
      * @return Object array if object is array based, otherwise {@code null}.
@@ -81,22 +82,29 @@ public abstract class BinaryObjectExImpl implements BinaryObjectEx {
      *
      * @return Field value.
      */
-    @Nullable protected abstract int dataStartOffset();
+    public abstract int dataStartOffset();
 
     /**
      * Get offset of the footer begin.
      *
      * @return Field value.
      */
-    @Nullable protected abstract int footerStartOffset();
+    public abstract int footerStartOffset();
 
     /**
      * Get field by offset.
      *
-     * @param order Field order.
+     * @param order Field offset.
      * @return Field value.
      */
-    @Nullable protected abstract <F> F fieldByOrder(int order);
+    @Nullable public abstract <F> F fieldByOrder(int order);
+
+    /**
+     * Create field comparer.
+     *
+     * @return Comparer.
+     */
+    public abstract BinarySerializedFieldComparator createFieldComparator();
 
     /**
      * @param ctx Reader context.
@@ -106,18 +114,30 @@ public abstract class BinaryObjectExImpl implements BinaryObjectEx {
     @Nullable protected abstract <F> F field(BinaryReaderHandles ctx, String fieldName);
 
     /**
+     * @return {@code True} if object has schema.
+     */
+    public abstract boolean hasSchema();
+
+    /**
      * Get schema ID.
      *
      * @return Schema ID.
      */
-    protected abstract int schemaId();
+    public abstract int schemaId();
 
     /**
      * Create schema for object.
      *
      * @return Schema.
      */
-    protected abstract BinarySchema createSchema();
+    public abstract BinarySchema createSchema();
+
+    /**
+     * Get binary context.
+     *
+     * @return Binary context.
+     */
+    public abstract BinaryContext context();
 
     /** {@inheritDoc} */
     @Override public BinaryObjectBuilder toBuilder() throws BinaryObjectException {
@@ -134,58 +154,15 @@ public abstract class BinaryObjectExImpl implements BinaryObjectEx {
         if (other == this)
             return true;
 
-        if (other == null)
-            return false;
-
-        if (!(other instanceof BinaryObjectExImpl))
-            return false;
-
-        BinaryObjectExImpl other0 = (BinaryObjectExImpl)other;
-
-        if (typeId() != other0.typeId())
-            return false;
-
-        int start = dataStartOffset();
-        int end = footerStartOffset();
-
-        int otherStart = other0.dataStartOffset();
-        int otherEnd = other0.footerStartOffset();
-
-        int len = end - start;
-
-        if (len != otherEnd - otherStart)
+        if (!(other instanceof BinaryObject))
             return false;
 
-        if (hasArray()) {
-            byte[] arr = array();
+        BinaryIdentityResolver identity = context().identity(typeId());
 
-            if (other0.hasArray()) {
-                byte[] otherArr = other0.array();
+        if (identity == null)
+            identity = BinaryArrayIdentityResolver.instance();
 
-                for (int i = start, j = otherStart; i < end; i++, j++) {
-                    if (arr[i] != otherArr[j])
-                        return false;
-                }
-
-                return true;
-            }
-            else {
-                assert other0.offheapAddress() > 0;
-
-                return GridUnsafeMemory.compare(other0.offheapAddress() + otherStart, arr, start, len);
-            }
-        }
-        else {
-            assert offheapAddress() > 0;
-
-            if (other0.hasArray())
-                return GridUnsafeMemory.compare(offheapAddress() + start, other0.array(), otherStart, len);
-            else {
-                assert other0.offheapAddress() > 0;
-
-                return GridUnsafeMemory.compare(offheapAddress() + start, other0.offheapAddress() + otherStart, len);
-            }
-        }
+        return identity.equals(this, (BinaryObject)other);
     }
 
     /** {@inheritDoc} */
@@ -250,6 +227,7 @@ public abstract class BinaryObjectExImpl implements BinaryObjectEx {
      * @param ctx Reader context.
      * @param handles Handles for already traversed objects.
      */
+    @SuppressWarnings("unchecked")
     private void appendValue(Object val, SB buf, BinaryReaderHandles ctx,
         IdentityHashMap<BinaryObject, Integer> handles) {
         if (val instanceof byte[])

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
index 54a7c08..360c71a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
@@ -211,10 +211,8 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
         this.detachAllowed = detachAllowed;
     }
 
-    /**
-     * @return Context.
-     */
-    public BinaryContext context() {
+    /** {@inheritDoc} */
+    @Override public BinaryContext context() {
         return ctx;
     }
 
@@ -241,7 +239,7 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean hasArray() {
+    @Override public boolean hasArray() {
         return true;
     }
 
@@ -296,20 +294,34 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override protected int dataStartOffset() {
+    @Override public BinarySerializedFieldComparator createFieldComparator() {
+        int schemaOff = BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.SCHEMA_OR_RAW_OFF_POS);
+
+        short flags = BinaryPrimitives.readShort(arr, start + GridBinaryMarshaller.FLAGS_POS);
+
+        int fieldIdLen = BinaryUtils.isCompactFooter(flags) ? 0 : BinaryUtils.FIELD_ID_LEN;
+        int fieldOffLen = BinaryUtils.fieldOffsetLength(flags);
+
+        int orderBase = start + schemaOff + fieldIdLen;
+        int orderMultiplier = fieldIdLen + fieldOffLen;
+
+        return new BinarySerializedFieldComparator(this, arr, 0L, start, orderBase, orderMultiplier, fieldOffLen);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int dataStartOffset() {
         int typeId = BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.TYPE_ID_POS);
 
         if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) {
             int len = BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.DFLT_HDR_LEN + 1);
 
             return start + GridBinaryMarshaller.DFLT_HDR_LEN + len + 5;
-        }
-        else
+        } else
             return start + GridBinaryMarshaller.DFLT_HDR_LEN;
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override protected int footerStartOffset() {
+    @Override public int footerStartOffset() {
         short flags = BinaryPrimitives.readShort(arr, start + GridBinaryMarshaller.FLAGS_POS);
 
         if (!BinaryUtils.hasSchema(flags))
@@ -318,9 +330,12 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
         return start + BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.SCHEMA_OR_RAW_OFF_POS);
     }
 
-   /** {@inheritDoc} */
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Nullable @Override protected <F> F fieldByOrder(int order) {
+    @Nullable @Override public <F> F fieldByOrder(int order) {
+        if (order == BinarySchema.ORDER_NOT_FOUND)
+            return null;
+
         Object val;
 
         // Calculate field position.
@@ -490,12 +505,19 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     }
 
     /** {@inheritDoc} */
-    @Override protected int schemaId() {
+    @Override public boolean hasSchema() {
+        short flags = BinaryPrimitives.readShort(arr, start + GridBinaryMarshaller.FLAGS_POS);
+
+        return BinaryUtils.hasSchema(flags);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int schemaId() {
         return BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.SCHEMA_ID_POS);
     }
 
     /** {@inheritDoc} */
-    @Override protected BinarySchema createSchema() {
+    @Override public BinarySchema createSchema() {
         return reader(null, false).getOrCreateSchema();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
index 7550b19..354ac11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
@@ -96,7 +96,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
         if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) {
             int off = start + GridBinaryMarshaller.DFLT_HDR_LEN;
 
-            String clsName = BinaryUtils.doReadClassName(new BinaryOffheapInputStream(ptr + off, size));
+            String clsName = BinaryUtils.doReadClassName(new BinaryOffheapInputStream(off, size));
 
             typeId = ctx.typeId(clsName);
         }
@@ -115,16 +115,28 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
     }
 
     /** {@inheritDoc} */
-    @Override protected int schemaId() {
+    @Override public boolean hasSchema() {
+        short flags = BinaryPrimitives.readShort(ptr, start + GridBinaryMarshaller.FLAGS_POS);
+
+        return BinaryUtils.hasSchema(flags);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int schemaId() {
         return BinaryPrimitives.readInt(ptr, start + GridBinaryMarshaller.SCHEMA_ID_POS);
     }
 
     /** {@inheritDoc} */
-    @Override protected BinarySchema createSchema() {
+    @Override public BinarySchema createSchema() {
         return reader(null, false).getOrCreateSchema();
     }
 
     /** {@inheritDoc} */
+    @Override public BinaryContext context() {
+        return ctx;
+    }
+
+    /** {@inheritDoc} */
     @Override public int start() {
         return start;
     }
@@ -140,7 +152,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean hasArray() {
+    @Override public boolean hasArray() {
         return false;
     }
 
@@ -174,7 +186,22 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override protected int dataStartOffset() {
+    @Override public BinarySerializedFieldComparator createFieldComparator() {
+        int schemaOff = BinaryPrimitives.readInt(ptr, start + GridBinaryMarshaller.SCHEMA_OR_RAW_OFF_POS);
+
+        short flags = BinaryPrimitives.readShort(ptr, start + GridBinaryMarshaller.FLAGS_POS);
+
+        int fieldIdLen = BinaryUtils.isCompactFooter(flags) ? 0 : BinaryUtils.FIELD_ID_LEN;
+        int fieldOffLen = BinaryUtils.fieldOffsetLength(flags);
+
+        int orderBase = start + schemaOff + fieldIdLen;
+        int orderMultiplier = fieldIdLen + fieldOffLen;
+
+        return new BinarySerializedFieldComparator(this, null, ptr, start, orderBase, orderMultiplier, fieldOffLen);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int dataStartOffset() {
         int typeId = BinaryPrimitives.readInt(ptr, start + GridBinaryMarshaller.TYPE_ID_POS);
 
         if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) {
@@ -186,7 +213,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override protected int footerStartOffset() {
+    @Override public int footerStartOffset() {
         short flags = BinaryPrimitives.readShort(ptr, start + GridBinaryMarshaller.FLAGS_POS);
 
         if (!BinaryUtils.hasSchema(flags))
@@ -197,7 +224,10 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Nullable @Override protected <F> F fieldByOrder(int order) {
+    @Nullable @Override public <F> F fieldByOrder(int order) {
+        if (order == BinarySchema.ORDER_NOT_FOUND)
+            return null;
+
         Object val;
 
         // Calculate field position.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPrimitives.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPrimitives.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPrimitives.java
index 8b82fad..86f5040 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPrimitives.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPrimitives.java
@@ -120,6 +120,18 @@ public abstract class BinaryPrimitives {
     }
 
     /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @param val Value.
+     */
+    public static void writeShort(long ptr, int off, short val) {
+        if (BIG_ENDIAN)
+            GridUnsafe.putShortLE(ptr + off, val);
+        else
+            GridUnsafe.putShort(ptr + off, val);
+    }
+
+    /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.
@@ -228,6 +240,18 @@ public abstract class BinaryPrimitives {
     }
 
     /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @param val Value.
+     */
+    public static void writeInt(long ptr, int off, int val) {
+        if (BIG_ENDIAN)
+            GridUnsafe.putIntLE(ptr + off, val);
+        else
+            GridUnsafe.putInt(ptr + off, val);
+    }
+
+    /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySerializedFieldComparator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySerializedFieldComparator.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySerializedFieldComparator.java
new file mode 100644
index 0000000..130bb0c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySerializedFieldComparator.java
@@ -0,0 +1,343 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Arrays;
+
+/**
+ * Compares fiels in serialized form when possible.
+ */
+public class BinarySerializedFieldComparator {
+    /** Position: not found. */
+    private static final int POS_NOT_FOUND = -1;
+
+    /** Original object. */
+    private final BinaryObjectExImpl obj;
+
+    /** Pointer to data (onheap). */
+    private final byte[] arr;
+
+    /** Pointer to data (offheap). */
+    private final long ptr;
+
+    /** Object start offset. */
+    private final int startOff;
+
+    /** Order base. */
+    private final int orderBase;
+
+    /** Order multiplier. */
+    private final int orderMultiplier;
+
+    /** Field offset length. */
+    private final int fieldOffLen;
+
+    /** Current field order. */
+    private int curFieldOrder;
+
+    /** Current field offset. */
+    private int curFieldPos;
+
+    /**
+     * Constructor.
+     *
+     * @param obj Original object.
+     * @param arr Array.
+     * @param ptr Pointer.
+     * @param startOff Start offset.
+     * @param orderBase Order base.
+     * @param orderMultiplier Order multiplier.
+     * @param fieldOffLen Field offset length.
+     */
+    public BinarySerializedFieldComparator(BinaryObjectExImpl obj, byte[] arr, long ptr, int startOff, int orderBase,
+        int orderMultiplier, int fieldOffLen) {
+        assert arr != null && ptr == 0L || arr == null && ptr != 0L;
+
+        this.obj = obj;
+        this.arr = arr;
+        this.ptr = ptr;
+        this.startOff = startOff;
+        this.orderBase = orderBase;
+        this.orderMultiplier = orderMultiplier;
+        this.fieldOffLen = fieldOffLen;
+    }
+
+    /**
+     * Locate the field.
+     *
+     * @param order Field order.
+     */
+    public void findField(int order) {
+        curFieldOrder = order;
+
+        if (order == BinarySchema.ORDER_NOT_FOUND)
+            curFieldPos = POS_NOT_FOUND;
+        else {
+            int pos = orderBase + order * orderMultiplier;
+
+            if (fieldOffLen == BinaryUtils.OFFSET_1) {
+                byte val = offheap() ? BinaryPrimitives.readByte(ptr, pos) : BinaryPrimitives.readByte(arr, pos);
+
+                curFieldPos = startOff + ((int)val & 0xFF);
+            }
+            else if (fieldOffLen == BinaryUtils.OFFSET_2) {
+                short val = offheap() ? BinaryPrimitives.readShort(ptr, pos) : BinaryPrimitives.readShort(arr, pos);
+
+                curFieldPos = startOff + ((int)val & 0xFFFF);
+            }
+            else {
+                int val = offheap() ? BinaryPrimitives.readInt(ptr, pos) : BinaryPrimitives.readInt(arr, pos);
+
+                curFieldPos = startOff + val;
+            }
+        }
+    }
+
+    /**
+     * Get field type.
+     *
+     * @return Field type.
+     */
+    private byte fieldType() {
+        if (curFieldPos == POS_NOT_FOUND)
+            return GridBinaryMarshaller.NULL;
+        else
+            return offheap() ?
+                BinaryPrimitives.readByte(ptr, curFieldPos) : BinaryPrimitives.readByte(arr, curFieldPos);
+    }
+
+    /**
+     * @return Whether this is offheap object.
+     */
+    private boolean offheap() {
+        return ptr != 0L;
+    }
+
+    /**
+     * Get current field.
+     *
+     * @return Current field.
+     */
+    private Object currentField() {
+        return obj.fieldByOrder(curFieldOrder);
+    }
+
+    /**
+     * Read byte value.
+     *
+     * @param off Offset.
+     * @return Value.
+     */
+    private byte readByte(int off) {
+        if (offheap())
+            return BinaryPrimitives.readByte(ptr, curFieldPos + off);
+        else
+            return arr[curFieldPos + off];
+    }
+
+    /**
+     * Read short value.
+     *
+     * @param off Offset.
+     * @return Value.
+     */
+    private short readShort(int off) {
+        if (offheap())
+            return BinaryPrimitives.readShort(ptr, curFieldPos + off);
+        else
+            return BinaryPrimitives.readShort(arr, curFieldPos + off);
+    }
+
+    /**
+     * Read int value.
+     *
+     * @param off Offset.
+     * @return Value.
+     */
+    private int readInt(int off) {
+        if (offheap())
+            return BinaryPrimitives.readInt(ptr, curFieldPos + off);
+        else
+            return BinaryPrimitives.readInt(arr, curFieldPos + off);
+    }
+
+    /**
+     * Read long value.
+     *
+     * @param off Offset.
+     * @return Value.
+     */
+    private long readLong(int off) {
+        if (offheap())
+            return BinaryPrimitives.readLong(ptr, curFieldPos + off);
+        else
+            return BinaryPrimitives.readLong(arr, curFieldPos + off);
+    }
+
+    /**
+     * Compare fields.
+     *
+     * @param c1 First comparer.
+     * @param c2 Second comparer.
+     * @return {@code True} if both fields are equal.
+     */
+    public static boolean equals(BinarySerializedFieldComparator c1, BinarySerializedFieldComparator c2) {
+        // Compare field types.
+        byte typ = c1.fieldType();
+
+        if (typ != c2.fieldType())
+            return false;
+
+        // Switch by type and compare.
+        switch (typ) {
+            case GridBinaryMarshaller.BYTE:
+            case GridBinaryMarshaller.BOOLEAN:
+                return c1.readByte(1) == c2.readByte(1);
+
+            case GridBinaryMarshaller.SHORT:
+            case GridBinaryMarshaller.CHAR:
+                return c1.readShort(1) == c2.readShort(1);
+
+            case GridBinaryMarshaller.INT:
+            case GridBinaryMarshaller.FLOAT:
+                return c1.readInt(1) == c2.readInt(1);
+
+            case GridBinaryMarshaller.LONG:
+            case GridBinaryMarshaller.DOUBLE:
+            case GridBinaryMarshaller.DATE:
+                return c1.readLong(1) == c2.readLong(1);
+
+            case GridBinaryMarshaller.TIMESTAMP:
+                return c1.readLong(1) == c2.readLong(1) && c1.readInt(1 + 8) == c2.readInt(1 + 8);
+
+            case GridBinaryMarshaller.UUID:
+                return c1.readLong(1) == c2.readLong(1) && c1.readLong(1 + 8) == c2.readLong(1 + 8);
+
+            case GridBinaryMarshaller.STRING:
+                return compareByteArrays(c1, c2, 1);
+
+            case GridBinaryMarshaller.DECIMAL:
+                return c1.readInt(1) == c2.readInt(1) && compareByteArrays(c1, c2, 5);
+
+            case GridBinaryMarshaller.NULL:
+                return true;
+
+            default:
+                Object val1 = c1.currentField();
+                Object val2 = c2.currentField();
+
+                return isArray(val1) ? compareArrays(val1, val2) : F.eq(val1, val2);
+        }
+    }
+
+    /**
+     * Compare arrays.
+     *
+     * @param val1 Value 1.
+     * @param val2 Value 2.
+     * @return Result.
+     */
+    private static boolean compareArrays(Object val1, Object val2) {
+        if (val1.getClass() == val2.getClass()) {
+            if (val1 instanceof byte[])
+                return Arrays.equals((byte[])val1, (byte[])val2);
+            else if (val1 instanceof boolean[])
+                return Arrays.equals((boolean[])val1, (boolean[])val2);
+            else if (val1 instanceof short[])
+                return Arrays.equals((short[])val1, (short[])val2);
+            else if (val1 instanceof char[])
+                return Arrays.equals((char[])val1, (char[])val2);
+            else if (val1 instanceof int[])
+                return Arrays.equals((int[])val1, (int[])val2);
+            else if (val1 instanceof long[])
+                return Arrays.equals((long[])val1, (long[])val2);
+            else if (val1 instanceof float[])
+                return Arrays.equals((float[])val1, (float[])val2);
+            else if (val1 instanceof double[])
+                return Arrays.equals((double[])val1, (double[])val2);
+            else
+                return Arrays.deepEquals((Object[])val1, (Object[])val2);
+        }
+
+        return false;
+    }
+
+    /**
+     * @param field Field.
+     * @return {@code True} if field is array.
+     */
+    private static boolean isArray(@Nullable Object field) {
+        return field != null && field.getClass().isArray();
+    }
+
+    /**
+     * Compare byte arrays.
+     *
+     * @param c1 Comparer 1.
+     * @param c2 Comparer 2.
+     * @param off Offset (where length is located).
+     * @return {@code True} if equal.
+     */
+    private static boolean compareByteArrays(BinarySerializedFieldComparator c1, BinarySerializedFieldComparator c2,
+                                             int off) {
+        int len = c1.readInt(off);
+
+        if (len != c2.readInt(off))
+            return false;
+        else {
+            off += 4;
+
+            if (c1.offheap()) {
+                if (c2.offheap())
+                    // Case 1: both offheap.
+                    return GridUnsafeMemory.compare(c1.curFieldPos + c1.ptr + off, c2.curFieldPos + c2.ptr + off, len);
+            }
+            else {
+                if (!c2.offheap()) {
+                    // Case 2: both onheap.
+                    for (int i = 0; i < len; i++) {
+                        if (c1.arr[c1.curFieldPos + off + i] != c2.arr[c2.curFieldPos + off + i])
+                            return false;
+                    }
+
+                    return true;
+                }
+                else {
+                    // Swap.
+                    BinarySerializedFieldComparator tmp = c1;
+                    c1 = c2;
+                    c2 = tmp;
+                }
+            }
+
+            // Case 3: offheap vs onheap.
+            assert c1.offheap() && !c2.offheap();
+
+            for (int i = 0; i < len; i++) {
+                if (BinaryPrimitives.readByte(c1.ptr, c1.curFieldPos + off + i) != c2.arr[c2.curFieldPos + off + i])
+                    return false;
+            }
+
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index b304082..cb6e641 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -312,7 +312,7 @@ public class BinaryUtils {
      * @param flag Flag.
      * @return {@code True} if flag is set in flags.
      */
-    static boolean isFlagSet(short flags, short flag) {
+    public static boolean isFlagSet(short flags, short flag) {
         return (flags & flag) == flag;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
index 1de0a65..adaacdd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
@@ -29,6 +29,7 @@ import java.util.Date;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryIdentityResolver;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.binary.BinaryWriter;
@@ -326,6 +327,48 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     }
 
     /**
+     * Perform post-write hash code update if necessary.
+     *
+     * @param clsName Class name. Always null if class is registered.
+     */
+    public void postWriteHashCode(@Nullable String clsName) {
+        int typeId = clsName == null ? this.typeId : ctx.typeId(clsName);
+
+        BinaryIdentityResolver identity = ctx.identity(typeId);
+
+        if (identity != null) {
+            if (out.hasArray()) {
+                // Heap.
+                byte[] data = out.array();
+
+                BinaryObjectImpl obj = new BinaryObjectImpl(ctx, data, start);
+
+                short flags = BinaryPrimitives.readShort(data, start + GridBinaryMarshaller.FLAGS_POS);
+
+                BinaryPrimitives.writeShort(data, start + GridBinaryMarshaller.FLAGS_POS,
+                    (short) (flags & ~BinaryUtils.FLAG_EMPTY_HASH_CODE));
+
+                BinaryPrimitives.writeInt(data, start + GridBinaryMarshaller.HASH_CODE_POS, identity.hashCode(obj));
+            }
+            else {
+                // Offheap.
+                long ptr = out.rawOffheapPointer();
+
+                assert ptr != 0;
+
+                BinaryObjectOffheapImpl obj = new BinaryObjectOffheapImpl(ctx, ptr, start, out.capacity());
+
+                short flags = BinaryPrimitives.readShort(ptr, start + GridBinaryMarshaller.FLAGS_POS);
+
+                BinaryPrimitives.writeShort(ptr, start + GridBinaryMarshaller.FLAGS_POS,
+                    (short) (flags & ~BinaryUtils.FLAG_EMPTY_HASH_CODE));
+
+                BinaryPrimitives.writeInt(ptr, start + GridBinaryMarshaller.HASH_CODE_POS, identity.hashCode(obj));
+            }
+        }
+    }
+
+    /**
      * Pop schema.
      */
     public void popSchema() {
@@ -337,8 +380,6 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      * @param val Byte array.
      */
     public void write(byte[] val) {
-        assert val != null;
-
         out.writeByteArray(val);
     }
 
@@ -348,8 +389,6 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      * @param len Length.
      */
     public void write(byte[] val, int off, int len) {
-        assert val != null;
-
         out.write(val, off, len);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
index f0bc874..ddd2423 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
@@ -130,6 +130,8 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * @param start Start.
      */
     BinaryObjectBuilderImpl(BinaryBuilderReader reader, int start) {
+        assert reader != null;
+
         this.reader = reader;
         this.start = start;
         this.flags = reader.readShortPositioned(start + GridBinaryMarshaller.FLAGS_POS);
@@ -193,6 +195,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * @param writer Writer.
      * @param serializer Serializer.
      */
+    @SuppressWarnings("ResultOfMethodCallIgnored")
     void serializeTo(BinaryWriterExImpl writer, BinaryBuilderSerializer serializer) {
         try {
             writer.preWrite(registeredType ? null : clsNameToWrite);
@@ -357,6 +360,9 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
 
                 schemaReg.addSchema(curSchema.schemaId(), curSchema);
             }
+
+            // Update hash code after schema is written.
+            writer.postWriteHashCode(registeredType ? null : clsNameToWrite);
         }
         finally {
             writer.popSchema();

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryAbstractInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryAbstractInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryAbstractInputStream.java
index b6c30bb..46aa03d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryAbstractInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryAbstractInputStream.java
@@ -289,6 +289,11 @@ public abstract class BinaryAbstractInputStream extends BinaryAbstractStream
         return 0;
     }
 
+    /** {@inheritDoc} */
+    @Override public long rawOffheapPointer() {
+        return 0;
+    }
+
     /**
      * Ensure that there is enough data.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryAbstractOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryAbstractOutputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryAbstractOutputStream.java
index b9df68e..769031f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryAbstractOutputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryAbstractOutputStream.java
@@ -244,6 +244,11 @@ public abstract class BinaryAbstractOutputStream extends BinaryAbstractStream
     }
 
     /** {@inheritDoc} */
+    @Override public long rawOffheapPointer() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
     @Override public void unsafeEnsure(int cap) {
         ensureCapacity(pos + cap);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryHeapInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryHeapInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryHeapInputStream.java
index b584373..b5edc02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryHeapInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryHeapInputStream.java
@@ -92,6 +92,11 @@ public final class BinaryHeapInputStream extends BinaryAbstractInputStream {
     }
 
     /** {@inheritDoc} */
+    @Override public int capacity() {
+        return data.length;
+    }
+
+    /** {@inheritDoc} */
     @Override public byte[] array() {
         return data;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryHeapOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryHeapOutputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryHeapOutputStream.java
index 2c31641..f06c980 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryHeapOutputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryHeapOutputStream.java
@@ -209,4 +209,9 @@ public final class BinaryHeapOutputStream extends BinaryAbstractOutputStream {
 
         shift(8);
     }
+
+    /** {@inheritDoc} */
+    @Override public int capacity() {
+        return data.length;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryOffheapInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryOffheapInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryOffheapInputStream.java
index 9230846..9dc92c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryOffheapInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryOffheapInputStream.java
@@ -66,6 +66,11 @@ public class BinaryOffheapInputStream extends BinaryAbstractInputStream {
     }
 
     /** {@inheritDoc} */
+    @Override public int capacity() {
+        return cap;
+    }
+
+    /** {@inheritDoc} */
     @Override public byte[] array() {
         return arrayCopy();
     }
@@ -147,4 +152,9 @@ public class BinaryOffheapInputStream extends BinaryAbstractInputStream {
     @Override public long offheapPointer() {
         return forceHeap ? 0 : ptr;
     }
+
+    /** {@inheritDoc} */
+    @Override public long rawOffheapPointer() {
+        return ptr;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryOffheapOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryOffheapOutputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryOffheapOutputStream.java
index 1cb9f4f..be9f7d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryOffheapOutputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryOffheapOutputStream.java
@@ -89,9 +89,7 @@ public class BinaryOffheapOutputStream extends BinaryAbstractOutputStream {
         return ptr;
     }
 
-    /**
-     * @return Capacity.
-     */
+    /** {@inheritDoc} */
     public int capacity() {
         return cap;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryStream.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryStream.java
index b868199..5bdd644 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryStream.java
@@ -42,12 +42,22 @@ public interface BinaryStream {
     public byte[] arrayCopy();
 
     /**
-     * @return Offheap pointer if stream is offheap based, otherwise {@code 0}.
+     * @return Offheap pointer if stream is offheap based and "forceHeap" flag is not set; otherwise {@code 0}.
      */
     public long offheapPointer();
 
     /**
+     * @return Offheap pointer if stream is offheap based; otherwise {@code 0}.
+     */
+    public long rawOffheapPointer();
+
+    /**
      * @return {@code True} is stream is array based.
      */
     public boolean hasArray();
+
+    /**
+     * @return Total capacity.
+     */
+    public int capacity();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
index 6f0d9c4..5c4a147 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
@@ -55,6 +55,7 @@ import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.compute.ComputeTaskTimeoutException;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.F;
@@ -697,6 +698,13 @@ public class JdbcConnection implements Connection {
     }
 
     /**
+     * @return {@code true} if target node has DML support, {@code false} otherwise.
+     */
+    boolean isDmlSupported() {
+        return ignite.version().greaterThanEqual(1, 8, 0);
+    }
+
+    /**
      * @return Local query flag.
      */
     boolean isLocalQuery() {
@@ -736,6 +744,15 @@ public class JdbcConnection implements Connection {
     }
 
     /**
+     * @param sql Query.
+     * @return {@link PreparedStatement} from underlying engine to supply metadata to Prepared - most likely H2.
+     */
+    PreparedStatement prepareNativeStatement(String sql) throws SQLException {
+        return ((IgniteCacheProxy) ignite().cache(cacheName())).context()
+            .kernalContext().query().prepareNativeStatement(cacheName(), sql);
+    }
+
+    /**
      * JDBC connection validation task.
      */
     private static class JdbcConnectionValidationTask implements IgniteCallable<Boolean> {

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
index a99f24c..57badd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
@@ -31,8 +31,10 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     /** SQL query. */
     private final String sql;
 
-    /** Arguments count. */
-    private final int argsCnt;
+    /**
+     * H2's parsed statement to retrieve metadata from.
+     */
+    private PreparedStatement nativeStatement;
 
     /**
      * Creates new prepared statement.
@@ -44,12 +46,21 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
         super(conn);
 
         this.sql = sql;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addBatch(String sql) throws SQLException {
+        ensureNotClosed();
 
-        argsCnt = sql.replaceAll("[^?]", "").length();
+        throw new SQLFeatureNotSupportedException("Adding new SQL command to batch not supported for prepared statement.");
     }
 
+
+
     /** {@inheritDoc} */
     @Override public ResultSet executeQuery() throws SQLException {
+        ensureNotClosed();
+
         ResultSet rs = executeQuery(sql);
 
         args = null;
@@ -61,7 +72,11 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     @Override public int executeUpdate() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        int res = executeUpdate(sql);
+
+        args = null;
+
+        return res;
     }
 
     /** {@inheritDoc} */
@@ -163,6 +178,13 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     }
 
     /** {@inheritDoc} */
+    @Override public void clearBatch() throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Batch statements are not supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public void setObject(int paramIdx, Object x, int targetSqlType) throws SQLException {
         setArgument(paramIdx, x);
     }
@@ -181,7 +203,12 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     @Override public void addBatch() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        throw new SQLFeatureNotSupportedException("Batch statements are not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] executeBatch() throws SQLException {
+        throw new SQLFeatureNotSupportedException("Batch statements are not supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -223,7 +250,7 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     @Override public ResultSetMetaData getMetaData() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Meta data for prepared statement is not supported.");
+        return getNativeStatement().getMetaData();
     }
 
     /** {@inheritDoc} */
@@ -255,7 +282,7 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     @Override public ParameterMetaData getParameterMetaData() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Meta data for prepared statement is not supported.");
+        return getNativeStatement().getParameterMetaData();
     }
 
     /** {@inheritDoc} */
@@ -400,12 +427,36 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     private void setArgument(int paramIdx, Object val) throws SQLException {
         ensureNotClosed();
 
-        if (paramIdx < 1 || paramIdx > argsCnt)
+        if (paramIdx < 1)
             throw new SQLException("Parameter index is invalid: " + paramIdx);
 
+        ensureArgsSize(paramIdx);
+
+        args.set(paramIdx - 1, val);
+    }
+
+    /**
+     * Initialize {@link #args} and increase its capacity and size up to given argument if needed.
+     * @param size new expected size.
+     */
+    private void ensureArgsSize(int size) {
         if (args == null)
-            args = new Object[argsCnt];
+            args = new ArrayList<>(size);
+
+        args.ensureCapacity(size);
+
+        while (args.size() < size)
+            args.add(null);
+    }
+
+    /**
+     * @return H2's prepared statement to get metadata from.
+     * @throws SQLException if failed.
+     */
+    private PreparedStatement getNativeStatement() throws SQLException {
+        if (nativeStatement != null)
+            return nativeStatement;
 
-        args[paramIdx - 1] = val;
+        return (nativeStatement = conn.prepareNativeStatement(sql));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
index c4911cb..0b23f9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
@@ -49,7 +49,10 @@ import org.apache.ignite.resources.IgniteInstanceResource;
  * Not closed cursors will be removed after {@link #RMV_DELAY} milliseconds.
  * This parameter can be configured via {@link IgniteSystemProperties#IGNITE_JDBC_DRIVER_CURSOR_REMOVE_DELAY}
  * system property.
+ *
+ * Deprecated due to introduction of DML features - see {@link JdbcQueryTaskV2}.
  */
+@Deprecated
 class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
     /** Serial version uid. */
     private static final long serialVersionUID = 0L;
@@ -178,7 +181,7 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
             List<Object> row0 = new ArrayList<>(row.size());
 
             for (Object val : row)
-                row0.add(JdbcUtils.sqlType(val) ? val : val.toString());
+                row0.add(val == null || JdbcUtils.isSqlType(val.getClass()) ? val : val.toString());
 
             rows.add(row0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java
new file mode 100644
index 0000000..9093d15
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java
@@ -0,0 +1,406 @@
+/*
+ * 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.jdbc2;
+
+import java.io.Serializable;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteJdbcDriver;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
+import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
+import org.apache.ignite.internal.util.typedef.CAX;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.resources.IgniteInstanceResource;
+
+/**
+ * Task for SQL queries execution through {@link IgniteJdbcDriver}.
+ * <p>
+ * Not closed cursors will be removed after {@link #RMV_DELAY} milliseconds.
+ * This parameter can be configured via {@link IgniteSystemProperties#IGNITE_JDBC_DRIVER_CURSOR_REMOVE_DELAY}
+ * system property.
+ */
+class JdbcQueryTaskV2 implements IgniteCallable<JdbcQueryTaskV2.QueryResult> {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** How long to store open cursor. */
+    private static final long RMV_DELAY = IgniteSystemProperties.getLong(
+        IgniteSystemProperties.IGNITE_JDBC_DRIVER_CURSOR_REMOVE_DELAY, 600000);
+
+    /** Scheduler. */
+    private static final ScheduledExecutorService SCHEDULER = Executors.newScheduledThreadPool(1);
+
+    /** Open cursors. */
+    private static final ConcurrentMap<UUID, Cursor> CURSORS = new ConcurrentHashMap<>();
+
+    /** Ignite. */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** Uuid. */
+    private final UUID uuid;
+
+    /** Cache name. */
+    private final String cacheName;
+
+    /** Sql. */
+    private final String sql;
+
+    /** Operation type flag - query or not. */
+    private Boolean isQry;
+
+    /** Args. */
+    private final Object[] args;
+
+    /** Fetch size. */
+    private final int fetchSize;
+
+    /** Local execution flag. */
+    private final boolean loc;
+
+    /** Local query flag. */
+    private final boolean locQry;
+
+    /** Collocated query flag. */
+    private final boolean collocatedQry;
+
+    /** Distributed joins flag. */
+    private final boolean distributedJoins;
+
+    /**
+     * @param ignite Ignite.
+     * @param cacheName Cache name.
+     * @param sql Sql query.
+     * @param isQry Operation type flag - query or not - to enforce query type check.
+     * @param loc Local execution flag.
+     * @param args Args.
+     * @param fetchSize Fetch size.
+     * @param uuid UUID.
+     * @param locQry Local query flag.
+     * @param collocatedQry Collocated query flag.
+     * @param distributedJoins Distributed joins flag.
+     */
+    public JdbcQueryTaskV2(Ignite ignite, String cacheName, String sql,
+                           Boolean isQry, boolean loc, Object[] args, int fetchSize, UUID uuid,
+                           boolean locQry, boolean collocatedQry, boolean distributedJoins) {
+        this.ignite = ignite;
+        this.args = args;
+        this.uuid = uuid;
+        this.cacheName = cacheName;
+        this.sql = sql;
+        this.isQry = isQry;
+        this.fetchSize = fetchSize;
+        this.loc = loc;
+        this.locQry = locQry;
+        this.collocatedQry = collocatedQry;
+        this.distributedJoins = distributedJoins;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JdbcQueryTaskV2.QueryResult call() throws Exception {
+        Cursor cursor = CURSORS.get(uuid);
+
+        List<String> tbls = null;
+        List<String> cols = null;
+        List<String> types = null;
+
+        boolean first;
+
+        if (first = (cursor == null)) {
+            IgniteCache<?, ?> cache = ignite.cache(cacheName);
+
+            // Don't create caches on server nodes in order to avoid of data rebalancing.
+            boolean start = ignite.configuration().isClientMode();
+
+            if (cache == null && cacheName == null)
+                cache = ((IgniteKernal)ignite).context().cache().getOrStartPublicCache(start, !loc && locQry);
+
+            if (cache == null) {
+                if (cacheName == null)
+                    throw new SQLException("Failed to execute query. No suitable caches found.");
+                else
+                    throw new SQLException("Cache not found [cacheName=" + cacheName + ']');
+            }
+
+            SqlFieldsQuery qry = (isQry != null ? new JdbcSqlFieldsQuery(sql, isQry) : new SqlFieldsQuery(sql))
+                .setArgs(args);
+
+            qry.setPageSize(fetchSize);
+            qry.setLocal(locQry);
+            qry.setCollocated(collocatedQry);
+            qry.setDistributedJoins(distributedJoins);
+
+            QueryCursorImpl<List<?>> qryCursor = (QueryCursorImpl<List<?>>)cache.query(qry);
+
+            if (isQry == null)
+                isQry = qryCursor.isQuery();
+
+            Collection<GridQueryFieldMetadata> meta = qryCursor.fieldsMeta();
+
+            tbls = new ArrayList<>(meta.size());
+            cols = new ArrayList<>(meta.size());
+            types = new ArrayList<>(meta.size());
+
+            for (GridQueryFieldMetadata desc : meta) {
+                tbls.add(desc.typeName());
+                cols.add(desc.fieldName().toUpperCase());
+                types.add(desc.fieldTypeName());
+            }
+
+            CURSORS.put(uuid, cursor = new Cursor(qryCursor, qryCursor.iterator()));
+        }
+
+        List<List<?>> rows = new ArrayList<>();
+
+        for (List<?> row : cursor) {
+            List<Object> row0 = new ArrayList<>(row.size());
+
+            for (Object val : row)
+                row0.add(val == null || JdbcUtils.isSqlType(val.getClass()) ? val : val.toString());
+
+            rows.add(row0);
+
+            if (rows.size() == fetchSize) // If fetchSize is 0 then unlimited
+                break;
+        }
+
+        boolean finished = !cursor.hasNext();
+
+        if (finished)
+            remove(uuid, cursor);
+        else if (first) {
+            if (!loc)
+                scheduleRemoval(uuid, RMV_DELAY);
+        }
+        else if (!loc && !CURSORS.replace(uuid, cursor, new Cursor(cursor.cursor, cursor.iter)))
+            assert !CURSORS.containsKey(uuid) : "Concurrent cursor modification.";
+
+        assert isQry != null : "Query flag must be set prior to returning result";
+
+        return new QueryResult(uuid, finished, isQry, rows, cols, tbls, types);
+    }
+
+    /**
+     * Schedules removal of stored cursor in case of remote query execution.
+     *
+     * @param uuid Cursor UUID.
+     * @param delay Delay in milliseconds.
+     */
+    private void scheduleRemoval(final UUID uuid, long delay) {
+        assert !loc;
+
+        SCHEDULER.schedule(new CAX() {
+            @Override public void applyx() {
+                while (true) {
+                    Cursor c = CURSORS.get(uuid);
+
+                    if (c == null)
+                        break;
+
+                    // If the cursor was accessed since last scheduling then reschedule.
+                    long untouchedTime = U.currentTimeMillis() - c.lastAccessTime;
+
+                    if (untouchedTime < RMV_DELAY) {
+                        scheduleRemoval(uuid, RMV_DELAY - untouchedTime);
+
+                        break;
+                    }
+                    else if (remove(uuid, c))
+                        break;
+                }
+            }
+        }, delay, TimeUnit.MILLISECONDS);
+    }
+
+    /**
+     * @param uuid Cursor UUID.
+     * @param c Cursor.
+     * @return {@code true} If succeeded.
+     */
+    private static boolean remove(UUID uuid, Cursor c) {
+        boolean rmv = CURSORS.remove(uuid, c);
+
+        if (rmv)
+            c.cursor.close();
+
+        return rmv;
+    }
+
+    /**
+     * Closes and removes cursor.
+     *
+     * @param uuid Cursor UUID.
+     */
+    static void remove(UUID uuid) {
+        Cursor c = CURSORS.remove(uuid);
+
+        if (c != null)
+            c.cursor.close();
+    }
+
+
+    /**
+     * Result of query execution.
+     */
+    static class QueryResult implements Serializable {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** Uuid. */
+        private final UUID uuid;
+
+        /** Finished. */
+        private final boolean finished;
+
+        /** Result type - query or update. */
+        private final boolean isQry;
+
+        /** Rows. */
+        private final List<List<?>> rows;
+
+        /** Tables. */
+        private final List<String> tbls;
+
+        /** Columns. */
+        private final List<String> cols;
+
+        /** Types. */
+        private final List<String> types;
+
+        /**
+         * @param uuid UUID..
+         * @param finished Finished.
+         * @param isQry
+         * @param rows Rows.
+         * @param cols Columns.
+         * @param tbls Tables.
+         * @param types Types.
+         */
+        public QueryResult(UUID uuid, boolean finished, boolean isQry, List<List<?>> rows, List<String> cols,
+            List<String> tbls, List<String> types) {
+            this.isQry = isQry;
+            this.cols = cols;
+            this.uuid = uuid;
+            this.finished = finished;
+            this.rows = rows;
+            this.tbls = tbls;
+            this.types = types;
+        }
+
+        /**
+         * @return Query result rows.
+         */
+        public List<List<?>> getRows() {
+            return rows;
+        }
+
+        /**
+         * @return Tables metadata.
+         */
+        public List<String> getTbls() {
+            return tbls;
+        }
+
+        /**
+         * @return Columns metadata.
+         */
+        public List<String> getCols() {
+            return cols;
+        }
+
+        /**
+         * @return Types metadata.
+         */
+        public List<String> getTypes() {
+            return types;
+        }
+
+        /**
+         * @return Query UUID.
+         */
+        public UUID getUuid() {
+            return uuid;
+        }
+
+        /**
+         * @return {@code True} if it is finished query.
+         */
+        public boolean isFinished() {
+            return finished;
+        }
+
+        /**
+         * @return {@code true} if it is result of a query operation, not update; {@code false} otherwise.
+         */
+        public boolean isQuery() {
+            return isQry;
+        }
+    }
+
+    /**
+     * Cursor.
+     */
+    private static final class Cursor implements Iterable<List<?>> {
+        /** Cursor. */
+        final QueryCursor<List<?>> cursor;
+
+        /** Iterator. */
+        final Iterator<List<?>> iter;
+
+        /** Last access time. */
+        final long lastAccessTime;
+
+        /**
+         * @param cursor Cursor.
+         * @param iter Iterator.
+         */
+        private Cursor(QueryCursor<List<?>> cursor, Iterator<List<?>> iter) {
+            this.cursor = cursor;
+            this.iter = iter;
+            this.lastAccessTime = U.currentTimeMillis();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Iterator<List<?>> iterator() {
+            return iter;
+        }
+
+        /**
+         * @return {@code True} if cursor has next element.
+         */
+        public boolean hasNext() {
+            return iter.hasNext();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
index 1bf5223..c1a5f4c 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
@@ -45,6 +45,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -146,6 +147,29 @@ public class JdbcResultSet implements ResultSet {
 
             boolean loc = nodeId == null;
 
+            if (conn.isDmlSupported()) {
+                // Connections from new clients send queries with new tasks, so we have to continue in the same manner
+                JdbcQueryTaskV2 qryTask = new JdbcQueryTaskV2(loc ? ignite : null, conn.cacheName(), null, true, loc, null,
+                    fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins());
+
+                try {
+                    JdbcQueryTaskV2.QueryResult res =
+                        loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask);
+
+                    finished = res.isFinished();
+
+                    it = res.getRows().iterator();
+
+                    return next();
+                }
+                catch (IgniteSQLException e) {
+                    throw e.toJdbcException();
+                }
+                catch (Exception e) {
+                    throw new SQLException("Failed to query Ignite.", e);
+                }
+            }
+
             JdbcQueryTask qryTask = new JdbcQueryTask(loc ? ignite : null, conn.cacheName(), null, loc, null,
                 fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins());
 
@@ -159,6 +183,9 @@ public class JdbcResultSet implements ResultSet {
 
                 return next();
             }
+            catch (IgniteSQLException e) {
+                throw e.toJdbcException();
+            }
             catch (Exception e) {
                 throw new SQLException("Failed to query Ignite.", e);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcSqlFieldsQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcSqlFieldsQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcSqlFieldsQuery.java
new file mode 100644
index 0000000..1b27296
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcSqlFieldsQuery.java
@@ -0,0 +1,49 @@
+/*
+ * 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.jdbc2;
+
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * {@link SqlFieldsQuery} with JDBC flavor - it has additional flag indicating whether JDBC driver expects
+ * this query to return a result set or an update counter. This class is not intended for use outside JDBC driver.
+ */
+public final class JdbcSqlFieldsQuery extends SqlFieldsQuery {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Flag set by JDBC driver to enforce checks for correct operation type. */
+    private final boolean isQry;
+
+    /**
+     * @param sql SQL query.
+     * @param isQry Flag indicating whether this object denotes a query or an update operation.
+     */
+    JdbcSqlFieldsQuery(String sql, boolean isQry) {
+        super(sql);
+        this.isQry = isQry;
+    }
+
+    /**
+     * @return Flag indicating whether this object denotes a query or an update operation..
+     */
+    public boolean isQuery() {
+        return isQry;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
index e187dc0..dbb2390 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
@@ -17,19 +17,25 @@
 
 package org.apache.ignite.internal.jdbc2;
 
+import java.sql.BatchUpdateException;
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.SQLWarning;
 import java.sql.Statement;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static java.sql.ResultSet.CONCUR_READ_ONLY;
 import static java.sql.ResultSet.FETCH_FORWARD;
@@ -44,7 +50,7 @@ public class JdbcStatement implements Statement {
     private static final int DFLT_FETCH_SIZE = 1024;
 
     /** Connection. */
-    private final JdbcConnection conn;
+    protected final JdbcConnection conn;
 
     /** Closed flag. */
     private boolean closed;
@@ -53,10 +59,10 @@ public class JdbcStatement implements Statement {
     private int maxRows;
 
     /** Current result set. */
-    private ResultSet rs;
+    protected ResultSet rs;
 
     /** Query arguments. */
-    protected Object[] args;
+    protected ArrayList<Object> args;
 
     /** Fetch size. */
     private int fetchSize = DFLT_FETCH_SIZE;
@@ -67,6 +73,12 @@ public class JdbcStatement implements Statement {
     /** Fields indexes. */
     Map<String, Integer> fieldsIdxs = new HashMap<>();
 
+    /** Current updated items count. */
+    long updateCnt = -1;
+
+    /** Batch statements. */
+    private List<String> batch;
+
     /**
      * Creates new statement.
      *
@@ -79,12 +91,15 @@ public class JdbcStatement implements Statement {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
     @Override public ResultSet executeQuery(String sql) throws SQLException {
         ensureNotClosed();
 
         rs = null;
 
-        if (sql == null || sql.isEmpty())
+        updateCnt = -1;
+
+        if (F.isEmpty(sql))
             throw new SQLException("SQL query is empty");
 
         Ignite ignite = conn.ignite();
@@ -95,8 +110,8 @@ public class JdbcStatement implements Statement {
 
         boolean loc = nodeId == null;
 
-        JdbcQueryTask qryTask = new JdbcQueryTask(loc ? ignite : null, conn.cacheName(),
-            sql, loc, args, fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins());
+        JdbcQueryTask qryTask = new JdbcQueryTask(loc ? ignite : null, conn.cacheName(), sql, loc, getArgs(),
+            fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins());
 
         try {
             JdbcQueryTask.QueryResult res =
@@ -111,6 +126,9 @@ public class JdbcStatement implements Statement {
 
             return rs;
         }
+        catch (IgniteSQLException e) {
+            throw e.toJdbcException();
+        }
         catch (Exception e) {
             throw new SQLException("Failed to query Ignite.", e);
         }
@@ -120,7 +138,82 @@ public class JdbcStatement implements Statement {
     @Override public int executeUpdate(String sql) throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        rs = null;
+
+        updateCnt = -1;
+
+        return doUpdate(sql, getArgs());
+    }
+
+    /**
+     * Run update query.
+     * @param sql SQL query.
+     * @param args Update arguments.
+     * @return Number of affected items.
+     * @throws SQLException
+     */
+    int doUpdate(String sql, Object[] args) throws SQLException {
+        if (F.isEmpty(sql))
+            throw new SQLException("SQL query is empty");
+
+        Ignite ignite = conn.ignite();
+
+        UUID nodeId = conn.nodeId();
+
+        UUID uuid = UUID.randomUUID();
+
+        boolean loc = nodeId == null;
+
+        if (!conn.isDmlSupported())
+            throw new SQLException("Failed to query Ignite: DML operations are supported in versions 1.8.0 and newer");
+
+        JdbcQueryTaskV2 qryTask = new JdbcQueryTaskV2(loc ? ignite : null, conn.cacheName(), sql, false, loc, args,
+            fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins());
+
+        try {
+            JdbcQueryTaskV2.QueryResult qryRes =
+                loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask);
+
+            Long res = updateCounterFromQueryResult(qryRes.getRows());
+
+            updateCnt = res;
+
+            return res.intValue();
+        }
+        catch (IgniteSQLException e) {
+            throw e.toJdbcException();
+        }
+        catch (SQLException e) {
+            throw e;
+        }
+        catch (Exception e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /**
+     * @param rows query result.
+     * @return update counter, if found
+     * @throws SQLException if getting an update counter from result proved to be impossible.
+     */
+    private static Long updateCounterFromQueryResult(List<List<?>> rows) throws SQLException {
+         if (F.isEmpty(rows))
+            return 0L;
+
+        if (rows.size() != 1)
+            throw new SQLException("Expected number of rows of 1 for update operation");
+
+        List<?> row = rows.get(0);
+
+        if (row.size() != 1)
+            throw new SQLException("Expected row size of 1 for update operation");
+
+        Object objRes = row.get(0);
+
+        if (!(objRes instanceof Long))
+            throw new SQLException("Unexpected update result type");
+
+        return (Long) objRes;
     }
 
     /** {@inheritDoc} */
@@ -220,11 +313,59 @@ public class JdbcStatement implements Statement {
 
     /** {@inheritDoc} */
     @Override public boolean execute(String sql) throws SQLException {
+        if (!conn.isDmlSupported()) {
+            // We attempt to run a query without any checks as long as server does not support DML anyway,
+            // so it simply will throw an exception when given a DML statement instead of a query.
+            rs = executeQuery(sql);
+
+            return true;
+        }
+
         ensureNotClosed();
 
-        rs = executeQuery(sql);
+        rs = null;
+
+        updateCnt = -1;
+
+        if (F.isEmpty(sql))
+            throw new SQLException("SQL query is empty");
+
+        Ignite ignite = conn.ignite();
+
+        UUID nodeId = conn.nodeId();
 
-        return true;
+        UUID uuid = UUID.randomUUID();
+
+        boolean loc = nodeId == null;
+
+        JdbcQueryTaskV2 qryTask = new JdbcQueryTaskV2(loc ? ignite : null, conn.cacheName(), sql, null, loc, getArgs(),
+            fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins());
+
+        try {
+            JdbcQueryTaskV2.QueryResult res =
+                loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask);
+
+            if (res.isQuery()) {
+                JdbcResultSet rs = new JdbcResultSet(uuid, this, res.getTbls(), res.getCols(), res.getTypes(),
+                    res.getRows(), res.isFinished());
+
+                rs.setFetchSize(fetchSize);
+
+                resSets.add(rs);
+
+                this.rs = rs;
+            }
+            else
+                updateCnt = updateCounterFromQueryResult(res.getRows());
+
+            return res.isQuery();
+        }
+        catch (IgniteSQLException e) {
+            throw e.toJdbcException();
+        }
+        catch (Exception e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
     }
 
     /** {@inheritDoc} */
@@ -242,7 +383,11 @@ public class JdbcStatement implements Statement {
     @Override public int getUpdateCount() throws SQLException {
         ensureNotClosed();
 
-        return -1;
+        long res = updateCnt;
+
+        updateCnt = -1;
+
+        return Long.valueOf(res).intValue();
     }
 
     /** {@inheritDoc} */
@@ -302,21 +447,27 @@ public class JdbcStatement implements Statement {
     @Override public void addBatch(String sql) throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        if (F.isEmpty(sql))
+            throw new SQLException("SQL query is empty");
+
+        if (batch == null)
+            batch = new ArrayList<>();
+
+        batch.add(sql);
     }
 
     /** {@inheritDoc} */
     @Override public void clearBatch() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        batch = null;
     }
 
     /** {@inheritDoc} */
     @Override public int[] executeBatch() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        throw new SQLFeatureNotSupportedException("Batch statements are not supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -340,28 +491,37 @@ public class JdbcStatement implements Statement {
     @Override public ResultSet getGeneratedKeys() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        throw new SQLFeatureNotSupportedException("Auto generated keys are not supported.");
     }
 
     /** {@inheritDoc} */
     @Override public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        if (autoGeneratedKeys == RETURN_GENERATED_KEYS)
+            throw new SQLFeatureNotSupportedException("Auto generated keys are not supported.");
+
+        return executeUpdate(sql);
     }
 
     /** {@inheritDoc} */
     @Override public int executeUpdate(String sql, int[] colIndexes) throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        if (!F.isEmpty(colIndexes))
+            throw new SQLFeatureNotSupportedException("Auto generated keys are not supported.");
+
+        return executeUpdate(sql);
     }
 
     /** {@inheritDoc} */
     @Override public int executeUpdate(String sql, String[] colNames) throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        if (!F.isEmpty(colNames))
+            throw new SQLFeatureNotSupportedException("Auto generated keys are not supported.");
+
+        return executeUpdate(sql);
     }
 
     /** {@inheritDoc} */
@@ -369,7 +529,7 @@ public class JdbcStatement implements Statement {
         ensureNotClosed();
 
         if (autoGeneratedKeys == RETURN_GENERATED_KEYS)
-            throw new SQLFeatureNotSupportedException("Updates are not supported.");
+            throw new SQLFeatureNotSupportedException("Auto generated keys are not supported.");
 
         return execute(sql);
     }
@@ -378,8 +538,8 @@ public class JdbcStatement implements Statement {
     @Override public boolean execute(String sql, int[] colIndexes) throws SQLException {
         ensureNotClosed();
 
-        if (colIndexes != null && colIndexes.length > 0)
-            throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        if (!F.isEmpty(colIndexes))
+            throw new SQLFeatureNotSupportedException("Auto generated keys are not supported.");
 
         return execute(sql);
     }
@@ -388,8 +548,8 @@ public class JdbcStatement implements Statement {
     @Override public boolean execute(String sql, String[] colNames) throws SQLException {
         ensureNotClosed();
 
-        if (colNames != null && colNames.length > 0)
-            throw new SQLFeatureNotSupportedException("Updates are not supported.");
+        if (!F.isEmpty(colNames))
+            throw new SQLFeatureNotSupportedException("Auto generated keys are not supported.");
 
         return execute(sql);
     }
@@ -448,11 +608,18 @@ public class JdbcStatement implements Statement {
     }
 
     /**
+     * @return Args for current statement
+     */
+    protected final Object[] getArgs() {
+        return args != null ? args.toArray() : null;
+    }
+
+    /**
      * Ensures that statement is not closed.
      *
      * @throws SQLException If statement is closed.
      */
-    protected void ensureNotClosed() throws SQLException {
+    void ensureNotClosed() throws SQLException {
         if (closed)
             throw new SQLException("Statement is closed.");
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcUtils.java
index b519340..38a838e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcUtils.java
@@ -17,13 +17,14 @@
 
 package org.apache.ignite.internal.jdbc2;
 
-import java.math.BigDecimal;
 import java.net.URL;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.Date;
 
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+
 import static java.sql.Types.BIGINT;
 import static java.sql.Types.BINARY;
 import static java.sql.Types.BOOLEAN;
@@ -132,24 +133,12 @@ public class JdbcUtils {
     }
 
     /**
-     * Checks whether type of the object is SQL-complaint.
+     * Checks whether a class is SQL-compliant.
      *
-     * @param obj Object.
-     * @return Whether type of the object is SQL-complaint.
+     * @param cls Class.
+     * @return Whether given type is SQL-compliant.
      */
-    public static boolean sqlType(Object obj) {
-        return obj == null ||
-            obj instanceof BigDecimal ||
-            obj instanceof Boolean ||
-            obj instanceof Byte ||
-            obj instanceof byte[] ||
-            obj instanceof java.util.Date ||
-            obj instanceof Double ||
-            obj instanceof Float ||
-            obj instanceof Integer ||
-            obj instanceof Long ||
-            obj instanceof Short ||
-            obj instanceof String ||
-            obj instanceof URL;
+    static boolean isSqlType(Class<?> cls) {
+        return GridQueryProcessor.isSqlType(cls) || cls == URL.class;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
index 5a46d65..852c432 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
@@ -17,7 +17,12 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.sql.ResultSet;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
@@ -43,7 +48,10 @@ public class QueryCursorImpl<T> implements QueryCursorEx<T> {
         AtomicReferenceFieldUpdater.newUpdater(QueryCursorImpl.class, State.class, "state");
 
     /** Query executor. */
-    private Iterable<T> iterExec;
+    private final Iterable<T> iterExec;
+
+    /** Result type flag - result set or update counter. */
+    private final boolean isQry;
 
     /** */
     private Iterator<T> iter;
@@ -62,8 +70,7 @@ public class QueryCursorImpl<T> implements QueryCursorEx<T> {
      * @param cancel Cancellation closure.
      */
     public QueryCursorImpl(Iterable<T> iterExec, GridQueryCancel cancel) {
-        this.iterExec = iterExec;
-        this.cancel = cancel;
+        this(iterExec, cancel, true);
     }
 
     /**
@@ -73,6 +80,16 @@ public class QueryCursorImpl<T> implements QueryCursorEx<T> {
         this(iterExec, null);
     }
 
+    /**
+     * @param iterExec Query executor.
+     * @param isQry Result type flag - {@code true} for query, {@code false} for update operation.
+     */
+    public QueryCursorImpl(Iterable<T> iterExec, GridQueryCancel cancel, boolean isQry) {
+        this.iterExec = iterExec;
+        this.cancel = cancel;
+        this.isQry = isQry;
+    }
+
     /** {@inheritDoc} */
     @Override public Iterator<T> iterator() {
         if (!STATE_UPDATER.compareAndSet(this, IDLE, EXECUTION))
@@ -154,6 +171,14 @@ public class QueryCursorImpl<T> implements QueryCursorEx<T> {
     }
 
     /**
+     * @return {@code true} if this cursor corresponds to a {@link ResultSet} as a result of query,
+     * {@code false} if query was modifying operation like INSERT, UPDATE, or DELETE.
+     */
+    public boolean isQuery() {
+        return isQry;
+    }
+
+    /**
      * @param fieldsMeta SQL Fields query result metadata.
      */
     public void fieldsMeta(List<GridQueryFieldMetadata> fieldsMeta) {


[39/52] ignite git commit: IGNITE-4319: IgniteCacheAbstractSqlDmlQuerySelfTest fix. This closes #1291.

Posted by vo...@apache.org.
IGNITE-4319: IgniteCacheAbstractSqlDmlQuerySelfTest fix. This closes #1291.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/99eb4e0c
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/99eb4e0c
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/99eb4e0c

Branch: refs/heads/master
Commit: 99eb4e0c802a5c967a4b4feda5e7de252912b962
Parents: b0127d3
Author: Alexander Paschenko <al...@gmail.com>
Authored: Mon Nov 28 13:28:16 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Nov 28 13:28:16 2016 +0300

----------------------------------------------------------------------
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java | 44 ++++++++++++++++----
 1 file changed, 37 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/99eb4e0c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
index 123c32a..1424163 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
@@ -89,25 +89,53 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
         startGridsMultiThreaded(3, true);
 
         ignite(0).createCache(cacheConfig("S2P", true, false).setIndexedTypes(String.class, Person.class));
-        ignite(0).createCache(createBinCacheConfig());
+
+        if (isBinaryMarshaller())
+            ignite(0).createCache(createBinCacheConfig());
     }
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
+
+        // Put to S2P only non binary Persons
         ignite(0).cache("S2P").put("FirstKey", new Person(1, "John", "White"));
         ignite(0).cache("S2P").put("SecondKey", new Person(2, "Joe", "Black"));
         ignite(0).cache("S2P").put("k3", new Person(3, "Sylvia", "Green"));
         ignite(0).cache("S2P").put("f0u4thk3y", new Person(4, "Jane", "Silver"));
 
-        ignite(0).cache("S2P-bin").put("FirstKey", createPerson(1, "John", "White"));
-        ignite(0).cache("S2P-bin").put("SecondKey", createPerson(2, "Joe", "Black"));
-        ignite(0).cache("S2P-bin").put("k3", createPerson(3, "Sylvia", "Green"));
-        ignite(0).cache("S2P-bin").put("f0u4thk3y", createPerson(4, "Jane", "Silver"));
+        if (isBinaryMarshaller()) {
+            ignite(0).cache("S2P-bin").put("FirstKey", createBinPerson(1, "John", "White"));
+            ignite(0).cache("S2P-bin").put("SecondKey", createBinPerson(2, "Joe", "Black"));
+            ignite(0).cache("S2P-bin").put("k3", createBinPerson(3, "Sylvia", "Green"));
+            ignite(0).cache("S2P-bin").put("f0u4thk3y", createBinPerson(4, "Jane", "Silver"));
+        }
     }
 
-    /** */
+    /**
+     * Create person.
+     *
+     * @param id ID.
+     * @param name Name.
+     * @param secondName Second name.
+     * @return Person.
+     */
     Object createPerson(int id, String name, String secondName) {
+        if (!isBinaryMarshaller())
+            return new Person(id, name, secondName);
+        else
+            return createBinPerson(id, name, secondName);
+    }
+
+    /**
+     * Create binary person.
+     *
+     * @param id ID.
+     * @param name Name.
+     * @param secondName Second name.
+     * @return Person.
+     */
+    private Object createBinPerson(int id, String name, String secondName) {
         BinaryObjectBuilder bldr = ignite(0).binary().builder("Person");
 
         bldr.setField("id", id);
@@ -117,7 +145,9 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
         return bldr.build();
     }
 
-    /** */
+    /**
+     * @return Cache.
+     */
     protected IgniteCache cache() {
         if (!isBinaryMarshaller())
             return ignite(0).cache("S2P");


[35/52] ignite git commit: IGNITE-4308: Ignore QueryEntity.keyFields when key is an SQL type. This closes #1286.

Posted by vo...@apache.org.
IGNITE-4308: Ignore QueryEntity.keyFields when key is an SQL type. This closes #1286.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/2ca709f8
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/2ca709f8
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/2ca709f8

Branch: refs/heads/master
Commit: 2ca709f8efb83c33ebe85d4b45844644a6fee7d4
Parents: ac6d910
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Nov 25 19:17:49 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 19:17:49 2016 +0300

----------------------------------------------------------------------
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java       |  2 --
 .../internal/processors/query/GridQueryProcessor.java | 14 +++++++++++++-
 .../IgniteCacheAbstractInsertSqlQuerySelfTest.java    |  8 --------
 .../cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java |  2 --
 4 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2ca709f8/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
index 1590067..4a97aef 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
@@ -118,8 +118,6 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac
         e.addQueryField("firstName", String.class.getName(), null);
         e.addQueryField("lastName", String.class.getName(), null);
 
-        e.setKeyFields(Collections.<String>emptySet());
-
         ccfg.setQueryEntities(Collections.singletonList(e));
 
         return cfg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/2ca709f8/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 0fc6044..1594cee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -1502,10 +1502,22 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             aliases = Collections.emptyMap();
 
         Set<String> keyFields = qryEntity.getKeyFields();
+
+        // We have to distinguish between empty and null keyFields when the key is not of SQL type -
+        // when a key is not of SQL type, absence of a field in nonnull keyFields tell us that this field
+        // is a value field, and null keyFields tells us that current configuration
+        // does not tell us anything about this field's ownership.
         boolean hasKeyFields = (keyFields != null);
 
+        boolean isKeyClsSqlType = isSqlType(d.keyClass());
+
         for (Map.Entry<String, String> entry : qryEntity.getFields().entrySet()) {
-            Boolean isKeyField = (hasKeyFields ? keyFields.contains(entry.getKey()) : null);
+            Boolean isKeyField;
+
+            if (isKeyClsSqlType) // We don't care about keyFields in this case - it might be null, or empty, or anything
+                isKeyField = false;
+            else
+                isKeyField = (hasKeyFields ? keyFields.contains(entry.getKey()) : null);
 
             BinaryProperty prop = buildBinaryProperty(entry.getKey(), U.classForName(entry.getValue(), Object.class, true),
                 aliases, isKeyField);

http://git-wip-us.apache.org/repos/asf/ignite/blob/2ca709f8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
index 2c3ab23..47369ee 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
@@ -148,8 +148,6 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
 
             QueryEntity s2p = new QueryEntity(String.class.getName(), "Person");
 
-            s2p.setKeyFields(Collections.<String>emptySet());
-
             LinkedHashMap<String, String> flds = new LinkedHashMap<>();
 
             flds.put("id", Integer.class.getName());
@@ -161,8 +159,6 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
 
             QueryEntity s2s = new QueryEntity(String.class.getName(), String.class.getName());
 
-            s2s.setKeyFields(Collections.<String>emptySet());
-
             s2pCcfg.setQueryEntities(Arrays.asList(s2p, s2s));
 
             ignite(0).createCache(s2pCcfg);
@@ -173,8 +169,6 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
 
             QueryEntity i2p = new QueryEntity(Integer.class.getName(), "Person");
 
-            i2p.setKeyFields(Collections.<String>emptySet());
-
             LinkedHashMap<String, String> flds = new LinkedHashMap<>();
 
             flds.put("id", Integer.class.getName());
@@ -285,8 +279,6 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
 
             QueryEntity i2i = new QueryEntity(Integer.class.getName(), Integer.class.getName());
 
-            i2i.setKeyFields(Collections.<String>emptySet());
-
             i2i.setFields(new LinkedHashMap<String, String>());
 
             i2i.setIndexes(Collections.<QueryIndex>emptyList());

http://git-wip-us.apache.org/repos/asf/ignite/blob/2ca709f8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
index 22116a9..123c32a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
@@ -153,8 +153,6 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
 
         QueryEntity e = new QueryEntity(String.class.getName(), "Person");
 
-        e.setKeyFields(Collections.<String>emptySet());
-
         LinkedHashMap<String, String> flds = new LinkedHashMap<>();
 
         flds.put("id", Integer.class.getName());


[33/52] ignite git commit: IGNITE-4287 DmlStatementsProcessor logic fix (always force keepBinary for SELECTs inside DML logic). This closes #1280.

Posted by vo...@apache.org.
IGNITE-4287 DmlStatementsProcessor logic fix (always force keepBinary for SELECTs inside DML logic). This closes #1280.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/e7c0d453
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/e7c0d453
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/e7c0d453

Branch: refs/heads/master
Commit: e7c0d453e53821d11f92f102ccfc06e2426d2dce
Parents: dda4fc9
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Nov 25 19:07:50 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 25 19:07:50 2016 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        | 209 +++++++++----------
 .../processors/query/h2/sql/DmlAstUtils.java    |  19 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |  46 ++--
 .../IgniteCacheDeleteSqlQuerySelfTest.java      |  25 +++
 4 files changed, 162 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e7c0d453/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 7634965..469e36c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -46,7 +46,6 @@ import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.processors.cache.CacheOperationContext;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
@@ -130,9 +129,34 @@ public class DmlStatementsProcessor {
 
         UpdatePlan plan = getPlanForStatement(spaceName, stmt, null);
 
+        GridCacheContext<?, ?> cctx = plan.tbl.rowDescriptor().context();
+
         for (int i = 0; i < DFLT_DML_RERUN_ATTEMPTS; i++) {
-            UpdateResult r = executeUpdateStatement(plan.tbl.rowDescriptor().context(), stmt, fieldsQry, loc, filters,
-                cancel, errKeys);
+            CacheOperationContext opCtx = cctx.operationContextPerCall();
+
+            // Force keepBinary for operation context to avoid binary deserialization inside entry processor
+            if (cctx.binaryMarshaller()) {
+                CacheOperationContext newOpCtx = null;
+
+                if (opCtx == null)
+                    // Mimics behavior of GridCacheAdapter#keepBinary and GridCacheProxyImpl#keepBinary
+                    newOpCtx = new CacheOperationContext(false, null, true, null, false, null);
+                else if (!opCtx.isKeepBinary())
+                    newOpCtx = opCtx.keepBinary();
+
+                if (newOpCtx != null)
+                    cctx.operationContextPerCall(newOpCtx);
+            }
+
+            UpdateResult r;
+
+            try {
+                r = executeUpdateStatement(cctx, stmt, fieldsQry, loc, filters,
+                    cancel, errKeys);
+            }
+            finally {
+                cctx.operationContextPerCall(opCtx);
+            }
 
             if (F.isEmpty(r.errKeys))
                 return r.cnt + items;
@@ -195,12 +219,16 @@ public class DmlStatementsProcessor {
         throws IgniteCheckedException {
         Integer errKeysPos = null;
 
-        if (!F.isEmpty(failedKeys))
-            errKeysPos = F.isEmpty(fieldsQry.getArgs()) ? 1 : fieldsQry.getArgs().length + 1;
+        Object[] params = fieldsQry.getArgs();
 
-        UpdatePlan plan = getPlanForStatement(cctx.name(), prepStmt, errKeysPos);
+        if (!F.isEmpty(failedKeys)) {
+            int paramsCnt = F.isEmpty(params) ? 0 : params.length;
+            params = Arrays.copyOf(U.firstNotNull(params, X.EMPTY_OBJECT_ARRAY), paramsCnt + 1);
+            params[paramsCnt] = failedKeys;
+            errKeysPos = paramsCnt; // Last position
+        }
 
-        Object[] params = fieldsQry.getArgs();
+        UpdatePlan plan = getPlanForStatement(cctx.name(), prepStmt, errKeysPos);
 
         if (plan.fastUpdateArgs != null) {
             assert F.isEmpty(failedKeys) && errKeysPos == null;
@@ -351,78 +379,58 @@ public class DmlStatementsProcessor {
         // With DELETE, we have only two columns - key and value.
         long res = 0;
 
-        CacheOperationContext opCtx = cctx.operationContextPerCall();
-
-        // Force keepBinary for operation context to avoid binary deserialization inside entry processor
-        if (cctx.binaryMarshaller()) {
-            CacheOperationContext newOpCtx = null;
-
-            if (opCtx == null)
-                // Mimics behavior of GridCacheAdapter#keepBinary and GridCacheProxyImpl#keepBinary
-                newOpCtx = new CacheOperationContext(false, null, true, null, false, null);
-            else if (!opCtx.isKeepBinary())
-                newOpCtx = opCtx.keepBinary();
-
-            if (newOpCtx != null)
-                cctx.operationContextPerCall(newOpCtx);
-        }
-
         // Keys that failed to DELETE due to concurrent updates.
         List<Object> failedKeys = new ArrayList<>();
 
         SQLException resEx = null;
 
-        try {
-            Iterator<List<?>> it = cursor.iterator();
-            Map<Object, EntryProcessor<Object, Object, Boolean>> rows = new LinkedHashMap<>();
 
-            while (it.hasNext()) {
-                List<?> e = it.next();
-                if (e.size() != 2) {
-                    U.warn(indexing.getLogger(), "Invalid row size on DELETE - expected 2, got " + e.size());
-                    continue;
-                }
+        Iterator<List<?>> it = cursor.iterator();
+        Map<Object, EntryProcessor<Object, Object, Boolean>> rows = new LinkedHashMap<>();
 
-                rows.put(e.get(0), new ModifyingEntryProcessor(e.get(1), RMV));
+        while (it.hasNext()) {
+            List<?> e = it.next();
+            if (e.size() != 2) {
+                U.warn(indexing.getLogger(), "Invalid row size on DELETE - expected 2, got " + e.size());
+                continue;
+            }
 
-                if ((pageSize > 0 && rows.size() == pageSize) || (!it.hasNext())) {
-                    PageProcessingResult pageRes = processPage(cctx, rows);
+            rows.put(e.get(0), new ModifyingEntryProcessor(e.get(1), RMV));
 
-                    res += pageRes.cnt;
+            if ((pageSize > 0 && rows.size() == pageSize) || (!it.hasNext())) {
+                PageProcessingResult pageRes = processPage(cctx, rows);
 
-                    failedKeys.addAll(F.asList(pageRes.errKeys));
+                res += pageRes.cnt;
 
-                    if (pageRes.ex != null) {
-                        if (resEx == null)
-                            resEx = pageRes.ex;
-                        else
-                            resEx.setNextException(pageRes.ex);
-                    }
+                failedKeys.addAll(F.asList(pageRes.errKeys));
 
-                    if (it.hasNext())
-                        rows.clear(); // No need to clear after the last batch.
+                if (pageRes.ex != null) {
+                    if (resEx == null)
+                        resEx = pageRes.ex;
+                    else
+                        resEx.setNextException(pageRes.ex);
                 }
-            }
 
-            if (resEx != null) {
-                if (!F.isEmpty(failedKeys)) {
-                    // Don't go for a re-run if processing of some keys yielded exceptions and report keys that
-                    // had been modified concurrently right away.
-                    String msg = "Failed to DELETE some keys because they had been modified concurrently " +
-                        "[keys=" + failedKeys + ']';
+                if (it.hasNext())
+                    rows.clear(); // No need to clear after the last batch.
+            }
+        }
 
-                    SQLException conEx = createJdbcSqlException(msg, IgniteQueryErrorCode.CONCURRENT_UPDATE);
+        if (resEx != null) {
+            if (!F.isEmpty(failedKeys)) {
+                // Don't go for a re-run if processing of some keys yielded exceptions and report keys that
+                // had been modified concurrently right away.
+                String msg = "Failed to DELETE some keys because they had been modified concurrently " +
+                    "[keys=" + failedKeys + ']';
 
-                    conEx.setNextException(resEx);
+                SQLException conEx = createJdbcSqlException(msg, IgniteQueryErrorCode.CONCURRENT_UPDATE);
 
-                    resEx = conEx;
-                }
+                conEx.setNextException(resEx);
 
-                throw new IgniteSQLException(resEx);
+                resEx = conEx;
             }
-        }
-        finally {
-            cctx.operationContextPerCall(opCtx);
+
+            throw new IgniteSQLException(resEx);
         }
 
         return new UpdateResult(res, failedKeys.toArray());
@@ -689,22 +697,6 @@ public class DmlStatementsProcessor {
                     IgniteQueryErrorCode.DUPLICATE_KEY);
         }
         else {
-            CacheOperationContext opCtx = cctx.operationContextPerCall();
-
-            // Force keepBinary for operation context to avoid binary deserialization inside entry processor
-            if (cctx.binaryMarshaller()) {
-                CacheOperationContext newOpCtx = null;
-
-                if (opCtx == null)
-                    // Mimics behavior of GridCacheAdapter#keepBinary and GridCacheProxyImpl#keepBinary
-                    newOpCtx = new CacheOperationContext(false, null, true, null, false, null);
-                else if (!opCtx.isKeepBinary())
-                    newOpCtx = opCtx.keepBinary();
-
-                if (newOpCtx != null)
-                    cctx.operationContextPerCall(newOpCtx);
-            }
-
             Map<Object, EntryProcessor<Object, Object, Boolean>> rows = plan.isLocSubqry ?
                 new LinkedHashMap<Object, EntryProcessor<Object, Object, Boolean>>(plan.rowsNum) :
                 new LinkedHashMap<Object, EntryProcessor<Object, Object, Boolean>>();
@@ -716,55 +708,50 @@ public class DmlStatementsProcessor {
 
             SQLException resEx = null;
 
-            try {
-                Iterator<List<?>> it = cursor.iterator();
-
-                while (it.hasNext()) {
-                    List<?> row = it.next();
+            Iterator<List<?>> it = cursor.iterator();
 
-                    final IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.keySupplier,
-                        plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc.type());
+            while (it.hasNext()) {
+                List<?> row = it.next();
 
-                    rows.put(t.getKey(), new InsertEntryProcessor(t.getValue()));
+                final IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.keySupplier,
+                    plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc.type());
 
-                    if (!it.hasNext() || (pageSize > 0 && rows.size() == pageSize)) {
-                        PageProcessingResult pageRes = processPage(cctx, rows);
+                rows.put(t.getKey(), new InsertEntryProcessor(t.getValue()));
 
-                        resCnt += pageRes.cnt;
+                if (!it.hasNext() || (pageSize > 0 && rows.size() == pageSize)) {
+                    PageProcessingResult pageRes = processPage(cctx, rows);
 
-                        duplicateKeys.addAll(F.asList(pageRes.errKeys));
+                    resCnt += pageRes.cnt;
 
-                        if (pageRes.ex != null) {
-                            if (resEx == null)
-                                resEx = pageRes.ex;
-                            else
-                                resEx.setNextException(pageRes.ex);
-                        }
+                    duplicateKeys.addAll(F.asList(pageRes.errKeys));
 
-                        rows.clear();
+                    if (pageRes.ex != null) {
+                        if (resEx == null)
+                            resEx = pageRes.ex;
+                        else
+                            resEx.setNextException(pageRes.ex);
                     }
+
+                    rows.clear();
                 }
+            }
 
-                if (!F.isEmpty(duplicateKeys)) {
-                    String msg = "Failed to INSERT some keys because they are already in cache " +
-                        "[keys=" + duplicateKeys + ']';
+            if (!F.isEmpty(duplicateKeys)) {
+                String msg = "Failed to INSERT some keys because they are already in cache " +
+                    "[keys=" + duplicateKeys + ']';
 
-                    SQLException dupEx = new SQLException(msg, null, IgniteQueryErrorCode.DUPLICATE_KEY);
+                SQLException dupEx = new SQLException(msg, null, IgniteQueryErrorCode.DUPLICATE_KEY);
 
-                    if (resEx == null)
-                        resEx = dupEx;
-                    else
-                        resEx.setNextException(dupEx);
-                }
+                if (resEx == null)
+                    resEx = dupEx;
+                else
+                    resEx.setNextException(dupEx);
+            }
 
-                if (resEx != null)
-                    throw new IgniteSQLException(resEx);
+            if (resEx != null)
+                throw new IgniteSQLException(resEx);
 
-                return resCnt;
-            }
-            finally {
-                cctx.operationContextPerCall(opCtx);
-            }
+            return resCnt;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e7c0d453/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
index 39b1b74..5ff715e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
@@ -417,7 +417,24 @@ public final class DmlAstUtils {
      * @return New condition.
      */
     private static GridSqlElement injectKeysFilterParam(GridSqlElement where, GridSqlColumn keyCol, int paramIdx) {
-        GridSqlElement e = new GridSqlOperation(GridSqlOperationType.IN, keyCol, new GridSqlParameter(paramIdx));
+        // Yes, we need a subquery for "WHERE _key IN ?" to work with param being an array without dirty query rewriting.
+        GridSqlSelect sel = new GridSqlSelect();
+
+        GridSqlFunction from = new GridSqlFunction(GridSqlFunctionType.TABLE);
+
+        sel.from(from);
+
+        GridSqlColumn col = new GridSqlColumn(null, from, "_IGNITE_ERR_KEYS", "TABLE._IGNITE_ERR_KEYS");
+
+        sel.addColumn(col, true);
+
+        GridSqlAlias alias = new GridSqlAlias("_IGNITE_ERR_KEYS", new GridSqlParameter(paramIdx));
+
+        alias.resultType(keyCol.resultType());
+
+        from.addChild(alias);
+
+        GridSqlElement e = new GridSqlOperation(GridSqlOperationType.IN, keyCol, new GridSqlSubquery(sel));
 
         if (where == null)
             return e;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e7c0d453/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
index 2dbf1b4..22116a9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
@@ -88,45 +88,41 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
     @Override protected void beforeTestsStarted() throws Exception {
         startGridsMultiThreaded(3, true);
 
-        ignite(0).createCache(cacheConfig());
+        ignite(0).createCache(cacheConfig("S2P", true, false).setIndexedTypes(String.class, Person.class));
+        ignite(0).createCache(createBinCacheConfig());
     }
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
-        ignite(0).cache("S2P").put("FirstKey", createPerson(1, "John", "White"));
-        ignite(0).cache("S2P").put("SecondKey", createPerson(2, "Joe", "Black"));
-        ignite(0).cache("S2P").put("k3", createPerson(3, "Sylvia", "Green"));
-        ignite(0).cache("S2P").put("f0u4thk3y", createPerson(4, "Jane", "Silver"));
+        ignite(0).cache("S2P").put("FirstKey", new Person(1, "John", "White"));
+        ignite(0).cache("S2P").put("SecondKey", new Person(2, "Joe", "Black"));
+        ignite(0).cache("S2P").put("k3", new Person(3, "Sylvia", "Green"));
+        ignite(0).cache("S2P").put("f0u4thk3y", new Person(4, "Jane", "Silver"));
+
+        ignite(0).cache("S2P-bin").put("FirstKey", createPerson(1, "John", "White"));
+        ignite(0).cache("S2P-bin").put("SecondKey", createPerson(2, "Joe", "Black"));
+        ignite(0).cache("S2P-bin").put("k3", createPerson(3, "Sylvia", "Green"));
+        ignite(0).cache("S2P-bin").put("f0u4thk3y", createPerson(4, "Jane", "Silver"));
     }
 
     /** */
     Object createPerson(int id, String name, String secondName) {
-        if (!isBinaryMarshaller())
-            return new Person(id, name, secondName);
-        else {
-            BinaryObjectBuilder bldr = ignite(0).binary().builder("Person");
-
-            bldr.setField("id", id);
-            bldr.setField("name", name);
-            bldr.setField("secondName", secondName);
-
-            return bldr.build();
-        }
+        BinaryObjectBuilder bldr = ignite(0).binary().builder("Person");
 
-    }
+        bldr.setField("id", id);
+        bldr.setField("name", name);
+        bldr.setField("secondName", secondName);
 
-    /** */
-    protected IgniteCache<?, ?> cache() {
-        return ignite(0).cache("S2P").withKeepBinary();
+        return bldr.build();
     }
 
     /** */
-    protected CacheConfiguration cacheConfig() {
+    protected IgniteCache cache() {
         if (!isBinaryMarshaller())
-            return cacheConfig("S2P", true, false).setIndexedTypes(String.class, Person.class);
+            return ignite(0).cache("S2P");
         else
-            return createBinCacheConfig();
+            return ignite(0).cache("S2P-bin").withKeepBinary();
     }
 
     /** {@inheritDoc} */
@@ -153,7 +149,7 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
      *
      */
     private static CacheConfiguration createBinCacheConfig() {
-        CacheConfiguration ccfg = cacheConfig("S2P", true, false);
+        CacheConfiguration ccfg = cacheConfig("S2P-bin", true, false);
 
         QueryEntity e = new QueryEntity(String.class.getName(), "Person");
 
@@ -177,7 +173,7 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
     /**
      *
      */
-    private static class Person implements Serializable {
+    static class Person implements Serializable {
         /** */
         public Person(int id, String name, String secondName) {
             this.id = id;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e7c0d453/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
index 8b0a033..12662db 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
@@ -78,4 +78,29 @@ public class IgniteCacheDeleteSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
         assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
             leftovers.get(2));
     }
+
+    /**
+     * In binary mode, this test checks that inner forcing of keepBinary works - without it, EntryProcessors
+     * inside DML engine would compare binary and non-binary objects with the same keys and thus fail.
+     */
+    public void testDeleteSimpleWithoutKeepBinary() {
+        IgniteCache p = ignite(0).cache("S2P");
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("delete from Person p where length(p._key) = 2 " +
+            "or p.secondName like '%ite'"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by id"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(2, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("SecondKey", new Person(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", new Person(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(1));
+    }
 }


[21/52] ignite git commit: IGNITE-4306 .NET: Add diagnostic information to examples test

Posted by vo...@apache.org.
IGNITE-4306 .NET: Add diagnostic information to examples test


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b04460c6
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b04460c6
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b04460c6

Branch: refs/heads/master
Commit: b04460c6b3ad5891f12a23ad6dc8a08625aad031
Parents: 7b35ba7
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Nov 25 12:11:49 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Nov 25 12:11:49 2016 +0300

----------------------------------------------------------------------
 .../Examples/ExamplesTest.cs                    |  7 +++-
 .../Process/IgniteProcess.cs                    | 40 ++++++--------------
 2 files changed, 16 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b04460c6/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
index 8c017d1..61bfb5c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
@@ -92,8 +92,11 @@ namespace Apache.Ignite.Core.Tests.Examples
 
                     var proc = new IgniteProcess(args.ToArray());
 
-                    Assert.IsTrue(ignite.WaitTopology(i + 2));
-                    Assert.IsTrue(proc.Alive);
+                    Assert.IsTrue(ignite.WaitTopology(i + 2), 
+                        string.Format("Standalone node failed to join topology: [{0}]", proc.GetInfo()));
+
+                    Assert.IsTrue(proc.Alive, string.Format("Standalone node stopped unexpectedly: [{0}]", 
+                        proc.GetInfo()));
                 }
 
                 Ignition.ClientMode = clientMode;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b04460c6/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
index 852551a..11fe614 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
@@ -214,6 +214,17 @@ namespace Apache.Ignite.Core.Tests.Process
         }
 
         /// <summary>
+        /// Gets the process.
+        /// </summary>
+        public string GetInfo()
+        {
+            return Alive
+                ? string.Format("Id={0}, Alive={1}", _proc.Id, Alive)
+                : string.Format("Id={0}, Alive={1}, ExitCode={2}, ExitTime={3}",
+                    _proc.Id, Alive, _proc.ExitCode, _proc.ExitTime);
+        }
+
+        /// <summary>
         /// Kill process.
         /// </summary>
         public void Kill()
@@ -249,35 +260,6 @@ namespace Apache.Ignite.Core.Tests.Process
         }
 
         /// <summary>
-        /// Join process with timeout.
-        /// </summary>
-        /// <param name="timeout">Timeout in milliseconds.</param>
-        /// <returns><c>True</c> if process exit occurred before timeout.</returns>
-        public bool Join(int timeout)
-        {
-            return _proc.WaitForExit(timeout);
-        }
-
-        /// <summary>
-        /// Join process with timeout.
-        /// </summary>
-        /// <param name="timeout">Timeout in milliseconds.</param>
-        /// <param name="exitCode">Exit code.</param>
-        /// <returns><c>True</c> if process exit occurred before timeout.</returns>
-        public bool Join(int timeout, out int exitCode)
-        {
-            if (_proc.WaitForExit(timeout))
-            {
-                exitCode = _proc.ExitCode;
-
-                return true;
-            }
-            exitCode = 0;
-
-            return false;
-        }
-
-        /// <summary>
         /// Attach output reader to the process.
         /// </summary>
         /// <param name="proc">Process.</param>