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

[3/3] ignite git commit: IGNITE-2546 - Adding transformers to SCAN queries

IGNITE-2546 - Adding transformers to SCAN queries


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

Branch: refs/heads/ignite-2546
Commit: 7d7e1489dbdf8038b8b76a81664e340c82a8d02e
Parents: 00f47d7
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Mon Aug 8 20:15:49 2016 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Mon Aug 8 20:15:49 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |  15 +
 .../processors/cache/IgniteCacheProxy.java      |   8 +
 .../GridCacheQueryTransformerSelfTest.java      | 570 +++++++++++++++++++
 .../multijvm/IgniteCacheProcessProxy.java       |   6 +
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 5 files changed, 601 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7d7e1489/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 40eedaf..2290fc5 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -56,6 +56,7 @@ import org.apache.ignite.lang.IgniteAsyncSupport;
 import org.apache.ignite.lang.IgniteAsyncSupported;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.mxbean.CacheMetricsMXBean;
 import org.apache.ignite.transactions.TransactionHeuristicException;
@@ -295,6 +296,20 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
     public <R> QueryCursor<R> query(Query<R> qry);
 
     /**
+     * Queries the cache transforming the entries on the server nodes. Can be used, for example,
+     * to avoid network overhead in case only one field out of the large is required by client.
+     * <p>
+     * Currently transformers are supported ONLY for {@link ScanQuery}. Passing any other
+     * subclass of {@link Query} interface to this method will end up with
+     * {@link UnsupportedOperationException}.
+     *
+     * @param qry Query.
+     * @param transformer Transformer.
+     * @return Cursor.
+     */
+    public <T, R> QueryCursor<R> query(Query<T> qry, IgniteClosure<T, R> transformer);
+
+    /**
      * Allows for iteration over local cache entries.
      *
      * @param peekModes Peek modes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d7e1489/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 0d7bc6a..af434f9 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
@@ -82,6 +82,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.mxbean.CacheMetricsMXBean;
 import org.apache.ignite.plugin.security.SecurityPermission;
@@ -705,6 +706,13 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public <T, R> QueryCursor<R> query(Query<T> qry, IgniteClosure<T, R> transformer) {
+        // TODO: IGNITE-2546
+
+        return null;
+    }
+
     /**
      * @return {@code true} If this is a replicated cache and we are on a data node.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d7e1489/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
new file mode 100644
index 0000000..6b13e05
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java
@@ -0,0 +1,570 @@
+/*
+ * 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.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.query.ContinuousQuery;
+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.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.resources.IgniteInstanceResource;
+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;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test for scan query with transformer.
+ */
+public class GridCacheQueryTransformerSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
+        cfg.setMarshaller(null);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3);
+
+        Ignition.setClientMode(true);
+
+        startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetKeys() throws Exception {
+        IgniteCache<Integer, String> cache = grid().createCache("test-cache");
+
+        try {
+            for (int i = 0; i < 50; i++)
+                cache.put(i, "val" + i);
+
+            IgniteClosure<Cache.Entry<Integer, String>, Integer> transformer =
+                new IgniteClosure<Cache.Entry<Integer, String>, Integer>() {
+                    @Override public Integer apply(Cache.Entry<Integer, String> e) {
+                        return e.getKey();
+                    }
+                };
+
+            List<Integer> keys = cache.query(new ScanQuery<Integer, String>(), transformer).getAll();
+
+            assertEquals(50, keys.size());
+
+            Collections.sort(keys);
+
+            for (int i = 0; i < 50; i++)
+                assertEquals(i, keys.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetKeysFiltered() throws Exception {
+        IgniteCache<Integer, String> cache = grid().createCache("test-cache");
+
+        try {
+            for (int i = 0; i < 50; i++)
+                cache.put(i, "val" + i);
+
+            IgniteBiPredicate<Integer, String> filter = new IgniteBiPredicate<Integer, String>() {
+                @Override public boolean apply(Integer k, String v) {
+                    return k % 10 == 0;
+                }
+            };
+
+            IgniteClosure<Cache.Entry<Integer, String>, Integer> transformer =
+                new IgniteClosure<Cache.Entry<Integer, String>, Integer>() {
+                    @Override public Integer apply(Cache.Entry<Integer, String> e) {
+                        return e.getKey();
+                    }
+                };
+
+            List<Integer> keys = cache.query(new ScanQuery<>(filter), transformer).getAll();
+
+            assertEquals(5, keys.size());
+
+            Collections.sort(keys);
+
+            for (int i = 0; i < 5; i++)
+                assertEquals(i * 10, keys.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetObjectField() throws Exception {
+        IgniteCache<Integer, Value> cache = grid().createCache("test-cache");
+
+        try {
+            for (int i = 0; i < 50; i++)
+                cache.put(i, new Value("str" + i, i * 100));
+
+            IgniteClosure<Cache.Entry<Integer, Value>, Integer> transformer =
+                new IgniteClosure<Cache.Entry<Integer, Value>, Integer>() {
+                    @Override public Integer apply(Cache.Entry<Integer, Value> e) {
+                        return e.getValue().idx;
+                    }
+                };
+
+            List<Integer> res = cache.query(new ScanQuery<Integer, Value>(), transformer).getAll();
+
+            assertEquals(50, res.size());
+
+            Collections.sort(res);
+
+            for (int i = 0; i < 50; i++)
+                assertEquals(i * 100, res.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetObjectFieldFiltered() throws Exception {
+        IgniteCache<Integer, Value> cache = grid().createCache("test-cache");
+
+        try {
+            for (int i = 0; i < 50; i++)
+                cache.put(i, new Value("str" + i, i * 100));
+
+            IgniteBiPredicate<Integer, Value> filter = new IgniteBiPredicate<Integer, Value>() {
+                @Override public boolean apply(Integer k, Value v) {
+                    return v.idx % 1000 == 0;
+                }
+            };
+
+            IgniteClosure<Cache.Entry<Integer, Value>, Integer> transformer =
+                new IgniteClosure<Cache.Entry<Integer, Value>, Integer>() {
+                    @Override public Integer apply(Cache.Entry<Integer, Value> e) {
+                        return e.getValue().idx;
+                    }
+                };
+
+            List<Integer> res = cache.query(new ScanQuery<>(filter), transformer).getAll();
+
+            assertEquals(5, res.size());
+
+            Collections.sort(res);
+
+            for (int i = 0; i < 5; i++)
+                assertEquals(i * 1000, res.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testKeepBinary() throws Exception {
+        IgniteCache<Integer, Value> cache = grid().createCache("test-cache");
+
+        try {
+            for (int i = 0; i < 50; i++)
+                cache.put(i, new Value("str" + i, i * 100));
+
+            IgniteCache<Integer, BinaryObject> binaryCache = cache.withKeepBinary();
+
+            IgniteClosure<Cache.Entry<Integer, BinaryObject>, Integer> transformer =
+                new IgniteClosure<Cache.Entry<Integer, BinaryObject>, Integer>() {
+                    @Override public Integer apply(Cache.Entry<Integer, BinaryObject> e) {
+                        return e.getValue().field("idx");
+                    }
+                };
+
+            List<Integer> res = binaryCache.query(new ScanQuery<Integer, BinaryObject>(), transformer).getAll();
+
+            assertEquals(50, res.size());
+
+            Collections.sort(res);
+
+            for (int i = 0; i < 50; i++)
+                assertEquals(i * 100, res.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testKeepBinaryFiltered() throws Exception {
+        IgniteCache<Integer, Value> cache = grid().createCache("test-cache");
+
+        try {
+            for (int i = 0; i < 50; i++)
+                cache.put(i, new Value("str" + i, i * 100));
+
+            IgniteCache<Integer, BinaryObject> binaryCache = cache.withKeepBinary();
+
+            IgniteBiPredicate<Integer, BinaryObject> filter = new IgniteBiPredicate<Integer, BinaryObject>() {
+                @Override public boolean apply(Integer k, BinaryObject v) {
+                    return v.<Integer>field("idx") % 1000 == 0;
+                }
+            };
+
+            IgniteClosure<Cache.Entry<Integer, BinaryObject>, Integer> transformer =
+                new IgniteClosure<Cache.Entry<Integer, BinaryObject>, Integer>() {
+                    @Override public Integer apply(Cache.Entry<Integer, BinaryObject> e) {
+                        return e.getValue().field("idx");
+                    }
+                };
+
+            List<Integer> res = binaryCache.query(new ScanQuery<>(filter), transformer).getAll();
+
+            assertEquals(5, res.size());
+
+            Collections.sort(res);
+
+            for (int i = 0; i < 5; i++)
+                assertEquals(i * 1000, res.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocal() throws Exception {
+        IgniteCache<Integer, Value> cache = grid().createCache("test-cache");
+
+        try {
+            for (int i = 0; i < 50; i++)
+                cache.put(i, new Value("str" + i, i * 100));
+
+            Collection<List<Integer>> lists = grid().compute().broadcast(new IgniteCallable<List<Integer>>() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                @Override public List<Integer> call() throws Exception {
+                    IgniteClosure<Cache.Entry<Integer, Value>, Integer> transformer =
+                        new IgniteClosure<Cache.Entry<Integer, Value>, Integer>() {
+                            @Override public Integer apply(Cache.Entry<Integer, Value> e) {
+                                return e.getValue().idx;
+                            }
+                        };
+
+                    return ignite.cache("test-cache").query(new ScanQuery<Integer, Value>().setLocal(true),
+                        transformer).getAll();
+                }
+            });
+
+            List<Integer> res = new ArrayList<>(F.flatCollections(lists));
+
+            assertEquals(50, res.size());
+
+            Collections.sort(res);
+
+            for (int i = 0; i < 50; i++)
+                assertEquals(i * 100, res.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocalFiltered() throws Exception {
+        IgniteCache<Integer, Value> cache = grid().createCache("test-cache");
+
+        try {
+            for (int i = 0; i < 50; i++)
+                cache.put(i, new Value("str" + i, i * 100));
+
+            Collection<List<Integer>> lists = grid().compute().broadcast(new IgniteCallable<List<Integer>>() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                @Override public List<Integer> call() throws Exception {
+                    IgniteBiPredicate<Integer, Value> filter = new IgniteBiPredicate<Integer, Value>() {
+                        @Override public boolean apply(Integer k, Value v) {
+                            return v.idx % 1000 == 0;
+                        }
+                    };
+
+                    IgniteClosure<Cache.Entry<Integer, Value>, Integer> transformer =
+                        new IgniteClosure<Cache.Entry<Integer, Value>, Integer>() {
+                            @Override public Integer apply(Cache.Entry<Integer, Value> e) {
+                                return e.getValue().idx;
+                            }
+                        };
+
+                    return ignite.cache("test-cache").query(new ScanQuery<>(filter).setLocal(true),
+                        transformer).getAll();
+                }
+            });
+
+            List<Integer> res = new ArrayList<>(F.flatCollections(lists));
+
+            assertEquals(5, res.size());
+
+            Collections.sort(res);
+
+            for (int i = 0; i < 5; i++)
+                assertEquals(i * 1000, res.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocalKeepBinary() throws Exception {
+        IgniteCache<Integer, Value> cache = grid().createCache("test-cache");
+
+        try {
+            for (int i = 0; i < 50; i++)
+                cache.put(i, new Value("str" + i, i * 100));
+
+            Collection<List<Integer>> lists = grid().compute().broadcast(new IgniteCallable<List<Integer>>() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                @Override public List<Integer> call() throws Exception {
+                    IgniteClosure<Cache.Entry<Integer, BinaryObject>, Integer> transformer =
+                        new IgniteClosure<Cache.Entry<Integer, BinaryObject>, Integer>() {
+                            @Override public Integer apply(Cache.Entry<Integer, BinaryObject> e) {
+                                return e.getValue().field("idx");
+                            }
+                        };
+
+                    return ignite.cache("test-cache").withKeepBinary().query(
+                        new ScanQuery<Integer, BinaryObject>().setLocal(true), transformer).getAll();
+                }
+            });
+
+            List<Integer> res = new ArrayList<>(F.flatCollections(lists));
+
+            assertEquals(50, res.size());
+
+            Collections.sort(res);
+
+            for (int i = 0; i < 50; i++)
+                assertEquals(i * 100, res.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocalKeepBinaryFiltered() throws Exception {
+        IgniteCache<Integer, Value> cache = grid().createCache("test-cache");
+
+        try {
+            for (int i = 0; i < 50; i++)
+                cache.put(i, new Value("str" + i, i * 100));
+
+            Collection<List<Integer>> lists = grid().compute().broadcast(new IgniteCallable<List<Integer>>() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                @Override public List<Integer> call() throws Exception {
+                    IgniteBiPredicate<Integer, BinaryObject> filter = new IgniteBiPredicate<Integer, BinaryObject>() {
+                        @Override public boolean apply(Integer k, BinaryObject v) {
+                            return v.<Integer>field("idx") % 1000 == 0;
+                        }
+                    };
+
+                    IgniteClosure<Cache.Entry<Integer, BinaryObject>, Integer> transformer =
+                        new IgniteClosure<Cache.Entry<Integer, BinaryObject>, Integer>() {
+                            @Override public Integer apply(Cache.Entry<Integer, BinaryObject> e) {
+                                return e.getValue().field("idx");
+                            }
+                        };
+
+                    return ignite.cache("test-cache").withKeepBinary().query(new ScanQuery<>(filter).setLocal(true),
+                        transformer).getAll();
+                }
+            });
+
+            List<Integer> res = new ArrayList<>(F.flatCollections(lists));
+
+            assertEquals(5, res.size());
+
+            Collections.sort(res);
+
+            for (int i = 0; i < 5; i++)
+                assertEquals(i * 1000, res.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUnsupported() throws Exception {
+        final IgniteCache<Integer, Integer> cache = grid().createCache("test-cache");
+
+        final IgniteClosure<Cache.Entry<Integer, Integer>, Integer> transformer =
+            new IgniteClosure<Cache.Entry<Integer, Integer>, Integer>() {
+                @Override public Integer apply(Cache.Entry<Integer, Integer> e) {
+                    return null;
+                }
+            };
+
+        try {
+            GridTestUtils.assertThrows(
+                log,
+                new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        cache.query(new SqlQuery<Integer, Integer>(Integer.class, "clause"), transformer);
+
+                        return null;
+                    }
+                },
+                UnsupportedOperationException.class,
+                "Transformers are supported only for SCAN queries."
+            );
+
+            GridTestUtils.assertThrows(
+                log,
+                new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        cache.query(new SqlFieldsQuery("clause"), new IgniteClosure<List<?>, Object>() {
+                            @Override public Object apply(List<?> objects) {
+                                return null;
+                            }
+                        });
+
+                        return null;
+                    }
+                },
+                UnsupportedOperationException.class,
+                "Transformers are supported only for SCAN queries."
+            );
+
+            GridTestUtils.assertThrows(
+                log,
+                new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        cache.query(new TextQuery<Integer, Integer>(Integer.class, "clause"), transformer);
+
+                        return null;
+                    }
+                },
+                UnsupportedOperationException.class,
+                "Transformers are supported only for SCAN queries."
+            );
+
+            GridTestUtils.assertThrows(
+                log,
+                new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        cache.query(new SpiQuery<Integer, Integer>(), transformer);
+
+                        return null;
+                    }
+                },
+                UnsupportedOperationException.class,
+                "Transformers are supported only for SCAN queries."
+            );
+
+            GridTestUtils.assertThrows(
+                log,
+                new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        cache.query(new ContinuousQuery<Integer, Integer>(), transformer);
+
+                        return null;
+                    }
+                },
+                UnsupportedOperationException.class,
+                "Transformers are supported only for SCAN queries."
+            );
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     */
+    private static class Value {
+        /** */
+        @SuppressWarnings("unused")
+        private String str;
+
+        /** */
+        private int idx;
+
+        /**
+         * @param str String.
+         * @param idx Integer.
+         */
+        public Value(String str, int idx) {
+            this.str = str;
+            this.idx = idx;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d7e1489/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
index 740b201..71dc964 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
@@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
 import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.mxbean.CacheMetricsMXBean;
 import org.apache.ignite.resources.IgniteInstanceResource;
@@ -172,6 +173,11 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public <T, R> QueryCursor<R> query(Query<T> qry, IgniteClosure<T, R> transformer) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
     @Override public Iterable<Entry<K, V>> localEntries(CachePeekMode... peekModes) throws CacheException {
         return compute.call(new LocalEntriesTask<K, V>(cacheName, isAsync, peekModes));
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d7e1489/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 1b1908d..3652acd 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
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.distributed.replicated.Ignite
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryTransformerSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
 import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
@@ -115,6 +116,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteBinaryWrappedObjectFieldsQuerySelfTest.class);
         suite.addTestSuite(IgniteCacheQueryH2IndexingLeakTest.class);
         suite.addTestSuite(IgniteCacheQueryNoRebalanceSelfTest.class);
+        suite.addTestSuite(GridCacheQueryTransformerSelfTest.class);
 
         return suite;
     }