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 2017/07/11 09:27:16 UTC

[17/50] ignite git commit: IGNITE-5204: SQL: fixed incorrect partition calculation in case of unicast optimization, when WHERE clause argument type was different from key or affinity key type. This close #2107.

IGNITE-5204: SQL: fixed incorrect partition calculation in case of unicast optimization, when WHERE clause argument type was different from key or affinity key type. This close #2107.


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

Branch: refs/heads/master
Commit: 99713feea764fc8c3e5b247a24698a2c04d3bcf4
Parents: 55a5ca0
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Fri Jul 7 16:30:37 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Jul 7 16:30:37 2017 +0300

----------------------------------------------------------------------
 .../cache/query/CacheQueryPartitionInfo.java    |  46 +++-
 .../query/h2/DmlStatementsProcessor.java        |   9 +-
 .../internal/processors/query/h2/H2Utils.java   |  27 +++
 .../processors/query/h2/IgniteH2Indexing.java   |  30 ++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   5 +-
 .../processors/query/IgniteSqlRoutingTest.java  | 211 +++++++++++++++++--
 6 files changed, 290 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPartitionInfo.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPartitionInfo.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPartitionInfo.java
index 1329d5c..42bf070 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPartitionInfo.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPartitionInfo.java
@@ -39,22 +39,35 @@ import org.apache.ignite.internal.util.typedef.internal.S;
  */
 public class CacheQueryPartitionInfo {
     /** */
-    private int partId;
+    private final int partId;
 
     /** */
-    private String cacheName;
+    private final String cacheName;
 
     /** */
-    private int paramIdx;
+    private final String tableName;
+
+    /** */
+    private final int dataType;
+
+    /** */
+    private final int paramIdx;
 
     /**
      * @param partId Partition id, or -1 if parameter binding required.
      * @param cacheName Cache name required for partition calculation.
+     * @param tableName Table name required for proper type conversion.
+     * @param dataType Required data type id for the query parameter.
      * @param paramIdx Query parameter index required for partition calculation.
      */
-    public CacheQueryPartitionInfo(int partId, String cacheName, int paramIdx) {
+    public CacheQueryPartitionInfo(int partId, String cacheName, String tableName, int dataType, int paramIdx) {
+        // In case partition is not known, both cacheName and tableName must be provided.
+        assert (partId >= 0) ^ ((cacheName != null) && (tableName != null));
+
         this.partId = partId;
         this.cacheName = cacheName;
+        this.tableName = tableName;
+        this.dataType = dataType;
         this.paramIdx = paramIdx;
     }
 
@@ -73,6 +86,20 @@ public class CacheQueryPartitionInfo {
     }
 
     /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tableName;
+    }
+
+    /**
+     * @return Required data type for the query parameter.
+     */
+    public int dataType() {
+        return dataType;
+    }
+
+    /**
      * @return Query parameter index required for partition calculation.
      */
     public int paramIdx() {
@@ -81,7 +108,9 @@ public class CacheQueryPartitionInfo {
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return partId ^ paramIdx ^ (cacheName == null ? 0 : cacheName.hashCode());
+        return partId ^ dataType ^ paramIdx ^
+            (cacheName == null ? 0 : cacheName.hashCode()) ^
+            (tableName == null ? 0 : tableName.hashCode());
     }
 
     /** {@inheritDoc} */
@@ -97,10 +126,13 @@ public class CacheQueryPartitionInfo {
         if (partId >= 0)
             return partId == other.partId;
 
-        if (other.cacheName == null)
+        if (other.cacheName == null || other.tableName == null)
             return false;
 
-        return other.cacheName.equals(cacheName) && other.paramIdx == paramIdx;
+        return other.cacheName.equals(cacheName) &&
+            other.tableName.equals(tableName) &&
+            other.dataType == dataType &&
+            other.paramIdx == paramIdx;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/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 0c1dbf9..4f7c288 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
@@ -748,14 +748,7 @@ public class DmlStatementsProcessor {
             return newArr;
         }
 
-        int objType = DataType.getTypeFromClass(val.getClass());
-
-        if (objType == type)
-            return val;
-
-        Value h2Val = desc.wrap(val, objType);
-
-        return h2Val.convertTo(type).getObject();
+        return H2Utils.convert(val, desc, type);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
index ee88acf..157e1ba 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
@@ -29,6 +30,8 @@ import org.h2.engine.Session;
 import org.h2.jdbc.JdbcConnection;
 import org.h2.result.SortOrder;
 import org.h2.table.IndexColumn;
+import org.h2.value.DataType;
+import org.h2.value.Value;
 
 import java.lang.reflect.Constructor;
 import java.sql.Connection;
@@ -236,6 +239,30 @@ public class H2Utils {
     }
 
     /**
+     * Convert value to column's expected type by means of H2.
+     *
+     * @param val Source value.
+     * @param desc Row descriptor.
+     * @param type Expected column type to convert to.
+     * @return Converted object.
+     * @throws IgniteCheckedException if failed.
+     */
+    public static Object convert(Object val, GridH2RowDescriptor desc, int type)
+        throws IgniteCheckedException {
+        if (val == null)
+            return null;
+
+        int objType = DataType.getTypeFromClass(val.getClass());
+
+        if (objType == type)
+            return val;
+
+        Value h2Val = desc.wrap(val, objType);
+
+        return h2Val.convertTo(type).getObject();
+    }
+
+    /**
      * Private constructor.
      */
     private H2Utils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/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 255c5f1..40eae17 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
@@ -2265,7 +2265,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Bind query parameters and calculate partitions derived from the query.
      *
+     * @param partInfoList Collection of query derived partition info.
+     * @param params Query parameters.
      * @return Partitions.
+     * @throws IgniteCheckedException, If fails.
      */
     private int[] calculateQueryPartitions(CacheQueryPartitionInfo[] partInfoList, Object[] params)
         throws IgniteCheckedException {
@@ -2273,9 +2276,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         ArrayList<Integer> list = new ArrayList<>(partInfoList.length);
 
         for (CacheQueryPartitionInfo partInfo: partInfoList) {
-            int partId = partInfo.partition() < 0 ?
-                kernalContext().affinity().partition(partInfo.cacheName(), params[partInfo.paramIdx()]) :
-                partInfo.partition();
+            int partId = (partInfo.partition() >= 0) ? partInfo.partition() :
+                bindPartitionInfoParameter(partInfo, params);
 
             int i = 0;
 
@@ -2298,6 +2300,28 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         return result;
     }
 
+    /**
+     * Bind query parameter to partition info and calculate partition.
+     *
+     * @param partInfo Partition Info.
+     * @param params Query parameters.
+     * @return Partition.
+     * @throws IgniteCheckedException, If fails.
+     */
+    private int bindPartitionInfoParameter(CacheQueryPartitionInfo partInfo, Object[] params)
+        throws IgniteCheckedException {
+        assert partInfo != null;
+        assert partInfo.partition() < 0;
+
+        GridH2RowDescriptor desc = dataTable(partInfo.cacheName(),
+                partInfo.tableName()).rowDescriptor();
+
+        Object param = H2Utils.convert(params[partInfo.paramIdx()],
+                desc, partInfo.dataType());
+
+        return kernalContext().affinity().partition(partInfo.cacheName(), param);
+    }
+
     /** {@inheritDoc} */
     @Override public Collection<GridRunningQueryInfo> runningQueries(long duration) {
         Collection<GridRunningQueryInfo> res = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/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 aebf596..1578f9f 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
@@ -2318,14 +2318,15 @@ public class GridSqlQuerySplitter {
             GridSqlConst constant = (GridSqlConst)right;
 
             return new CacheQueryPartitionInfo(ctx.affinity().partition(tbl.cacheName(),
-                constant.value().getObject()), null, -1);
+                constant.value().getObject()), null, null, -1, -1);
         }
 
         assert right instanceof GridSqlParameter;
 
         GridSqlParameter param = (GridSqlParameter) right;
 
-        return new CacheQueryPartitionInfo(-1, tbl.cacheName(), param.index());
+        return new CacheQueryPartitionInfo(-1, tbl.cacheName(), tbl.getName(),
+            column.column().getType(), param.index());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlRoutingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlRoutingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlRoutingTest.java
index fddd3f4..323eb7a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlRoutingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlRoutingTest.java
@@ -33,19 +33,26 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.text.DateFormat;
 import java.text.MessageFormat;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Date;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 
+/** Tests for query partitions derivation. */
 public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
     /** IP finder. */
     private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
@@ -62,10 +69,10 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
     /** */
     private static int NODE_COUNT = 4;
 
-    /** broadcast query to ensure events came from all nodes */
+    /** Broadcast query to ensure events came from all nodes. */
     private static String FINAL_QRY = "select count(1) from {0} where name=?";
 
-    /** Param to distinguish the final query event */
+    /** Param to distinguish the final query event. */
     private static String FINAL_QRY_PARAM = "Abracadabra";
 
     /** {@inheritDoc} */
@@ -108,15 +115,19 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
 
         startGrid(NODE_CLIENT);
 
+        awaitPartitionMapExchange();
+
         fillCaches();
     }
 
+    /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         super.afterTestsStopped();
 
         stopAllGrids();
     }
 
+    /** */
     private CacheConfiguration buildCacheConfiguration(String name) {
         if (name.equals(CACHE_PERSON)) {
             CacheConfiguration ccfg = new CacheConfiguration(CACHE_PERSON);
@@ -176,7 +187,7 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
 
         List<List<?>> result = runQueryEnsureUnicast(cache,
-                new SqlFieldsQuery("select id, name, duration from Call where personId=100 order by id"), 1);
+            new SqlFieldsQuery("select id, name, duration from Call where personId=100 order by id"), 1);
 
         assertEquals(2, result.size());
 
@@ -189,7 +200,8 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
 
         List<List<?>> result = runQueryEnsureUnicast(cache,
-                new SqlFieldsQuery("select id, name, duration from Call where personId=? order by id").setArgs(100), 1);
+            new SqlFieldsQuery("select id, name, duration from Call where personId=? order by id")
+            .setArgs(100), 1);
 
         assertEquals(2, result.size());
 
@@ -203,7 +215,7 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
 
         for (int key : new int[] {0, 250, 500, 750, 1000} ) {
             List<List<?>> result = runQueryEnsureUnicast(cache,
-                    new SqlFieldsQuery("select name, age from Person where _key=?").setArgs(key), 1);
+                new SqlFieldsQuery("select name, age from Person where _key=?").setArgs(key), 1);
 
             assertEquals(1, result.size());
 
@@ -220,8 +232,8 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         CallKey callKey = new CallKey(5, 1);
 
         List<List<?>> result = runQueryEnsureUnicast(cache,
-                new SqlFieldsQuery("select name, duration from Call where _key=?")
-                .setArgs(callKey), 1);
+            new SqlFieldsQuery("select name, duration from Call where _key=?")
+            .setArgs(callKey), 1);
 
         assertEquals(1, result.size());
 
@@ -230,7 +242,7 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         checkResultsRow(result, 0, call.name, call.duration);
     }
 
-    /** Check group, having, ordering allowed to be unicast requests */
+    /** Check group, having, ordering allowed to be unicast requests. */
     public void testUnicastQueryGroups() throws Exception {
         IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
 
@@ -258,8 +270,8 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         CallKey callKey = new CallKey(5, 1);
 
         List<List<?>> result = runQueryEnsureUnicast(cache,
-                new SqlFieldsQuery("select name, duration from Call where _key=? and duration=?")
-                        .setArgs(callKey, 100), 1);
+            new SqlFieldsQuery("select name, duration from Call where _key=? and duration=?")
+            .setArgs(callKey, 100), 1);
 
         assertEquals(1, result.size());
 
@@ -276,8 +288,8 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         CallKey callKey2 = new CallKey(1000, 1);
 
         List<List<?>> result = runQueryEnsureUnicast(cache,
-                new SqlFieldsQuery("select name, duration from Call where (_key=? and duration=?) or (_key=?)")
-                        .setArgs(callKey1, 100, callKey2), 2);
+            new SqlFieldsQuery("select name, duration from Call where (_key=? and duration=?) or (_key=?)")
+            .setArgs(callKey1, 100, callKey2), 2);
 
         assertEquals(2, result.size());
 
@@ -291,19 +303,181 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
     }
 
     /** */
+    public void testUnicastQueryKeyTypeConversionParameter() throws Exception {
+        IgniteCache<Integer, Person> cache = grid(NODE_CLIENT).cache(CACHE_PERSON);
+
+        // Pass string argument to expression with integer
+        List<List<?>> result = runQueryEnsureUnicast(cache,
+            new SqlFieldsQuery("select name, age from Person where _key = ?")
+            .setArgs("5"), 1);
+
+        Person person = cache.get(5);
+
+        assertEquals(1, result.size());
+
+        assertEquals(person.name, result.get(0).get(0));
+        assertEquals(person.age, result.get(0).get(1));
+    }
+
+    /** */
+    public void testUnicastQueryKeyTypeConversionConstant() throws Exception {
+        IgniteCache<Integer, Person> cache = grid(NODE_CLIENT).cache(CACHE_PERSON);
+
+        // Use string within expression against integer key
+        List<List<?>> result = runQueryEnsureUnicast(cache,
+            new SqlFieldsQuery("select name, age from Person where _key = '5'"), 1);
+
+        Person person = cache.get(5);
+
+        assertEquals(1, result.size());
+
+        assertEquals(person.name, result.get(0).get(0));
+        assertEquals(person.age, result.get(0).get(1));
+    }
+
+    /** */
+    public void testUnicastQueryAffinityKeyTypeConversionParameter() throws Exception {
+        IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
+
+        // Pass string argument to expression with integer
+        List<List<?>> result = runQueryEnsureUnicast(cache,
+            new SqlFieldsQuery("select id, name, duration from Call where personId=? order by id")
+                .setArgs("100"), 1);
+
+        assertEquals(2, result.size());
+
+        checkResultsRow(result, 0, 1, "caller1", 100);
+        checkResultsRow(result, 1, 2, "caller2", 200);
+    }
+
+    /** */
+    public void testUnicastQueryAffinityKeyTypeConversionConstant() throws Exception {
+        IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
+
+        // Use string within expression against integer key
+        List<List<?>> result = runQueryEnsureUnicast(cache,
+            new SqlFieldsQuery("select id, name, duration from Call where personId='100' order by id"), 1);
+
+        assertEquals(2, result.size());
+
+        checkResultsRow(result, 0, 1, "caller1", 100);
+        checkResultsRow(result, 1, 2, "caller2", 200);
+    }
+
+    /** */
     public void testBroadcastQuerySelectKeyEqualsOrFieldParameter() throws Exception {
         IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
 
         CallKey callKey = new CallKey(5, 1);
 
         List<List<?>> result = runQueryEnsureBroadcast(cache,
-                new SqlFieldsQuery("select name, duration from Call where _key=? or duration=?")
-                        .setArgs(callKey, 100));
+            new SqlFieldsQuery("select name, duration from Call where _key=? or duration=?")
+            .setArgs(callKey, 100));
 
         assertEquals(cache.size() / 2, result.size());
     }
 
     /** */
+    public void testUuidKeyAsByteArrayParameter() throws Exception {
+        String cacheName = "uuidCache";
+
+        CacheConfiguration<UUID, UUID> ccfg = new CacheConfiguration<>(cacheName);
+
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+
+        ccfg.setIndexedTypes(UUID.class, UUID.class);
+
+        IgniteCache<UUID, UUID> cache = grid(NODE_CLIENT).createCache(ccfg);
+
+        try {
+            int count = 10;
+
+            UUID values[] = new UUID[count];
+
+            for (int i = 0; i < count; i++) {
+                UUID val = UUID.randomUUID();
+
+                cache.put(val, val);
+
+                values[i] = val;
+            }
+
+            for (UUID val : values) {
+                byte[] arr = convertUuidToByteArray(val);
+
+                List<List<?>> result = cache.query(new SqlFieldsQuery(
+                    "select _val from UUID where _key = ?").setArgs(arr)).getAll();
+
+                assertEquals(1, result.size());
+                assertEquals(val, result.get(0).get(0));
+            }
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /** */
+    public void testDateKeyAsTimestampParameter() throws Exception {
+        String cacheName = "dateCache";
+
+        CacheConfiguration<Date, Date> ccfg = new CacheConfiguration<>(cacheName);
+
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+
+        ccfg.setIndexedTypes(Date.class, Date.class);
+
+        IgniteCache<Date, Date> cache = grid(NODE_CLIENT).createCache(ccfg);
+
+        try {
+            int count = 30;
+
+            Date values[] = new Date[count];
+
+            DateFormat dateFormat = new SimpleDateFormat("dd/MM/yyyy");
+
+            for (int i = 0; i < count; i++) {
+                Date val = dateFormat.parse(String.format("%02d/06/2017", i + 1));
+
+                cache.put(val, val);
+
+                values[i] = val;
+            }
+
+            for (Date val : values) {
+                Timestamp ts = new Timestamp(val.getTime());
+
+                List<List<?>> result = cache.query(new SqlFieldsQuery(
+                    "select _val from Date where _key = ?").setArgs(ts)).getAll();
+
+                assertEquals(1, result.size());
+                assertEquals(val, result.get(0).get(0));
+            }
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * Convert UUID to byte[].
+     *
+     * @param val UUID to convert.
+     * @return Result.
+     */
+    private byte[] convertUuidToByteArray(UUID val) {
+        assert val != null;
+
+        ByteBuffer bb = ByteBuffer.wrap(new byte[16]);
+
+        bb.putLong(val.getMostSignificantBits());
+
+        bb.putLong(val.getLeastSignificantBits());
+
+        return bb.array();
+    }
+
+    /** */
     private void fillCaches() {
         IgniteCache<CallKey, Call> callCache = grid(NODE_CLIENT).cache(CACHE_CALL);
         IgniteCache<Integer, Person> personCache = grid(NODE_CLIENT).cache(CACHE_PERSON);
@@ -335,15 +509,15 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
             assertEquals(expected[col], row.get(col));
     }
 
-    /** Run query and check that only one node did generate 'query executed' event for it */
+    /** Run query and check that only one node did generate 'query executed' event for it. */
     private List<List<?>> runQueryEnsureUnicast(IgniteCache<?,?> cache, SqlFieldsQuery qry, int nodeCnt) throws Exception {
         try (EventCounter evtCounter = new EventCounter(nodeCnt)) {
             List<List<?>> result = cache.query(qry).getAll();
 
             // do broadcast 'marker' query to ensure that we received all events from previous qry
             cache.query(new SqlFieldsQuery(
-                    MessageFormat.format(FINAL_QRY, cache.getName()))
-                    .setArgs(FINAL_QRY_PARAM)).getAll();
+                MessageFormat.format(FINAL_QRY, cache.getName()))
+                .setArgs(FINAL_QRY_PARAM)).getAll();
 
             // wait for all events from 'marker' query
             evtCounter.await();
@@ -353,6 +527,7 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         }
     }
 
+    /** */
     private List<List<?>> runQueryEnsureBroadcast(IgniteCache<?, ?> cache, SqlFieldsQuery qry) throws Exception {
         final CountDownLatch execLatch = new CountDownLatch(NODE_COUNT);
 
@@ -492,7 +667,7 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         private int id;
 
         /** */
-        public CallKey(int personId, int id) {
+        private CallKey(int personId, int id) {
             this.personId = personId;
             this.id = id;
         }