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

[03/46] ignite git commit: IGNITE-2208 - Queries with object arguments doesn't work wth BinaryMarshaller.

IGNITE-2208 - Queries with object arguments doesn't work wth BinaryMarshaller.


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

Branch: refs/heads/master
Commit: ae7765329fd6f7d50d13183d13626f39c5682334
Parents: 12f5329
Author: dkarachentsev <dk...@gridgain.com>
Authored: Fri Sep 2 18:01:12 2016 +0300
Committer: dkarachentsev <dk...@gridgain.com>
Committed: Fri Sep 2 18:01:12 2016 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryMarshaller.java       |   7 +
 .../processors/cache/IgniteCacheProxy.java      |  41 ++
 .../cache/query/GridCacheSqlQuery.java          |  11 +-
 .../GridCacheQueryTransformerSelfTest.java      |   9 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    |   9 -
 ...niteBinaryObjectLocalQueryArgumentsTest.java |  28 ++
 ...aryObjectQueryArgumentsOffheapLocalTest.java |  28 ++
 ...teBinaryObjectQueryArgumentsOffheapTest.java |  30 ++
 .../IgniteBinaryObjectQueryArgumentsTest.java   | 469 ++++++++++++++++++-
 .../query/h2/sql/GridQueryParsingTest.java      |   4 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   9 +
 11 files changed, 610 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java
index 29a1fca..39015e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java
@@ -136,4 +136,11 @@ public class BinaryMarshaller extends AbstractMarshaller {
     @Override public void onUndeploy(ClassLoader ldr) {
         impl.context().onUndeploy(ldr);
     }
+
+    /**
+     * @return GridBinaryMarshaller instance.
+     */
+    public GridBinaryMarshaller binaryMarshaller() {
+        return impl;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 9b26c1d..8b2e605 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -63,6 +63,7 @@ import org.apache.ignite.internal.AsyncSupportAdapter;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.binary.BinaryUtils;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.query.CacheQuery;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
@@ -688,6 +689,8 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
 
             validate(qry);
 
+            convertToBinary(qry);
+
             final CacheOperationContext opCtxCall = ctx.operationContextPerCall();
 
             if (qry instanceof ContinuousQuery)
@@ -764,6 +767,44 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /**
+     * Convert query arguments to BinaryObjects if binary marshaller used.
+     *
+     * @param qry Query.
+     */
+    private void convertToBinary(final Query qry) {
+        if (ctx.binaryMarshaller()) {
+            if (qry instanceof SqlQuery) {
+                final SqlQuery sqlQry = (SqlQuery) qry;
+
+                convertToBinary(sqlQry.getArgs());
+            } else if (qry instanceof SpiQuery) {
+                final SpiQuery spiQry = (SpiQuery) qry;
+
+                convertToBinary(spiQry.getArgs());
+            } else if (qry instanceof SqlFieldsQuery) {
+                final SqlFieldsQuery fieldsQry = (SqlFieldsQuery) qry;
+
+                convertToBinary(fieldsQry.getArgs());
+            }
+        }
+    }
+
+    /**
+     * Converts query arguments to BinaryObjects if binary marshaller used.
+     *
+     * @param args Arguments.
+     */
+    private void convertToBinary(final Object[] args) {
+        if (args == null)
+            return;
+
+        for (int i = 0; i < args.length; i++) {
+            if (args[i] != null && !BinaryUtils.isBinaryType(args[i].getClass()))
+                args[i] = ctx.toCacheObject(args[i]);
+        }
+    }
+
+    /**
      * @return {@code true} If this is a replicated cache and we are on a data node.
      */
     private boolean isReplicatedDataNode() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
index 0733827..bcb37c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
@@ -22,6 +22,7 @@ import java.util.LinkedHashMap;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
@@ -153,7 +154,13 @@ public class GridCacheSqlQuery implements Message {
 
         assert paramsBytes != null;
 
-        params = m.unmarshal(paramsBytes, U.resolveClassLoader(ctx.config()));
+        final ClassLoader ldr = U.resolveClassLoader(ctx.config());
+
+        if (m instanceof BinaryMarshaller)
+            // To avoid deserializing of enum types.
+            params = ((BinaryMarshaller)m).binaryMarshaller().unmarshal(paramsBytes, ldr);
+        else
+            params = m.unmarshal(paramsBytes, ldr);
     }
 
     /** {@inheritDoc} */
@@ -271,4 +278,4 @@ public class GridCacheSqlQuery implements Message {
 
         return cp;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java
index 6b13e05..e7e173b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java
@@ -66,9 +66,14 @@ public class GridCacheQueryTransformerSelfTest extends GridCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         startGridsMultiThreaded(3);
 
-        Ignition.setClientMode(true);
+        try {
+            Ignition.setClientMode(true);
 
-        startGrid();
+            startGrid();
+        }
+        finally {
+            Ignition.setClientMode(false);
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
index 29f9675..fd0e6ed 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
@@ -21,8 +21,6 @@ import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.sql.Types;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.internal.binary.BinaryEnumObjectImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -146,13 +144,6 @@ public class GridH2ValueCacheObject extends Value {
             return c1.compareTo(o2);
         }
 
-        if (o1 instanceof BinaryEnumObjectImpl && o2 instanceof Enum) {
-            final BinaryEnumObjectImpl bo1 = (BinaryEnumObjectImpl)o1;
-
-            if (bo1.isTypeEquals(o2.getClass()))
-                return Integer.compare(bo1.enumOrdinal(), ((Enum)o2).ordinal());
-        }
-
         // Group by types.
         if (o1.getClass() != o2.getClass()) {
             if (o1Comparable != o2Comparable)

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectLocalQueryArgumentsTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectLocalQueryArgumentsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectLocalQueryArgumentsTest.java
new file mode 100644
index 0000000..7e35e51
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectLocalQueryArgumentsTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ *
+ */
+public class IgniteBinaryObjectLocalQueryArgumentsTest extends IgniteBinaryObjectQueryArgumentsTest {
+    /** {@inheritDoc} */
+    @Override protected boolean isLocal() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapLocalTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapLocalTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapLocalTest.java
new file mode 100644
index 0000000..560d258
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapLocalTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ *
+ */
+public class IgniteBinaryObjectQueryArgumentsOffheapLocalTest extends IgniteBinaryObjectQueryArgumentsOffheapTest {
+    /** {@inheritDoc} */
+    @Override protected boolean isLocal() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapTest.java
new file mode 100644
index 0000000..d1428ae
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapTest.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.cache;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+/**
+ *
+ */
+public class IgniteBinaryObjectQueryArgumentsOffheapTest extends IgniteBinaryObjectQueryArgumentsTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return CacheMemoryMode.OFFHEAP_TIERED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java
index 5676ddd..8a0c5c8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java
@@ -17,15 +17,27 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.Arrays;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
 import javax.cache.Cache;
-import org.apache.ignite.IgniteBinary;
+
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
+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.util.typedef.internal.S;
 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;
@@ -43,12 +55,64 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest
     /** */
     private static final int NODES = 3;
 
+    /** */
+    public static final String PRIM_CACHE = "prim-cache";
+
+    /** */
+    public static final String STR_CACHE = "str-cache";
+
+    /** */
+    public static final String ENUM_CACHE = "enum-cache";
+
+    /** */
+    public static final String UUID_CACHE = "uuid-cache";
+
+    /** */
+    public static final String DATE_CACHE = "date-cache";
+
+    /** */
+    public static final String TIMESTAMP_CACHE = "timestamp-cache";
+
+    /** */
+    public static final String BIG_DECIMAL_CACHE = "decimal-cache";
+
+    /** */
+    public static final String OBJECT_CACHE = "obj-cache";
+
+    /** */
+    public static final String FIELD_CACHE = "field-cache";
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
 
+        cfg.setCacheConfiguration(getCacheConfigurations());
+
+        cfg.setMarshaller(null);
+
+        return cfg;
+    }
+
+    /**
+     * @return {@code True} If query is local.
+     */
+    protected boolean isLocal() {
+        return false;
+    }
+
+    /**
+     * @return Memory mode.
+     */
+    protected CacheMemoryMode memoryMode() {
+        return CacheMemoryMode.ONHEAP_TIERED;
+    }
+
+    /**
+     * @return Cache config.
+     */
+    protected CacheConfiguration getCacheConfiguration(final String cacheName) {
         CacheConfiguration ccfg = new CacheConfiguration();
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
 
@@ -57,11 +121,64 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest
         person.setValueType(Person.class.getName());
         person.addQueryField("name", String.class.getName(), null);
 
-        ccfg.setQueryEntities(Arrays.asList(person));
+        ccfg.setQueryEntities(Collections.singletonList(person));
 
-        cfg.setCacheConfiguration(ccfg);
+        ccfg.setMemoryMode(memoryMode());
 
-        cfg.setMarshaller(null);
+        ccfg.setName(cacheName);
+
+        return ccfg;
+    }
+
+    /**
+     * @return Cache configurations.
+     */
+    private CacheConfiguration[] getCacheConfigurations() {
+        final ArrayList<CacheConfiguration> ccfgs = new ArrayList<>();
+
+        ccfgs.add(getCacheConfiguration(OBJECT_CACHE));
+        ccfgs.addAll(getCacheConfigurations(STR_CACHE, String.class, Person.class));
+        ccfgs.addAll(getCacheConfigurations(PRIM_CACHE, Integer.class, Person.class));
+        ccfgs.addAll(getCacheConfigurations(ENUM_CACHE, EnumKey.class, Person.class));
+        ccfgs.addAll(getCacheConfigurations(UUID_CACHE, UUID.class, Person.class));
+        ccfgs.addAll(getCacheConfigurations(DATE_CACHE, Date.class, Person.class));
+        ccfgs.addAll(getCacheConfigurations(TIMESTAMP_CACHE, Timestamp.class, Person.class));
+        ccfgs.addAll(getCacheConfigurations(BIG_DECIMAL_CACHE, BigDecimal.class, Person.class));
+        ccfgs.add(getCacheConfiguration(FIELD_CACHE, Integer.class, SearchValue.class));
+
+        return ccfgs.toArray(new CacheConfiguration[ccfgs.size()]);
+    }
+
+    /**
+     *
+     * @param cacheName Cache name.
+     * @param key Key type.
+     * @param val Value type.
+     * @return Configurations.
+     */
+    private List<CacheConfiguration> getCacheConfigurations(final String cacheName, final Class<?> key, final Class<?> val) {
+        final List<CacheConfiguration> res = new ArrayList<>();
+
+        res.add(getCacheConfiguration(cacheName, key, val));
+        res.add(getCacheConfiguration(cacheName + "-val", val, key));
+
+        return res;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @param key Key type.
+     * @param val Value type
+     * @return Configuration.
+     */
+    @SuppressWarnings("unchecked")
+    private CacheConfiguration getCacheConfiguration(final String cacheName, final Class<?> key, final Class<?> val) {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setName(cacheName);
+
+        cfg.setMemoryMode(memoryMode());
+        cfg.setIndexedTypes(key, val);
 
         return cfg;
     }
@@ -70,7 +187,9 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        startGridsMultiThreaded(NODES);
+        final int nodes = isLocal() ? 1 : NODES;
+
+        startGridsMultiThreaded(nodes);
     }
 
     /** {@inheritDoc} */
@@ -84,31 +203,210 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testObjectArgument() throws Exception {
-        IgniteCache<TestKey, Person> cache = ignite(0).cache(null);
+        testKeyQuery(OBJECT_CACHE, new TestKey(1), new TestKey(2));
+    }
 
-        for (int i = 0; i < 100; i++)
-            cache.put(new TestKey(i), new Person("name-" + i));
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimitiveObjectArgument() throws Exception {
+        testKeyValQuery(PRIM_CACHE, 1, 2);
+    }
 
-        SqlQuery<TestKey, Person> qry = new SqlQuery<>(Person.class, "where _key=?");
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStringObjectArgument() throws Exception {
+        testKeyValQuery(STR_CACHE, "str1", "str2");
+    }
 
-        IgniteBinary binary = ignite(0).binary();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEnumObjectArgument() throws Exception {
+       testKeyValQuery(ENUM_CACHE, EnumKey.KEY1, EnumKey.KEY2);
+    }
 
-        for (int i = 0; i < 100; i++) {
-            Object key = new TestKey(i);
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUuidObjectArgument() throws Exception {
+        final UUID uuid1 = UUID.randomUUID();
+        UUID uuid2 = UUID.randomUUID();
 
-            if (i % 2 == 0)
-                key = binary.toBinary(key);
+        while (uuid1.equals(uuid2))
+            uuid2 = UUID.randomUUID();
 
-            qry.setArgs(key);
+        testKeyValQuery(UUID_CACHE, uuid1, uuid2);
+    }
 
-            List<Cache.Entry<TestKey, Person>> res = cache.query(qry).getAll();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDateObjectArgument() throws Exception {
+        testKeyValQuery(DATE_CACHE, new Date(0), new Date(1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTimestampArgument() throws Exception {
+        testKeyValQuery(TIMESTAMP_CACHE, new Timestamp(0), new Timestamp(1));
+    }
 
-            assertEquals(1, res.size());
 
-            Person p = res.get(0).getValue();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBigDecimalArgument() throws Exception {
+        final ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
-            assertEquals("name-" + i, p.name);
+        final BigDecimal bd1 = new BigDecimal(rnd.nextDouble());
+        BigDecimal bd2 = new BigDecimal(rnd.nextDouble());
+
+        while (bd1.equals(bd2))
+            bd2 = new BigDecimal(rnd.nextDouble());
+
+        testKeyValQuery(BIG_DECIMAL_CACHE, bd1, bd2);
+    }
+
+    /**
+     * Test simple queries.
+     *
+     * @param cacheName Cache name.
+     * @param key1 Key 1.
+     * @param key2 Key 2.
+     * @param <T> Key type.
+     */
+    private <T> void testKeyValQuery(final String cacheName, final T key1, final T key2) {
+        testKeyQuery(cacheName, key1, key2);
+        testValQuery(cacheName + "-val", key1, key2);
+    }
+
+    /**
+     * Test simple query by key.
+     *
+     * @param cacheName Cache name.
+     * @param key1 Key 1.
+     * @param key2 Key 2.
+     * @param <T> Key type.
+     */
+    private <T> void testKeyQuery(final String cacheName, final T key1, final T key2) {
+        final IgniteCache<T, Person> cache = ignite(0).cache(cacheName);
+
+        final Person p1 = new Person("p1");
+        final Person p2 = new Person("p2");
+
+        cache.put(key1, p1);
+        cache.put(key2, p2);
+
+        final SqlQuery<T, Person> qry = new SqlQuery<>(Person.class, "where _key=?");
+
+        final SqlFieldsQuery fieldsQry = new SqlFieldsQuery("select * from Person where _key=?");
+
+        qry.setLocal(isLocal());
+        fieldsQry.setLocal(isLocal());
+
+        qry.setArgs(key1);
+        fieldsQry.setArgs(key1);
+
+        final List<Cache.Entry<T, Person>> res = cache.query(qry).getAll();
+        final List<List<?>> fieldsRes = cache.query(fieldsQry).getAll();
+
+        assertEquals(1, res.size());
+        assertEquals(1, fieldsRes.size());
+
+        assertEquals(p1, res.get(0).getValue());
+        assertEquals(key1, res.get(0).getKey());
+
+        assertTrue(fieldsRes.get(0).size() >= 2);
+        assertEquals(key1, fieldsRes.get(0).get(0));
+        assertEquals(p1, fieldsRes.get(0).get(1));
+    }
+
+    /**
+     * Test simple query by value.
+     *
+     * @param cacheName Cache name.
+     * @param val1 Value 1.
+     * @param val2 Value 2.
+     * @param <T> Value type.
+     */
+    private <T> void testValQuery(final String cacheName, final T val1, final T val2) {
+        final IgniteCache<Person, T> cache = ignite(0).cache(cacheName);
+
+        final Class<?> valType = val1.getClass();
+
+        final Person p1 = new Person("p1");
+        final Person p2 = new Person("p2");
+
+        cache.put(p1, val1);
+        cache.put(p2, val2);
+
+        final SqlQuery<Person, T> qry = new SqlQuery<>(valType, "where _val=?");
+
+        final SqlFieldsQuery fieldsQry = new SqlFieldsQuery("select * from " + valType.getSimpleName() + " where _val=?");
+
+        qry.setLocal(isLocal());
+        fieldsQry.setLocal(isLocal());
+
+        qry.setArgs(val1);
+        fieldsQry.setArgs(val1);
+
+        final List<Cache.Entry<Person, T>> res = cache.query(qry).getAll();
+        final List<List<?>> fieldsRes = cache.query(fieldsQry).getAll();
+
+        assertEquals(1, res.size());
+        assertEquals(1, fieldsRes.size());
+
+        assertEquals(p1, res.get(0).getKey());
+        assertEquals(val1, res.get(0).getValue());
+
+        assertTrue(fieldsRes.get(0).size() >= 2);
+        assertEquals(p1, fieldsRes.get(0).get(0));
+        assertEquals(val1, fieldsRes.get(0).get(1));
+    }
+
+    /**
+     * @throws Exception
+     */
+    public void testFieldSearch() throws Exception {
+        final IgniteCache<Integer, SearchValue> cache = ignite(0).cache(FIELD_CACHE);
+
+        final Map<Integer, SearchValue> map = new HashMap<>();
+
+        for (int i = 0; i < 10; i++) {
+            map.put(i,
+                new SearchValue(
+                    UUID.randomUUID(),
+                    String.valueOf(i),
+                    new BigDecimal(i * 0.1),
+                    i,
+                    new Date(i),
+                    new Timestamp(i),
+                    new Person(String.valueOf("name-" + i)),
+                    i % 2 == 0 ? EnumKey.KEY1 : EnumKey.KEY2)
+            );
         }
+
+        cache.putAll(map);
+
+        SqlQuery<Integer, SearchValue> qry = new SqlQuery<>(SearchValue.class,
+            "where uuid=? and str=? and decimal=? and integer=? and date=? and ts=? and person=? and enumKey=?");
+
+        final int k = ThreadLocalRandom.current().nextInt(10);
+
+        final SearchValue val = map.get(k);
+
+        qry.setLocal(isLocal());
+        qry.setArgs(val.uuid, val.str, val.decimal, val.integer, val.date, val.ts, val.person, val.enumKey);
+
+        final List<Cache.Entry<Integer, SearchValue>> res = cache.query(qry).getAll();
+
+        assertEquals(1, res.size());
+
+        assertEquals(val.integer, res.get(0).getKey());
+        assertEquals(val, res.get(0).getValue());
     }
 
     /**
@@ -124,6 +422,27 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest
         public Person(String name) {
             this.name = name;
         }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(final Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            final Person person = (Person) o;
+
+            return name != null ? name.equals(person.name) : person.name == null;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return name != null ? name.hashCode() : 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(Person.class, this);
+        }
     }
 
     /**
@@ -158,4 +477,114 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest
             return id;
         }
     }
+
+    /**
+     *
+     */
+    private enum EnumKey {
+        /** */
+        KEY1,
+
+        /** */
+        KEY2
+    }
+
+    /**
+     *
+     */
+    private static class SearchValue {
+        /** */
+        @QuerySqlField
+        private UUID uuid;
+
+        /** */
+        @QuerySqlField
+        private String str;
+
+        /** */
+        @QuerySqlField
+        private BigDecimal decimal;
+
+        /** */
+        @QuerySqlField
+        private Integer integer;
+
+        /** */
+        @QuerySqlField
+        private Date date;
+
+        /** */
+        @QuerySqlField
+        private Timestamp ts;
+
+        /** */
+        @QuerySqlField
+        private Person person;
+
+        /** */
+        @QuerySqlField
+        private EnumKey enumKey;
+
+        /**
+         *
+         * @param uuid UUID.
+         * @param str String.
+         * @param decimal Decimal.
+         * @param integer Integer.
+         * @param date Date.
+         * @param ts Timestamp.
+         * @param person Person.
+         * @param enumKey Enum.
+         */
+        public SearchValue(
+            final UUID uuid,
+            final String str,
+            final BigDecimal decimal,
+            final Integer integer,
+            final Date date,
+            final Timestamp ts,
+            final Person person,
+            final EnumKey enumKey
+        ) {
+            this.uuid = uuid;
+            this.str = str;
+            this.decimal = decimal;
+            this.integer = integer;
+            this.date = date;
+            this.ts = ts;
+            this.person = person;
+            this.enumKey = enumKey;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(final Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            final SearchValue that = (SearchValue) o;
+
+            if (uuid != null ? !uuid.equals(that.uuid) : that.uuid != null) return false;
+            if (str != null ? !str.equals(that.str) : that.str != null) return false;
+            if (decimal != null ? !decimal.equals(that.decimal) : that.decimal != null) return false;
+            if (integer != null ? !integer.equals(that.integer) : that.integer != null) return false;
+            if (date != null ? !date.equals(that.date) : that.date != null) return false;
+            if (ts != null ? !ts.equals(that.ts) : that.ts != null) return false;
+            if (person != null ? !person.equals(that.person) : that.person != null) return false;
+            return enumKey == that.enumKey;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = uuid != null ? uuid.hashCode() : 0;
+            res = 31 * res + (str != null ? str.hashCode() : 0);
+            res = 31 * res + (decimal != null ? decimal.hashCode() : 0);
+            res = 31 * res + (integer != null ? integer.hashCode() : 0);
+            res = 31 * res + (date != null ? date.hashCode() : 0);
+            res = 31 * res + (ts != null ? ts.hashCode() : 0);
+            res = 31 * res + (person != null ? person.hashCode() : 0);
+            res = 31 * res + (enumKey != null ? enumKey.hashCode() : 0);
+            return res;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/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 1d54bbf..cf000e9 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
@@ -301,7 +301,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     private static String normalizeSql(String sql) {
         return sql.toLowerCase()
             .replaceAll("/\\*(?:.|\r|\n)*?\\*/", " ")
-            .replaceAll("\\s*on\\s+1\\s*=\\s*1\\s*", " on true ")
+            .replaceAll("\\s*on\\s+1\\s*=\\s*1\\s*", " ")
             .replaceAll("\\s+", " ")
             .replaceAll("\\( +", "(")
             .replaceAll(" +\\)", ")")
@@ -366,4 +366,4 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
         @QuerySqlField(index = true)
         public String street = "Nevskiy";
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae776532/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 3652acd..e7f55a1 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
@@ -24,6 +24,10 @@ import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSel
 import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectLocalQueryArgumentsTest;
+import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsOffheapLocalTest;
+import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsOffheapTest;
+import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsTest;
 import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
@@ -118,6 +122,11 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheQueryNoRebalanceSelfTest.class);
         suite.addTestSuite(GridCacheQueryTransformerSelfTest.class);
 
+        suite.addTestSuite(IgniteBinaryObjectQueryArgumentsTest.class);
+        suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapTest.class);
+        suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapLocalTest.class);
+        suite.addTestSuite(IgniteBinaryObjectLocalQueryArgumentsTest.class);
+
         return suite;
     }
 }