You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/10/16 12:55:47 UTC

[1/7] ignite git commit: IGNITE-6605: SQL: common backup filter. This closes #2836.

Repository: ignite
Updated Branches:
  refs/heads/ignite-3478 b7dce2b56 -> 1b4eb29ba


IGNITE-6605: SQL: common backup filter. This closes #2836.


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

Branch: refs/heads/ignite-3478
Commit: a6e28082ba08f2ecefe6c7bef898b201126997b9
Parents: ae02a1d
Author: devozerov <vo...@gridgain.com>
Authored: Fri Oct 13 14:12:44 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Oct 13 14:12:44 2017 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryManager.java      | 40 ++--------
 .../spi/indexing/IndexingQueryCacheFilter.java  | 72 ++++++++++++++++++
 .../spi/indexing/IndexingQueryFilter.java       | 12 +--
 .../spi/indexing/IndexingQueryFilterImpl.java   | 79 ++++++++++++++++++++
 .../internal/processors/query/h2/H2Cursor.java  | 25 +++----
 .../processors/query/h2/IgniteH2Indexing.java   | 61 +--------------
 .../query/h2/database/H2PkHashIndex.java        | 20 ++---
 .../query/h2/database/H2TreeIndex.java          |  4 +-
 .../query/h2/opt/GridH2IndexBase.java           | 58 +++++++-------
 .../query/h2/opt/GridLuceneIndex.java           | 25 ++-----
 10 files changed, 211 insertions(+), 185 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e28082/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 64e74fb..392b19f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -119,6 +119,7 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.IgniteSpiCloseableIterator;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.apache.ignite.spi.indexing.IndexingQueryFilterImpl;
 import org.apache.ignite.spi.indexing.IndexingSpi;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
@@ -130,7 +131,6 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridClosureCallMode.BROADCAST;
-import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SCAN;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SPI;
@@ -1985,39 +1985,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     }
 
     /**
-     * @param <K> Key type.
-     * @param <V> Value type.
-     * @param includeBackups Include backups.
-     * @return Predicate.
-     */
-    @SuppressWarnings("unchecked")
-    @Nullable public <K, V> IndexingQueryFilter backupsFilter(boolean includeBackups) {
-        if (includeBackups)
-            return null;
-
-        return new IndexingQueryFilter() {
-            @Nullable @Override public IgniteBiPredicate<K, V> forCache(final String cacheName) {
-                final GridKernalContext ctx = cctx.kernalContext();
-
-                final GridCacheAdapter<Object, Object> cache = ctx.cache().internalCache(cacheName);
-
-                if (cache.context().isReplicated() || cache.configuration().getBackups() == 0)
-                    return null;
-
-                return new IgniteBiPredicate<K, V>() {
-                    @Override public boolean apply(K k, V v) {
-                        return cache.context().affinity().primaryByKey(ctx.discovery().localNode(), k, NONE);
-                    }
-                };
-            }
-
-            @Override public boolean isValueRequired() {
-                return false;
-            }
-        };
-    }
-
-    /**
      * @return Topology version for query requests.
      */
     public AffinityTopologyVersion queryTopologyVersion() {
@@ -2029,7 +1996,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @return Filter.
      */
     private IndexingQueryFilter filter(GridCacheQueryAdapter<?> qry) {
-        return backupsFilter(qry.includeBackups());
+        if (qry.includeBackups())
+            return null;
+
+        return new IndexingQueryFilterImpl(cctx.kernalContext(), AffinityTopologyVersion.NONE, null);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e28082/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryCacheFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryCacheFilter.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryCacheFilter.java
new file mode 100644
index 0000000..6257f47
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryCacheFilter.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.indexing;
+
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
+
+import java.util.Set;
+
+/**
+ * Indexing query filter for specific cache.
+ */
+public class IndexingQueryCacheFilter {
+    /** Affinity manager. */
+    private final GridCacheAffinityManager aff;
+
+    /** Partitions. */
+    private final Set<Integer> parts;
+
+    /** Topology version. */
+    private final AffinityTopologyVersion topVer;
+
+    /** Local node. */
+    private final ClusterNode locNode;
+
+    /**
+     * Constructor.
+     *
+     * @param aff Affinity.
+     * @param parts Partitions.
+     * @param topVer Topology version.
+     * @param locNode Local node.
+     */
+    public IndexingQueryCacheFilter(GridCacheAffinityManager aff, Set<Integer> parts,
+        AffinityTopologyVersion topVer, ClusterNode locNode) {
+        this.aff = aff;
+        this.parts = parts;
+        this.topVer = topVer;
+        this.locNode = locNode;
+    }
+
+    /**
+     * Apply filter.
+     *
+     * @param key Key.
+     * @return {@code True} if passed.
+     */
+    public boolean apply(Object key) {
+        int part = aff.partition(key);
+
+        if (parts == null)
+            return aff.primaryByPartition(locNode, part, topVer);
+        else
+            return parts.contains(part);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e28082/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
index 74d349a..b0d9693 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.spi.indexing;
 
-import org.apache.ignite.lang.IgniteBiPredicate;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -30,14 +29,5 @@ public interface IndexingQueryFilter {
      * @param cacheName Cache name.
      * @return Predicate or {@code null} if no filtering is needed.
      */
-    @Nullable public <K, V> IgniteBiPredicate<K, V> forCache(String cacheName);
-
-    /**
-     * Is the value required for filtering logic?
-     * If false then null instead of value will be passed
-     * to IgniteBiPredicate returned by {@link #forCache(String)} method.
-     *
-     * @return true if value is required for filtering, false otherwise.
-     */
-    public boolean isValueRequired();
+    @Nullable public IndexingQueryCacheFilter forCache(String cacheName);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e28082/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilterImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilterImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilterImpl.java
new file mode 100644
index 0000000..53dcbf6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilterImpl.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.indexing;
+
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.HashSet;
+
+/**
+ * Indexing query filter.
+ */
+public class IndexingQueryFilterImpl implements IndexingQueryFilter {
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Topology version. */
+    private final AffinityTopologyVersion topVer;
+
+    /** Partitions. */
+    private final HashSet<Integer> parts;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Kernal context.
+     * @param topVer Topology version.
+     * @param partsArr Partitions array.
+     */
+    public IndexingQueryFilterImpl(GridKernalContext ctx, @Nullable AffinityTopologyVersion topVer,
+        @Nullable int[] partsArr) {
+        this.ctx = ctx;
+
+        this.topVer = topVer != null ? topVer : AffinityTopologyVersion.NONE;
+
+        if (F.isEmpty(partsArr))
+            parts = null;
+        else {
+            parts = new HashSet<>();
+
+            for (int part : partsArr)
+                parts.add(part);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IndexingQueryCacheFilter forCache(String cacheName) {
+        final GridCacheAdapter<Object, Object> cache = ctx.cache().internalCache(cacheName);
+
+        // REPLICATED -> nothing to filter (explicit partitions are not supported).
+        if (cache.context().isReplicated())
+            return null;
+
+        // No backups and explicit partitions -> nothing to filter.
+        if (cache.configuration().getBackups() == 0 && parts == null)
+            return null;
+
+        return new IndexingQueryCacheFilter(cache.context().affinity(), parts, topVer,
+            ctx.discovery().localNode());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e28082/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
index de3111d..e09108d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
@@ -17,14 +17,15 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.query.h2.opt.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.h2.index.*;
-import org.h2.message.*;
-import org.h2.result.*;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter;
+import org.h2.index.Cursor;
+import org.h2.message.DbException;
+import org.h2.result.Row;
+import org.h2.result.SearchRow;
 
 /**
  * Cursor.
@@ -34,7 +35,7 @@ public class H2Cursor implements Cursor {
     private final GridCursor<GridH2Row> cursor;
 
     /** */
-    private final IgniteBiPredicate<Object,Object> filter;
+    private final IndexingQueryCacheFilter filter;
 
     /** */
     private final long time = U.currentTimeMillis();
@@ -43,7 +44,7 @@ public class H2Cursor implements Cursor {
      * @param cursor Cursor.
      * @param filter Filter.
      */
-    public H2Cursor(GridCursor<GridH2Row> cursor, IgniteBiPredicate<Object, Object> filter) {
+    public H2Cursor(GridCursor<GridH2Row> cursor, IndexingQueryCacheFilter filter) {
         assert cursor != null;
 
         this.cursor = cursor;
@@ -85,12 +86,10 @@ public class H2Cursor implements Cursor {
                     return true;
 
                 Object key = row.getValue(0).getObject();
-                Object val = row.getValue(1).getObject();
 
                 assert key != null;
-                assert val != null;
 
-                if (filter.apply(key, val))
+                if (filter.apply(key))
                     return true;
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e28082/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 fddd2e8..c172e65 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
@@ -65,8 +65,6 @@ import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectUtils;
 import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
@@ -130,7 +128,6 @@ import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiClosure;
-import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
@@ -139,6 +136,7 @@ import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.apache.ignite.spi.indexing.IndexingQueryFilterImpl;
 import org.h2.api.ErrorCode;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.command.Prepared;
@@ -2375,62 +2373,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** {@inheritDoc} */
     @Override public IndexingQueryFilter backupFilter(@Nullable final AffinityTopologyVersion topVer,
         @Nullable final int[] parts) {
-        final AffinityTopologyVersion topVer0 = topVer != null ? topVer : AffinityTopologyVersion.NONE;
-
-        return new IndexingQueryFilter() {
-            @Nullable @Override public <K, V> IgniteBiPredicate<K, V> forCache(String cacheName) {
-                final GridCacheAdapter<Object, Object> cache = ctx.cache().internalCache(cacheName);
-
-                if (cache.context().isReplicated())
-                    return null;
-
-                final GridCacheAffinityManager aff = cache.context().affinity();
-
-                if (parts != null) {
-                    if (parts.length < 64) { // Fast scan for small arrays.
-                        return new IgniteBiPredicate<K, V>() {
-                            @Override public boolean apply(K k, V v) {
-                                int p = aff.partition(k);
-
-                                for (int p0 : parts) {
-                                    if (p0 == p)
-                                        return true;
-
-                                    if (p0 > p) // Array is sorted.
-                                        return false;
-                                }
-
-                                return false;
-                            }
-                        };
-                    }
-
-                    return new IgniteBiPredicate<K, V>() {
-                        @Override public boolean apply(K k, V v) {
-                            int p = aff.partition(k);
-
-                            return Arrays.binarySearch(parts, p) >= 0;
-                        }
-                    };
-                }
-
-                final ClusterNode locNode = ctx.discovery().localNode();
-
-                return new IgniteBiPredicate<K, V>() {
-                    @Override public boolean apply(K k, V v) {
-                        return aff.primaryByKey(locNode, k, topVer0);
-                    }
-                };
-            }
-
-            @Override public boolean isValueRequired() {
-                return false;
-            }
-
-            @Override public String toString() {
-                return "IndexingQueryFilter [ver=" + topVer + ']';
-            }
-        };
+        return new IndexingQueryFilterImpl(ctx, topVer, parts);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e28082/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
index b32bfb8..6691485 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
@@ -31,8 +31,8 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.util.lang.GridCursor;
-import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter;
 import org.h2.engine.Session;
 import org.h2.index.Cursor;
 import org.h2.index.IndexType;
@@ -85,7 +85,7 @@ public class H2PkHashIndex extends GridH2IndexBase {
     /** {@inheritDoc} */
     @Override public Cursor find(Session ses, final SearchRow lower, final SearchRow upper) {
         IndexingQueryFilter f = threadLocalFilter();
-        IgniteBiPredicate<Object, Object> p = null;
+        IndexingQueryCacheFilter p = null;
 
         if (f != null) {
             String cacheName = getTable().cacheName();
@@ -179,13 +179,13 @@ public class H2PkHashIndex extends GridH2IndexBase {
         final GridCursor<? extends CacheDataRow> cursor;
 
         /** */
-        final IgniteBiPredicate<Object, Object> filter;
+        final IndexingQueryCacheFilter filter;
 
         /**
          * @param cursor Cursor.
          * @param filter Filter.
          */
-        private H2Cursor(GridCursor<? extends CacheDataRow> cursor, IgniteBiPredicate<Object, Object> filter) {
+        private H2Cursor(GridCursor<? extends CacheDataRow> cursor, IndexingQueryCacheFilter filter) {
             assert cursor != null;
 
             this.cursor = cursor;
@@ -222,17 +222,7 @@ public class H2PkHashIndex extends GridH2IndexBase {
 
                     CacheDataRow dataRow = cursor.get();
 
-                    GridH2Row row = tbl.rowDescriptor().createRow(dataRow.key(), dataRow.partition(), dataRow.value(), dataRow.version(), 0);
-
-                    row.link(dataRow.link());
-
-                    Object key = row.getValue(0).getObject();
-                    Object val = row.getValue(1).getObject();
-
-                    assert key != null;
-                    assert val != null;
-
-                    if (filter.apply(key, val))
+                    if (filter.apply(dataRow.key()))
                         return true;
                 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e28082/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
index 5c3e1bd..1a3ea4a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
@@ -34,8 +34,8 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.util.IgniteTree;
 import org.apache.ignite.internal.util.lang.GridCursor;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter;
 import org.h2.engine.Session;
 import org.h2.index.Cursor;
 import org.h2.index.IndexType;
@@ -166,7 +166,7 @@ public class H2TreeIndex extends GridH2IndexBase {
     @Override public Cursor find(Session ses, SearchRow lower, SearchRow upper) {
         try {
             IndexingQueryFilter f = threadLocalFilter();
-            IgniteBiPredicate<Object, Object> p = null;
+            IndexingQueryCacheFilter p = null;
 
             if (f != null) {
                 String cacheName = getTable().cacheName();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e28082/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index 048192a..92b7d10 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -17,12 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.h2.opt;
 
-import java.util.*;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
@@ -41,17 +35,18 @@ import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2RowRange
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2RowRangeBounds;
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2ValueMessage;
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2ValueMessageFactory;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteTree;
+import org.apache.ignite.internal.util.lang.GridCursor;
 import org.apache.ignite.internal.util.typedef.CIX2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter;
 import org.h2.engine.Session;
 import org.h2.index.BaseIndex;
 import org.h2.index.Cursor;
@@ -68,13 +63,29 @@ import org.h2.value.Value;
 import org.h2.value.ValueNull;
 import org.jetbrains.annotations.Nullable;
 
+import javax.cache.CacheException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
 import static java.util.Collections.emptyIterator;
 import static java.util.Collections.singletonList;
 import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.LOCAL_ONLY;
 import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.OFF;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.KEY_COL;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.VAL_COL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2CollocationModel.buildCollocationModel;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.KEY_COL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.MAP;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.PREPARE;
 import static org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2IndexRangeResponse.STATUS_ERROR;
@@ -1574,15 +1585,13 @@ public abstract class GridH2IndexBase extends BaseIndex {
     protected static class FilteringCursor implements GridCursor<GridH2Row> {
         /** */
         private final GridCursor<GridH2Row> cursor;
+
         /** */
-        private final IgniteBiPredicate<Object, Object> fltr;
+        private final IndexingQueryCacheFilter fltr;
 
         /** */
         private final long time;
 
-        /** Is value required for filtering predicate? */
-        private final boolean isValRequired;
-
         /** */
         private GridH2Row next;
 
@@ -1595,19 +1604,8 @@ public abstract class GridH2IndexBase extends BaseIndex {
         protected FilteringCursor(GridCursor<GridH2Row> cursor, long time, IndexingQueryFilter qryFilter,
             String cacheName) {
             this.cursor = cursor;
-
             this.time = time;
-
-            if (qryFilter != null) {
-                this.fltr = qryFilter.forCache(cacheName);
-
-                this.isValRequired = qryFilter.isValueRequired();
-            }
-            else {
-                this.fltr = null;
-
-                this.isValRequired = false;
-            }
+            this.fltr = qryFilter != null ? qryFilter.forCache(cacheName) : null;
         }
 
         /**
@@ -1623,12 +1621,8 @@ public abstract class GridH2IndexBase extends BaseIndex {
                 return true;
 
             Object key = row.getValue(KEY_COL).getObject();
-            Object val = isValRequired ? row.getValue(VAL_COL).getObject() : null;
-
-            assert key != null;
-            assert !isValRequired || val != null;
 
-            return fltr.apply(key, val);
+            return fltr.apply(key);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e28082/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
index f8d3ef2..b5d2456 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
@@ -32,9 +32,9 @@ import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -290,7 +290,7 @@ public class GridLuceneIndex implements AutoCloseable {
             throw new IgniteCheckedException(e);
         }
 
-        IgniteBiPredicate<K, V> fltr = null;
+        IndexingQueryCacheFilter fltr = null;
 
         if (filters != null)
             fltr = filters.forCache(cacheName);
@@ -321,7 +321,7 @@ public class GridLuceneIndex implements AutoCloseable {
         private final ScoreDoc[] docs;
 
         /** */
-        private final IgniteBiPredicate<K, V> filters;
+        private final IndexingQueryCacheFilter filters;
 
         /** */
         private int idx;
@@ -341,7 +341,7 @@ public class GridLuceneIndex implements AutoCloseable {
          * @param filters Filters over result.
          * @throws IgniteCheckedException if failed.
          */
-        private It(IndexReader reader, IndexSearcher searcher, ScoreDoc[] docs, IgniteBiPredicate<K, V> filters)
+        private It(IndexReader reader, IndexSearcher searcher, ScoreDoc[] docs, IndexingQueryCacheFilter filters)
             throws IgniteCheckedException {
             this.reader = reader;
             this.searcher = searcher;
@@ -354,17 +354,6 @@ public class GridLuceneIndex implements AutoCloseable {
         }
 
         /**
-         * Filters key using predicates.
-         *
-         * @param key Key.
-         * @param val Value.
-         * @return {@code True} if key passes filter.
-         */
-        private boolean filter(K key, V val) {
-            return filters == null || filters.apply(key, val);
-        }
-
-        /**
          * @param bytes Bytes.
          * @param ldr Class loader.
          * @return Object.
@@ -404,15 +393,15 @@ public class GridLuceneIndex implements AutoCloseable {
 
                 K k = unmarshall(doc.getBinaryValue(KEY_FIELD_NAME).bytes, ldr);
 
+                if (filters != null && !filters.apply(k))
+                    continue;
+
                 V v = type.valueClass() == String.class ?
                     (V)doc.get(VAL_STR_FIELD_NAME) :
                     this.<V>unmarshall(doc.getBinaryValue(VAL_FIELD_NAME).bytes, ldr);
 
                 assert v != null;
 
-                if (!filter(k, v))
-                    continue;
-
                 curr = new IgniteBiTuple<>(k, v);
 
                 break;


[7/7] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-3478

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-3478


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

Branch: refs/heads/ignite-3478
Commit: 1b4eb29ba842b6d317ce7e6b74c2c5d8c3820070
Parents: b7dce2b 6679b6c
Author: sboikov <sb...@gridgain.com>
Authored: Mon Oct 16 15:50:31 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Oct 16 15:50:31 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/IgniteServicesImpl.java     |   4 +-
 .../cache/query/GridCacheQueryManager.java      |  40 +-
 .../service/GridServiceProcessor.java           |  59 +--
 .../spi/indexing/IndexingQueryCacheFilter.java  |  72 ++++
 .../spi/indexing/IndexingQueryFilter.java       |  12 +-
 .../spi/indexing/IndexingQueryFilterImpl.java   |  79 ++++
 .../GridServiceProcessorMultiNodeSelfTest.java  |  71 ++-
 .../internal/processors/query/h2/H2Cursor.java  |  25 +-
 .../processors/query/h2/H2RowDescriptor.java    | 431 -------------------
 .../processors/query/h2/H2TableDescriptor.java  |   2 +-
 .../processors/query/h2/H2TableEngine.java      |   5 +-
 .../internal/processors/query/h2/H2Utils.java   |   3 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  74 +---
 .../query/h2/database/H2PkHashIndex.java        |  27 +-
 .../query/h2/database/H2RowFactory.java         |   8 +-
 .../query/h2/database/H2TreeIndex.java          |   4 +-
 .../query/h2/database/io/H2ExtrasInnerIO.java   |   4 +-
 .../query/h2/database/io/H2ExtrasLeafIO.java    |   4 +-
 .../query/h2/database/io/H2InnerIO.java         |   4 +-
 .../query/h2/database/io/H2LeafIO.java          |   4 +-
 .../query/h2/opt/GridH2IndexBase.java           |  58 ++-
 .../query/h2/opt/GridH2KeyRowOnheap.java        |  59 +++
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   | 170 +++-----
 .../query/h2/opt/GridH2MetaTable.java           |  13 +-
 .../query/h2/opt/GridH2PlainRowFactory.java     | 181 ++++++++
 .../processors/query/h2/opt/GridH2Row.java      | 122 ++----
 .../query/h2/opt/GridH2RowDescriptor.java       | 410 ++++++++++++++++--
 .../query/h2/opt/GridH2RowFactory.java          | 194 ---------
 .../query/h2/opt/GridH2SearchRowAdapter.java    | 103 +++++
 .../processors/query/h2/opt/GridH2Table.java    |  30 +-
 .../query/h2/opt/GridLuceneIndex.java           |  25 +-
 .../query/h2/twostep/GridMergeIndexSorted.java  |   4 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   4 +-
 ...ributedJoinPartitionedAndReplicatedTest.java |   2 +
 .../query/h2/opt/GridH2TableSelfTest.java       | 369 ----------------
 .../IgniteCacheQuerySelfTestSuite.java          |  17 +
 .../IgniteCacheQuerySelfTestSuite2.java         |   6 +
 .../IgniteCacheQuerySelfTestSuite3.java         |   3 -
 .../IgniteDistributedJoinTestSuite.java         |  55 ---
 .../Cache/Query/Linq/CacheLinqTest.Strings.cs   |  18 +
 .../Impl/CacheQueryExpressionVisitor.cs         |   5 +
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |  55 ++-
 42 files changed, 1277 insertions(+), 1558 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1b4eb29b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b4eb29b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
----------------------------------------------------------------------


[4/7] ignite git commit: IGNITE-6632: SQL: simplified GridH2Row inheritance tree. This closes #2856.

Posted by sb...@apache.org.
IGNITE-6632: SQL: simplified GridH2Row inheritance tree. This closes #2856.


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

Branch: refs/heads/ignite-3478
Commit: 0323468816f3bb5fa8cdcaa3cac1958f3dc89f6e
Parents: 583e3461
Author: devozerov <vo...@gridgain.com>
Authored: Mon Oct 16 12:38:54 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 16 12:38:54 2017 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |   4 +-
 .../query/h2/database/H2RowFactory.java         |   2 +-
 .../query/h2/database/io/H2ExtrasInnerIO.java   |   4 +-
 .../query/h2/database/io/H2ExtrasLeafIO.java    |   4 +-
 .../query/h2/database/io/H2InnerIO.java         |   4 +-
 .../query/h2/database/io/H2LeafIO.java          |   4 +-
 .../query/h2/opt/GridH2KeyRowOnheap.java        |  59 +++
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |  35 --
 .../query/h2/opt/GridH2MetaTable.java           |  13 +-
 .../query/h2/opt/GridH2PlainRowFactory.java     | 181 +++++++++
 .../processors/query/h2/opt/GridH2Row.java      | 122 ++----
 .../query/h2/opt/GridH2RowDescriptor.java       |  12 +-
 .../query/h2/opt/GridH2RowFactory.java          | 194 ----------
 .../query/h2/opt/GridH2SearchRowAdapter.java    | 103 ++++++
 .../processors/query/h2/opt/GridH2Table.java    |   2 +-
 .../query/h2/twostep/GridMergeIndexSorted.java  |   4 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   4 +-
 .../query/h2/opt/GridH2TableSelfTest.java       | 369 -------------------
 18 files changed, 400 insertions(+), 720 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/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 dd35723..0fdc2e4 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
@@ -106,7 +106,7 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowFactory;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2PlainRowFactory;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
@@ -199,7 +199,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private static final String DB_OPTIONS = ";LOCK_MODE=3;MULTI_THREADED=1;DB_CLOSE_ON_EXIT=FALSE" +
         ";DEFAULT_LOCK_TIMEOUT=10000;FUNCTIONS_IN_SCHEMA=true;OPTIMIZE_REUSE_RESULTS=0;QUERY_CACHE_SIZE=0" +
         ";RECOMPILE_ALWAYS=1;MAX_OPERATION_MEMORY=0;NESTED_JOINS=0;BATCH_JOINS=1" +
-        ";ROW_FACTORY=\"" + GridH2RowFactory.class.getName() + "\"" +
+        ";ROW_FACTORY=\"" + GridH2PlainRowFactory.class.getName() + "\"" +
         ";DEFAULT_TABLE_ENGINE=" + GridH2DefaultTableEngine.class.getName();
 
         // Uncomment this setting to get debug output from H2 to sysout.

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java
index 92ecd3d..7116fe7 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java
@@ -71,7 +71,7 @@ public class H2RowFactory {
             throw new IgniteException(e);
         }
 
-        assert row.ver != null;
+        assert row.version() != null;
 
         return row;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java
index 7d41617..b8877e9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java
@@ -81,7 +81,7 @@ public class H2ExtrasInnerIO extends BPlusInnerIO<SearchRow> {
     @Override public void storeByOffset(long pageAddr, int off, SearchRow row) {
         GridH2Row row0 = (GridH2Row)row;
 
-        assert row0.link != 0 : row0;
+        assert row0.link() != 0 : row0;
 
         List<InlineIndexHelper> inlineIdxs = InlineIndexHelper.getCurrentInlineIndexes();
 
@@ -101,7 +101,7 @@ public class H2ExtrasInnerIO extends BPlusInnerIO<SearchRow> {
             fieldOff += size;
         }
 
-        PageUtils.putLong(pageAddr, off + payloadSize, row0.link);
+        PageUtils.putLong(pageAddr, off + payloadSize, row0.link());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java
index 3fe72b7..6161f8d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java
@@ -81,7 +81,7 @@ public class H2ExtrasLeafIO extends BPlusLeafIO<SearchRow> {
     @Override public void storeByOffset(long pageAddr, int off, SearchRow row) {
         GridH2Row row0 = (GridH2Row)row;
 
-        assert row0.link != 0;
+        assert row0.link() != 0;
 
         List<InlineIndexHelper> inlineIdxs = InlineIndexHelper.getCurrentInlineIndexes();
 
@@ -100,7 +100,7 @@ public class H2ExtrasLeafIO extends BPlusLeafIO<SearchRow> {
             fieldOff += size;
         }
 
-        PageUtils.putLong(pageAddr, off + payloadSize, row0.link);
+        PageUtils.putLong(pageAddr, off + payloadSize, row0.link());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
index 4d7b3a2..a1f1ce9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
@@ -47,9 +47,9 @@ public class H2InnerIO extends BPlusInnerIO<SearchRow> implements H2RowLinkIO {
     @Override public void storeByOffset(long pageAddr, int off, SearchRow row) {
         GridH2Row row0 = (GridH2Row)row;
 
-        assert row0.link != 0;
+        assert row0.link() != 0;
 
-        PageUtils.putLong(pageAddr, off, row0.link);
+        PageUtils.putLong(pageAddr, off, row0.link());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
index f292fc1..85dcf50 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
@@ -47,9 +47,9 @@ public class H2LeafIO extends BPlusLeafIO<SearchRow> implements H2RowLinkIO {
     @Override public void storeByOffset(long pageAddr, int off, SearchRow row) {
         GridH2Row row0 = (GridH2Row)row;
 
-        assert row0.link != 0;
+        assert row0.link() != 0;
 
-        PageUtils.putLong(pageAddr, off, row0.link);
+        PageUtils.putLong(pageAddr, off, row0.link());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java
new file mode 100644
index 0000000..a0716c9
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2.opt;
+
+import org.h2.value.Value;
+
+/**
+ * Heap-based key-only row for remove operations.
+ */
+public class GridH2KeyRowOnheap extends GridH2Row {
+    /** */
+    private Value key;
+
+    /**
+     * @param key Key.
+     */
+    public GridH2KeyRowOnheap(Value key) {
+        this.key = key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getColumnCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value getValue(int idx) {
+        assert idx == 0 : idx;
+
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setValue(int idx, Value v) {
+        assert idx == 0 : idx;
+
+        key = v;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long expireTime() {
+        return 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java
index 63b4606..ad93fec 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java
@@ -89,11 +89,6 @@ public class GridH2KeyValueRowOnheap extends GridH2Row {
     }
 
     /** {@inheritDoc} */
-    @Override public Value[] getValueList() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
     @Override public long expireTime() {
         return expirationTime;
     }
@@ -225,41 +220,11 @@ public class GridH2KeyValueRowOnheap extends GridH2Row {
     }
 
     /** {@inheritDoc} */
-    @Override public void setKeyAndVersion(SearchRow old) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
     @Override public void setKey(long key) {
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public Row getCopy() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setDeleted(boolean deleted) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getKey() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setSessionId(int sesId) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setVersion(int ver) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
     @Override public void setValue(int idx, Value v) {
         throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java
index d23515b..5e09a86 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java
@@ -86,7 +86,7 @@ public class GridH2MetaTable extends TableBase {
     /** {@inheritDoc} */
     @Override public SearchRow getTemplateSimpleRow(boolean singleColumn) {
         if (singleColumn)
-            return GridH2RowFactory.create((Value)null);
+            return GridH2PlainRowFactory.create((Value)null);
 
         return new MetaRow();
     }
@@ -219,7 +219,7 @@ public class GridH2MetaTable extends TableBase {
     /**
      * Get value row.
      */
-    private static class MetaRow extends GridH2Row {
+    private static class MetaRow extends GridH2SearchRowAdapter {
         /** */
         private Value v0;
 
@@ -284,11 +284,6 @@ public class GridH2MetaTable extends TableBase {
                     throw new IllegalStateException("Index: " + idx);
             }
         }
-
-        /** {@inheritDoc} */
-        @Override public long expireTime() {
-            return 0;
-        }
     }
 
     /**
@@ -296,7 +291,7 @@ public class GridH2MetaTable extends TableBase {
      */
     private static class MetaIndex extends BaseIndex {
         /** */
-        private final ConcurrentMap<ValueInt, GridH2Row> rows = new ConcurrentHashMap8<>();
+        private final ConcurrentMap<ValueInt, Row> rows = new ConcurrentHashMap8<>();
 
         /** {@inheritDoc} */
         @Override public void checkRename() {
@@ -322,7 +317,7 @@ public class GridH2MetaTable extends TableBase {
 
         /** {@inheritDoc} */
         @Override public void add(Session session, Row row) {
-            rows.put(id(row), (GridH2Row)row);
+            rows.put(id(row), row);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PlainRowFactory.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PlainRowFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PlainRowFactory.java
new file mode 100644
index 0000000..fd8a613
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PlainRowFactory.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2.opt;
+
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.h2.result.Row;
+import org.h2.result.RowFactory;
+import org.h2.value.Value;
+
+/**
+ * Row factory.
+ */
+public class GridH2PlainRowFactory extends RowFactory {
+    /**
+     * @param v Value.
+     * @return Row.
+     */
+    public static Row create(Value v) {
+        return new RowKey(v);
+    }
+
+    /**
+     * @param data Values.
+     * @return Row.
+     */
+    public static Row create(Value... data) {
+        switch (data.length) {
+            case 0:
+                throw new IllegalStateException("Zero columns row.");
+
+            case 1:
+                return new RowKey(data[0]);
+
+            case 2:
+                return new RowPair(data[0], data[1]);
+
+            default:
+                return new RowSimple(data);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Row createRow(Value[] data, int memory) {
+        return create(data);
+    }
+
+    /**
+     * Single value row.
+     */
+    private static final class RowKey extends GridH2SearchRowAdapter {
+        /** */
+        private Value key;
+
+        /**
+         * @param key Key.
+         */
+        public RowKey(Value key) {
+            this.key = key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int getColumnCount() {
+            return 1;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Value getValue(int idx) {
+            assert idx == 0 : idx;
+            return key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setValue(int idx, Value v) {
+            assert idx == 0 : idx;
+            key = v;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(RowKey.class, this);
+        }
+    }
+
+    /**
+     * Row of two values.
+     */
+    private static final class RowPair extends GridH2SearchRowAdapter  {
+        /** */
+        private Value v1;
+
+        /** */
+        private Value v2;
+
+        /**
+         * @param v1 First value.
+         * @param v2 Second value.
+         */
+        private RowPair(Value v1, Value v2) {
+            this.v1 = v1;
+            this.v2 = v2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int getColumnCount() {
+            return 2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Value getValue(int idx) {
+            return idx == 0 ? v1 : v2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setValue(int idx, Value v) {
+            if (idx == 0)
+                v1 = v;
+            else {
+                assert idx == 1 : idx;
+
+                v2 = v;
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(RowPair.class, this);
+        }
+    }
+
+    /**
+     * Simple array based row.
+     */
+    private static final class RowSimple extends GridH2SearchRowAdapter {
+        /** */
+        @GridToStringInclude
+        private Value[] vals;
+
+        /**
+         * @param vals Values.
+         */
+        private RowSimple(Value[] vals) {
+            this.vals = vals;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int getColumnCount() {
+            return vals.length;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Value getValue(int idx) {
+            return vals[idx];
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setValue(int idx, Value v) {
+            vals[idx] = v;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(RowSimple.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
index fdeb009..4cb603b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
@@ -21,29 +21,25 @@ import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.h2.result.Row;
-import org.h2.result.SearchRow;
-import org.h2.store.Data;
-import org.h2.value.Value;
 
 /**
  * Row with locking support needed for unique key conflicts resolution.
  */
-public abstract class GridH2Row implements SearchRow, CacheDataRow, Row {
-    /** */
-    public long link; // TODO remove
+public abstract class GridH2Row extends GridH2SearchRowAdapter implements CacheDataRow {
+    /** Link. */
+    private long link;
 
-    /** */
-    public KeyCacheObject key; // TODO remove
+    /** Key. */
+    private KeyCacheObject key;
 
-    /** */
-    public CacheObject val; // TODO remove
+    /** Value. */
+    private CacheObject val;
 
-    /** */
-    public GridCacheVersion ver; // TODO remove
+    /** Version. */
+    private GridCacheVersion ver;
 
-    /** */
-    public int partId; // TODO remove
+    /** Partition. */
+    private int partId;
 
     /** {@inheritDoc} */
     @Override public KeyCacheObject key() {
@@ -60,16 +56,37 @@ public abstract class GridH2Row implements SearchRow, CacheDataRow, Row {
         return val;
     }
 
+    /**
+     * @param val Value.
+     */
+    public void value(CacheObject val) {
+        this.val = val;
+    }
+
     /** {@inheritDoc} */
     @Override public GridCacheVersion version() {
         return ver;
     }
 
+    /**
+     * @param ver Version.
+     */
+    public void version(GridCacheVersion ver) {
+        this.ver = ver;
+    }
+
     /** {@inheritDoc} */
     @Override public int partition() {
         return partId;
     }
 
+    /**
+     * @param partId Partition.
+     */
+    public void partition(int partId) {
+        this.partId = partId;
+    }
+
     /** {@inheritDoc} */
     @Override public long link() {
         return link;
@@ -81,81 +98,6 @@ public abstract class GridH2Row implements SearchRow, CacheDataRow, Row {
     }
 
     /** {@inheritDoc} */
-    @Override public Row getCopy() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setVersion(int version) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getByteCount(Data dummy) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isEmpty() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setDeleted(boolean deleted) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setSessionId(int sessionId) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getSessionId() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void commit() {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isDeleted() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setKeyAndVersion(SearchRow old) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getVersion() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setKey(long key) {
-        // No-op, may be set in H2 INFORMATION_SCHEMA.
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getKey() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getMemory() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Value[] getValueList() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
     @Override public int hash() {
         throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
index 081805e..503e487 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
@@ -287,7 +287,7 @@ public class GridH2RowDescriptor {
 
         try {
             if (val == null) // Only can happen for remove operation, can create simple search row.
-                row = GridH2RowFactory.create(wrap(key, keyType));
+                row = new GridH2KeyRowOnheap(wrap(key, keyType));
             else
                 row = new GridH2KeyValueRowOnheap(this, key, keyType, val, valType, ver, expirationTime);
         }
@@ -297,12 +297,10 @@ public class GridH2RowDescriptor {
                 "or configure key type as common super class for all actual keys for this value type.", e);
         }
 
-        row.ver = ver;
-
-        row.key = key;
-        row.val = val;
-        row.partId = partId;
-
+        row.version(ver);
+        row.key(key);
+        row.value(val);
+        row.partition(partId);
         row.link(link);
 
         return row;

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java
deleted file mode 100644
index d33917f..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.h2.opt;
-
-import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.h2.result.RowFactory;
-import org.h2.value.Value;
-
-/**
- * Row factory.
- */
-public class GridH2RowFactory extends RowFactory {
-    /**
-     * @param v Value.
-     * @return Row.
-     */
-    public static GridH2Row create(Value v) {
-        return new RowKey(v);
-    }
-
-    /**
-     * @param v1 Value 1.
-     * @param v2 Value 2.
-     * @return Row.
-     */
-    public static GridH2Row create(Value v1, Value v2) {
-        return new RowPair(v1, v2);
-    }
-
-    /**
-     * @param data Values.
-     * @return Row.
-     */
-    public static GridH2Row create(Value... data) {
-        switch (data.length) {
-            case 0:
-                throw new IllegalStateException("Zero columns row.");
-
-            case 1:
-                return new RowKey(data[0]);
-
-            case 2:
-                return new RowPair(data[0], data[1]);
-
-            default:
-                return new RowSimple(data);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridH2Row createRow(Value[] data, int memory) {
-        return create(data);
-    }
-
-    /**
-     * Single value row.
-     */
-    private static final class RowKey extends GridH2Row {
-        /** */
-        private Value key;
-
-        /**
-         * @param key Key.
-         */
-        public RowKey(Value key) {
-            this.key = key;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int getColumnCount() {
-            return 1;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Value getValue(int idx) {
-            assert idx == 0 : idx;
-            return key;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setValue(int idx, Value v) {
-            assert idx == 0 : idx;
-            key = v;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long expireTime() {
-            return 0;
-        }
-    }
-
-    /**
-     * Row of two values.
-     */
-    private static final class RowPair extends GridH2Row  {
-        /** */
-        private Value v1;
-
-        /** */
-        private Value v2;
-
-        /**
-         * @param v1 First value.
-         * @param v2 Second value.
-         */
-        private RowPair(Value v1, Value v2) {
-            this.v1 = v1;
-            this.v2 = v2;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int getColumnCount() {
-            return 2;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Value getValue(int idx) {
-            return idx == 0 ? v1 : v2;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setValue(int idx, Value v) {
-            if (idx == 0)
-                v1 = v;
-            else {
-                assert idx == 1 : idx;
-
-                v2 = v;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public long expireTime() {
-            return 0;
-        }
-    }
-
-    /**
-     * Simple array based row.
-     */
-    private static final class RowSimple extends GridH2Row {
-        /** */
-        @GridToStringInclude
-        private Value[] vals;
-
-        /**
-         * @param vals Values.
-         */
-        private RowSimple(Value[] vals) {
-            this.vals = vals;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int getColumnCount() {
-            return vals.length;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Value getValue(int idx) {
-            return vals[idx];
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setValue(int idx, Value v) {
-            vals[idx] = v;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(RowSimple.class, this);
-        }
-
-        /** {@inheritDoc} */
-        @Override public long expireTime() {
-            return 0;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRowAdapter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRowAdapter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRowAdapter.java
new file mode 100644
index 0000000..24a90b3
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRowAdapter.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2.opt;
+
+import org.h2.result.Row;
+import org.h2.result.SearchRow;
+import org.h2.store.Data;
+import org.h2.value.Value;
+
+/**
+ * Dummy H2 search row adadpter.
+ */
+public abstract class GridH2SearchRowAdapter implements Row {
+    /** {@inheritDoc} */
+    @Override public void setKeyAndVersion(SearchRow old) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getVersion() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setKey(long key) {
+        // No-op, may be set in H2 INFORMATION_SCHEMA.
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getKey() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMemory() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Row getCopy() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setVersion(int version) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getByteCount(Data dummy) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isEmpty() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setDeleted(boolean deleted) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setSessionId(int sessionId) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getSessionId() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void commit() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isDeleted() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value[] getValueList() {
+        throw new UnsupportedOperationException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index d20b56b..add2488 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -463,7 +463,7 @@ public class GridH2Table extends TableBase {
             GridH2IndexBase pk = pk();
 
             if (!del) {
-                assert rowFactory == null || row.link != 0 : row;
+                assert rowFactory == null || row.link() != 0 : row;
 
                 GridH2Row old = pk.put(row); // Put to PK.
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexSorted.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexSorted.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexSorted.java
index 54c8dd4..0dc8354 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexSorted.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexSorted.java
@@ -33,7 +33,7 @@ import java.util.concurrent.locks.ReentrantLock;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Cursor;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowFactory;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2PlainRowFactory;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.h2.engine.Session;
@@ -368,7 +368,7 @@ public final class GridMergeIndexSorted extends GridMergeIndex {
             if (!iter.hasNext())
                 return false;
 
-            cur = GridH2RowFactory.create(iter.next());
+            cur = GridH2PlainRowFactory.create(iter.next());
 
             return true;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
index c53b58f..487d386 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
@@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Cursor;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowFactory;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2PlainRowFactory;
 import org.h2.engine.Session;
 import org.h2.index.Cursor;
 import org.h2.index.IndexType;
@@ -139,7 +139,7 @@ public final class GridMergeIndexUnsorted extends GridMergeIndex {
             }
 
             @Override public Row next() {
-                return GridH2RowFactory.create(iter.next());
+                return GridH2PlainRowFactory.create(iter.next());
             }
 
             @Override public void remove() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/03234688/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
deleted file mode 100644
index a1a64e8..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
+++ /dev/null
@@ -1,369 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.h2.opt;
-
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.sql.Timestamp;
-import java.util.Random;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.h2.value.ValueLong;
-import org.h2.value.ValueString;
-import org.h2.value.ValueTimestamp;
-import org.h2.value.ValueUuid;
-
-/**
- * Tests H2 Table.
- */
-@SuppressWarnings({"TypeMayBeWeakened", "FieldAccessedSynchronizedAndUnsynchronized"})
-public class GridH2TableSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final long MAX_X = 2000;
-
-    /** */
-    private static final String DB_URL = "jdbc:h2:mem:gg_table_engine;MULTI_THREADED=1;OPTIMIZE_REUSE_RESULTS=0;" +
-        "QUERY_CACHE_SIZE=0;RECOMPILE_ALWAYS=1";
-
-    /** */
-    private static final String CREATE_TABLE_SQL = "CREATE TABLE T(ID UUID, T TIMESTAMP, STR VARCHAR, X BIGINT)";
-
-    /** */
-    private static final String PK_NAME = "__GG_PK_";
-
-    /** Hash. */
-    private static final String HASH = "__GG_HASH";
-
-    /** */
-    private static final String STR_IDX_NAME = "__GG_IDX_";
-
-    /** */
-    private static final String NON_UNIQUE_IDX_NAME = "__GG_IDX_";
-
-    /** */
-    private static final String SCAN_IDX_NAME = GridH2PrimaryScanIndex.SCAN_INDEX_NAME_SUFFIX;
-
-    /** */
-    private Connection conn;
-
-    /** */
-    private GridH2Table tbl;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        // TODO: IGNITE-4994: Restore mock.
-//        Driver.load();
-//
-//        conn = DriverManager.getConnection(DB_URL);
-//
-//        tbl = GridH2Table.Engine.createTable(conn, CREATE_TABLE_SQL, null, new GridH2Table.IndexesFactory() {
-//            @Override public void onTableCreated(GridH2Table tbl) {
-//                // No-op.
-//            }
-//
-//            @Override public H2RowFactory createRowFactory(GridH2Table tbl) {
-//                return null;
-//            }
-//
-//            @Override public ArrayList<Index> createIndexes(GridH2Table tbl) {
-//                ArrayList<Index> idxs = new ArrayList<>();
-//
-//                IndexColumn id = tbl.indexColumn(0, SortOrder.ASCENDING);
-//                IndexColumn t = tbl.indexColumn(1, SortOrder.ASCENDING);
-//                IndexColumn str = tbl.indexColumn(2, SortOrder.DESCENDING);
-//                IndexColumn x = tbl.indexColumn(3, SortOrder.DESCENDING);
-//
-//                idxs.add(new H2PkHashIndex(null, tbl, HASH, F.asList(id)));
-//                idxs.add(new GridH2TreeIndex(PK_NAME, tbl, true, F.asList(id)));
-//                idxs.add(new GridH2TreeIndex(NON_UNIQUE_IDX_NAME, tbl, false, F.asList(x, t, id)));
-//                idxs.add(new GridH2TreeIndex(STR_IDX_NAME, tbl, false, F.asList(str, id)));
-//
-//                return idxs;
-//            }
-//        }, null);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        conn.close();
-
-        conn = null;
-        tbl = null;
-    }
-
-    /**
-     * @param id Id.
-     * @param t Timestamp.
-     * @param str String.
-     * @param x X.
-     * @return New row.
-     */
-    private GridH2Row row(UUID id, long t, String str, long x) {
-        return GridH2RowFactory.create(
-            ValueUuid.get(id.getMostSignificantBits(), id.getLeastSignificantBits()),
-            ValueTimestamp.get(new Timestamp(t)),
-            ValueString.get(str),
-            ValueLong.get(x));
-    }
-
-
-    /**
-     * Simple table test.
-     *
-     * @throws Exception If failed.
-     */
-    public void testTable() throws Exception {
-        // Test insert.
-        long x = MAX_X;
-
-        Random rnd = new Random();
-
-        while(x-- > 0) {
-            UUID id = UUID.randomUUID();
-
-            GridH2Row row = row(id, System.currentTimeMillis(), rnd.nextBoolean() ? id.toString() :
-                UUID.randomUUID().toString(), rnd.nextInt(100));
-
-            tbl.doUpdate(row, false);
-        }
-
-        assertEquals(MAX_X, tbl.getRowCountApproximation());
-        assertEquals(MAX_X, tbl.getRowCount(null));
-
-        for (GridH2IndexBase idx : tbl.indexes()) {
-            assertEquals(MAX_X, idx.getRowCountApproximation());
-            assertEquals(MAX_X, idx.getRowCount(null));
-        }
-
-        // Check unique index.
-        UUID id = UUID.randomUUID();
-        UUID id2 = UUID.randomUUID();
-
-        assertTrue(tbl.doUpdate(row(id, System.currentTimeMillis(), id.toString(), rnd.nextInt(100)), false));
-        assertTrue(tbl.doUpdate(row(id2, System.currentTimeMillis(), id2.toString(), rnd.nextInt(100)), false));
-
-        // Check index selection.
-        checkQueryPlan(conn, "SELECT * FROM T", SCAN_IDX_NAME);
-
-        checkQueryPlan(conn, "SELECT * FROM T WHERE ID IS NULL", PK_NAME);
-        checkQueryPlan(conn, "SELECT * FROM T WHERE ID = RANDOM_UUID()", PK_NAME);
-        checkQueryPlan(conn, "SELECT * FROM T WHERE ID > RANDOM_UUID()", PK_NAME);
-        checkQueryPlan(conn, "SELECT * FROM T ORDER BY ID", PK_NAME);
-
-        checkQueryPlan(conn, "SELECT * FROM T WHERE STR IS NULL", STR_IDX_NAME);
-        checkQueryPlan(conn, "SELECT * FROM T WHERE STR = 'aaaa'", STR_IDX_NAME);
-        checkQueryPlan(conn, "SELECT * FROM T WHERE STR > 'aaaa'", STR_IDX_NAME);
-        checkQueryPlan(conn, "SELECT * FROM T ORDER BY STR DESC", STR_IDX_NAME);
-
-        checkQueryPlan(conn, "SELECT * FROM T WHERE X IS NULL", NON_UNIQUE_IDX_NAME);
-        checkQueryPlan(conn, "SELECT * FROM T WHERE X = 10000", NON_UNIQUE_IDX_NAME);
-        checkQueryPlan(conn, "SELECT * FROM T WHERE X > 10000", NON_UNIQUE_IDX_NAME);
-        checkQueryPlan(conn, "SELECT * FROM T ORDER BY X DESC", NON_UNIQUE_IDX_NAME);
-        checkQueryPlan(conn, "SELECT * FROM T ORDER BY X DESC, T", NON_UNIQUE_IDX_NAME);
-
-        checkQueryPlan(conn, "SELECT * FROM T ORDER BY T, X DESC", SCAN_IDX_NAME);
-
-        // Simple queries.
-
-        Statement s = conn.createStatement();
-
-        ResultSet rs = s.executeQuery("select id from t where x between 0 and 100");
-
-        int i = 0;
-        while (rs.next())
-            i++;
-
-        assertEquals(MAX_X + 2, i);
-
-        // -----
-
-        rs = s.executeQuery("select id from t where t is not null");
-
-        i = 0;
-        while (rs.next())
-            i++;
-
-        assertEquals(MAX_X + 2, i);
-
-        // ----
-
-        int cnt = 10 + rnd.nextInt(25);
-
-        long t = System.currentTimeMillis();
-
-        for (i = 0; i < cnt; i++) {
-            id = UUID.randomUUID();
-
-            assertTrue(tbl.doUpdate(row(id, t, id.toString(), 51), false));
-        }
-
-        rs = s.executeQuery("select x, id from t where x = 51 limit " + cnt);
-
-        i = 0;
-
-        while (rs.next()) {
-            assertEquals(51, rs.getInt(1));
-
-            i++;
-        }
-
-        assertEquals(cnt, i);
-    }
-
-    /**
-      * @throws Exception If failed.
-     */
-    public void testRangeQuery() throws Exception {
-        int rows = 3000;
-        int xs = 37;
-
-        long t = System.currentTimeMillis();
-
-        Random rnd = new Random();
-
-        for (int i = 0 ; i < rows; i++) {
-            UUID id = UUID.randomUUID();
-
-            GridH2Row row = row(id, t++, id.toString(), rnd.nextInt(xs));
-
-            assertTrue(tbl.doUpdate(row, false));
-        }
-
-        PreparedStatement ps = conn.prepareStatement("select count(*) from t where x = ?");
-
-        int cnt = 0;
-
-        for (int x = 0; x < xs; x++) {
-            ps.setInt(1, x);
-
-            ResultSet rs = ps.executeQuery();
-
-            assertTrue(rs.next());
-
-            cnt += rs.getInt(1);
-        }
-
-        assertEquals(rows, cnt);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDataLoss() throws Exception {
-        final int threads = 37;
-        final int iterations = 15000;
-
-        final AtomicInteger cntr = new AtomicInteger();
-
-        final UUID[] ids = new UUID[threads * iterations];
-
-        for (int i = 0; i < ids.length; i++)
-            ids[i] = UUID.randomUUID();
-
-        final long t = System.currentTimeMillis();
-
-        final AtomicInteger deleted = new AtomicInteger();
-
-        multithreaded(new Callable<Void>() {
-            @Override public Void call() throws Exception {
-                Random rnd = new Random();
-
-                int offset = cntr.getAndIncrement() * iterations;
-
-                synchronized (ids[offset]) {
-                    for (int i = 0; i < iterations; i++) {
-                        UUID id = ids[offset + i];
-
-                        int x = rnd.nextInt(50);
-
-                        GridH2Row row = row(id, t, id.toString(), x);
-
-                        assertTrue(tbl.doUpdate(row, false));
-                    }
-                }
-
-                offset = (offset + iterations) % ids.length;
-
-                synchronized (ids[offset]) {
-                    for (int i = 0; i < iterations; i += 2) {
-                        UUID id = ids[offset + i];
-
-                        int x = rnd.nextInt(50);
-
-                        GridH2Row row = row(id, t, id.toString(), x);
-
-                        if (tbl.doUpdate(row, true))
-                            deleted.incrementAndGet();
-                    }
-                }
-
-                return null;
-            }
-        }, threads);
-
-        assertTrue(deleted.get() > 0);
-
-        PreparedStatement p = conn.prepareStatement("select count(*) from t where id = ?");
-
-        for (int i = 1; i < ids.length; i += 2) {
-            p.setObject(1, ids[i]);
-
-            ResultSet rs = p.executeQuery();
-
-            assertTrue(rs.next());
-
-            assertEquals(1, rs.getInt(1));
-        }
-
-        Statement s = conn.createStatement();
-
-        ResultSet rs = s.executeQuery("select count(*) from t");
-
-        assertTrue(rs.next());
-
-        assertEquals(ids.length - deleted.get(), rs.getInt(1));
-    }
-
-
-    /**
-     * Check query plan to correctly select index.
-     *
-     * @param conn Connection.
-     * @param sql Select.
-     * @param search Search token in result.
-     * @throws SQLException If failed.
-     */
-    private void checkQueryPlan(Connection conn, String sql, String search) throws SQLException {
-
-        try (Statement s = conn.createStatement()) {
-            try (ResultSet r = s.executeQuery("EXPLAIN ANALYZE " + sql)) {
-                assertTrue(r.next());
-
-                String plan = r.getString(1);
-
-                assertTrue("Execution plan for '" + sql + "' query should contain '" + search + "'",
-                        plan.contains(search));
-            }
-        }
-    }
-}
\ No newline at end of file


[6/7] ignite git commit: IGNITE-6562: Dynamic service deployment should use projection if NodeFilter is not set. This closes #2810.

Posted by sb...@apache.org.
IGNITE-6562: Dynamic service deployment should use projection if NodeFilter is not set. This closes #2810.

Signed-off-by: nikolay_tikhonov <nt...@gridgain.com>


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

Branch: refs/heads/ignite-3478
Commit: 6679b6cbe6a26f8e9ba2a02bcf56801811e99abd
Parents: 0b30404
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Mon Oct 16 14:35:21 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Oct 16 14:35:21 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/IgniteServicesImpl.java     |  4 +-
 .../service/GridServiceProcessor.java           | 59 +++++++++-------
 .../GridServiceProcessorMultiNodeSelfTest.java  | 71 +++++++++++++++++++-
 3 files changed, 105 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6679b6cb/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java
index 00d6078..7cbd4b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java
@@ -235,7 +235,7 @@ public class IgniteServicesImpl extends AsyncSupportAdapter implements IgniteSer
         guard();
 
         try {
-            saveOrGet(ctx.service().deployAll(cfgs));
+            saveOrGet(ctx.service().deployAll(prj, cfgs));
         }
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
@@ -252,7 +252,7 @@ public class IgniteServicesImpl extends AsyncSupportAdapter implements IgniteSer
         guard();
 
         try {
-            return (IgniteFuture<Void>)new IgniteFutureImpl<>(ctx.service().deployAll(cfgs));
+            return (IgniteFuture<Void>)new IgniteFutureImpl<>(ctx.service().deployAll(prj, cfgs));
         }
         finally {
             unguard();

http://git-wip-us.apache.org/repos/asf/ignite/blob/6679b6cb/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 6f1dfc7..7097735 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -89,6 +89,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.security.SecurityException;
@@ -263,15 +264,8 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
 
         ServiceConfiguration[] cfgs = ctx.config().getServiceConfiguration();
 
-        if (cfgs != null) {
-            for (ServiceConfiguration c : cfgs) {
-                // Deploy only on server nodes by default.
-                if (c.getNodeFilter() == null)
-                    c.setNodeFilter(ctx.cluster().get().forServers().predicate());
-            }
-
-            deployAll(Arrays.asList(cfgs)).get();
-        }
+        if (cfgs != null)
+            deployAll(Arrays.asList(cfgs), ctx.cluster().get().forServers().predicate()).get();
 
         if (log.isDebugEnabled())
             log.debug("Started service processor.");
@@ -474,9 +468,8 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
         cfg.setService(svc);
         cfg.setTotalCount(totalCnt);
         cfg.setMaxPerNodeCount(maxPerNodeCnt);
-        cfg.setNodeFilter(F.<ClusterNode>alwaysTrue() == prj.predicate() ? null : prj.predicate());
 
-        return deploy(cfg);
+        return deployAll(prj, Collections.singleton(cfg));
     }
 
     /**
@@ -499,14 +492,17 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
         cfg.setTotalCount(1);
         cfg.setMaxPerNodeCount(1);
 
-        return deploy(cfg);
+        // Ignore projection here.
+        return deployAll(Collections.singleton(cfg), null);
     }
 
     /**
      * @param cfgs Service configurations.
+     * @param dfltNodeFilter Default NodeFilter.
      * @return Configurations to deploy.
      */
-    private PreparedConfigurations prepareServiceConfigurations(Collection<ServiceConfiguration> cfgs) {
+    private PreparedConfigurations prepareServiceConfigurations(Collection<ServiceConfiguration> cfgs,
+        IgnitePredicate<ClusterNode> dfltNodeFilter) {
         List<ServiceConfiguration> cfgsCp = new ArrayList<>(cfgs.size());
 
         Marshaller marsh = ctx.config().getMarshaller();
@@ -516,6 +512,11 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
         for (ServiceConfiguration cfg : cfgs) {
             Exception err = null;
 
+            // Deploy to projection node by default
+            // or only on server nodes if no projection .
+            if (cfg.getNodeFilter() == null && dfltNodeFilter != null)
+                cfg.setNodeFilter(dfltNodeFilter);
+
             try {
                 validate(cfg);
             }
@@ -568,13 +569,31 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
     }
 
     /**
+     * @param prj Grid projection.
      * @param cfgs Service configurations.
      * @return Future for deployment.
      */
-    public IgniteInternalFuture<?> deployAll(Collection<ServiceConfiguration> cfgs) {
+    public IgniteInternalFuture<?> deployAll(ClusterGroup prj, Collection<ServiceConfiguration> cfgs) {
+        if (prj == null)
+            // Deploy to servers by default if no projection specified.
+            return deployAll(cfgs,  ctx.cluster().get().forServers().predicate());
+        else if (prj.predicate() == F.<ClusterNode>alwaysTrue())
+            return deployAll(cfgs,  null);
+        else
+            // Deploy to predicate nodes by default.
+            return deployAll(cfgs,  prj.predicate());
+    }
+
+    /**
+     * @param cfgs Service configurations.
+     * @param dfltNodeFilter Default NodeFilter.
+     * @return Future for deployment.
+     */
+    private IgniteInternalFuture<?> deployAll(Collection<ServiceConfiguration> cfgs,
+        @Nullable IgnitePredicate<ClusterNode> dfltNodeFilter) {
         assert cfgs != null;
 
-        PreparedConfigurations srvCfg = prepareServiceConfigurations(cfgs);
+        PreparedConfigurations srvCfg = prepareServiceConfigurations(cfgs, dfltNodeFilter);
 
         List<ServiceConfiguration> cfgsCp = srvCfg.cfgs;
 
@@ -733,16 +752,6 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
     }
 
     /**
-     * @param cfg Service configuration.
-     * @return Future for deployment.
-     */
-    public IgniteInternalFuture<?> deploy(ServiceConfiguration cfg) {
-        A.notNull(cfg, "cfg");
-
-        return deployAll(Collections.singleton(cfg));
-    }
-
-    /**
      * @param name Service name.
      * @return Future.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6679b6cb/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java
index df7ddf1..517f061 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java
@@ -169,7 +169,7 @@ public class GridServiceProcessorMultiNodeSelfTest extends GridServiceProcessorA
                 // Since we start extra nodes, there may be extra start and cancel events,
                 // so we check only the difference between start and cancel and
                 // not start and cancel events individually.
-                assertEquals(name, nodeCount() + servers,  DummyService.started(name) - DummyService.cancelled(name));
+                assertEquals(name, nodeCount() + servers, DummyService.started(name) - DummyService.cancelled(name));
 
                 checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers);
             }
@@ -185,6 +185,73 @@ public class GridServiceProcessorMultiNodeSelfTest extends GridServiceProcessorA
     /**
      * @throws Exception If failed.
      */
+    public void testDeployOnEachProjectionNodeUpdateTopology() throws Exception {
+        // Prestart client node.
+        Ignite client = startGrid("client", getConfiguration("client").setClientMode(true));
+
+        try {
+            final String name = "serviceOnEachProjectionNodeUpdateTopology";
+
+            Ignite g = randomGrid();
+
+            int prestartedSrvcs = 1;
+
+            CountDownLatch latch = new CountDownLatch(prestartedSrvcs);
+
+            DummyService.exeLatch(name, latch);
+
+            IgniteServices svcs = g.services(g.cluster().forClients());
+
+            IgniteFuture<?> fut = svcs.deployNodeSingletonAsync(name, new DummyService());
+
+            info("Deployed service: " + name);
+
+            fut.get();
+
+            info("Finished waiting for service future: " + name);
+
+            latch.await();
+
+            // Ensure service is deployed
+            assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000));
+
+            assertEquals(name, prestartedSrvcs, DummyService.started(name));
+            assertEquals(name, 0, DummyService.cancelled(name));
+
+            int servers = 2;
+
+            int clients = 2;
+
+            latch = new CountDownLatch(clients);
+
+            DummyService.exeLatch(name, latch);
+
+            startExtraNodes(servers, clients);
+
+            try {
+                latch.await();
+
+                waitForDeployment(name, clients);
+
+                // Since we start extra nodes, there may be extra start and cancel events,
+                // so we check only the difference between start and cancel and
+                // not start and cancel events individually.
+                assertEquals(name, clients + prestartedSrvcs, DummyService.started(name) - DummyService.cancelled(name));
+
+                checkCount(name, g.services().serviceDescriptors(), clients + prestartedSrvcs);
+            }
+            finally {
+                stopExtraNodes(servers + clients);
+            }
+        }
+        finally {
+            stopGrid("client");
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testDeployOnEachNodeUpdateTopology() throws Exception {
         // Prestart client node.
         Ignite client = startGrid("client", getConfiguration("client").setClientMode(true));
@@ -315,7 +382,7 @@ public class GridServiceProcessorMultiNodeSelfTest extends GridServiceProcessorA
             // Since we start extra nodes, there may be extra start and cancel events,
             // so we check only the difference between start and cancel and
             // not start and cancel events individually.
-            assertEquals(name, totalInstances,  DummyService.started(name) - DummyService.cancelled(name));
+            assertEquals(name, totalInstances, DummyService.started(name) - DummyService.cancelled(name));
 
             checkCount(name, g.services().serviceDescriptors(), totalInstances);
         }


[3/7] ignite git commit: IGNITE-6631: Minor improvements to GridH2KeyValueRowOnheap. This closes #2855.

Posted by sb...@apache.org.
IGNITE-6631: Minor improvements to GridH2KeyValueRowOnheap. This closes #2855.


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

Branch: refs/heads/ignite-3478
Commit: 583e34615bb58acf7fffeaa3e815358d7c033784
Parents: 5e1a22d
Author: devozerov <vo...@gridgain.com>
Authored: Mon Oct 16 10:33:36 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 16 10:33:36 2017 +0300

----------------------------------------------------------------------
 .../processors/query/h2/H2RowDescriptor.java    | 431 -------------------
 .../processors/query/h2/H2TableDescriptor.java  |   2 +-
 .../processors/query/h2/H2TableEngine.java      |   5 +-
 .../internal/processors/query/h2/H2Utils.java   |   3 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   9 +-
 .../query/h2/database/H2PkHashIndex.java        |   7 +-
 .../query/h2/database/H2RowFactory.java         |   6 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   | 147 +++----
 .../query/h2/opt/GridH2RowDescriptor.java       | 412 ++++++++++++++++--
 .../processors/query/h2/opt/GridH2Table.java    |  28 +-
 10 files changed, 473 insertions(+), 577 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/583e3461/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java
deleted file mode 100644
index 8fb81ba..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java
+++ /dev/null
@@ -1,431 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.h2;
-
-import java.math.BigDecimal;
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.Arrays;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.query.GridQueryProperty;
-import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowFactory;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
-import org.h2.message.DbException;
-import org.h2.result.SearchRow;
-import org.h2.result.SimpleRow;
-import org.h2.util.LocalDateTimeUtils;
-import org.h2.value.DataType;
-import org.h2.value.Value;
-import org.h2.value.ValueArray;
-import org.h2.value.ValueBoolean;
-import org.h2.value.ValueByte;
-import org.h2.value.ValueBytes;
-import org.h2.value.ValueDate;
-import org.h2.value.ValueDecimal;
-import org.h2.value.ValueDouble;
-import org.h2.value.ValueFloat;
-import org.h2.value.ValueGeometry;
-import org.h2.value.ValueInt;
-import org.h2.value.ValueJavaObject;
-import org.h2.value.ValueLong;
-import org.h2.value.ValueNull;
-import org.h2.value.ValueShort;
-import org.h2.value.ValueString;
-import org.h2.value.ValueTime;
-import org.h2.value.ValueTimestamp;
-import org.h2.value.ValueUuid;
-import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.KEY_COL;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.VAL_COL;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.VER_COL;
-
-/**
- * Row descriptor.
- */
-public class H2RowDescriptor implements GridH2RowDescriptor {
-    /** Indexing SPI. */
-    private final IgniteH2Indexing idx;
-
-    /** Table descriptor. */
-    private final H2TableDescriptor tbl;
-
-    /** */
-    private final GridQueryTypeDescriptor type;
-
-    /** */
-    private volatile String[] fields;
-
-    /** */
-    private volatile int[] fieldTypes;
-
-    /** */
-    private final int keyType;
-
-    /** */
-    private final int valType;
-
-    /** */
-    private volatile GridQueryProperty[] props;
-
-    /** Id of user-defined key column */
-    private volatile int keyAliasColId;
-
-    /** Id of user-defined value column */
-    private volatile int valAliasColId;
-
-    /**
-     * Constructor.
-     *
-     * @param idx Indexing.
-     * @param tbl Table.
-     * @param type Type descriptor.
-     */
-    H2RowDescriptor(IgniteH2Indexing idx, H2TableDescriptor tbl, GridQueryTypeDescriptor type) {
-        assert type != null;
-
-        this.idx = idx;
-        this.tbl = tbl;
-        this.type = type;
-
-        keyType = DataType.getTypeFromClass(type.keyClass());
-        valType = DataType.getTypeFromClass(type.valueClass());
-
-        refreshMetadataFromTypeDescriptor();
-    }
-
-    /**
-     * Update metadata of this row descriptor according to current state of type descriptor.
-     */
-    @SuppressWarnings("WeakerAccess")
-    public final void refreshMetadataFromTypeDescriptor() {
-        Map<String, Class<?>> allFields = new LinkedHashMap<>();
-
-        allFields.putAll(type.fields());
-
-        fields = allFields.keySet().toArray(new String[allFields.size()]);
-
-        fieldTypes = new int[fields.length];
-
-        Class[] classes = allFields.values().toArray(new Class[fields.length]);
-
-        for (int i = 0; i < fieldTypes.length; i++)
-            fieldTypes[i] = DataType.getTypeFromClass(classes[i]);
-
-        props = new GridQueryProperty[fields.length];
-
-        for (int i = 0; i < fields.length; i++) {
-            GridQueryProperty p = type.property(fields[i]);
-
-            assert p != null : fields[i];
-
-            props[i] = p;
-        }
-
-        List<String> fieldsList = Arrays.asList(fields);
-
-        keyAliasColId =
-            (type.keyFieldName() != null) ? DEFAULT_COLUMNS_COUNT + fieldsList.indexOf(type.keyFieldAlias()) : -1;
-
-        valAliasColId =
-            (type.valueFieldName() != null) ? DEFAULT_COLUMNS_COUNT + fieldsList.indexOf(type.valueFieldAlias()) : -1;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteH2Indexing indexing() {
-        return idx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridQueryTypeDescriptor type() {
-        return type;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheContext<?, ?> context() {
-        return tbl.cache();
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("ConstantConditions")
-    @Override public Value wrap(Object obj, int type) throws IgniteCheckedException {
-        assert obj != null;
-
-        if (obj instanceof CacheObject) { // Handle cache object.
-            CacheObject co = (CacheObject)obj;
-
-            if (type == Value.JAVA_OBJECT)
-                return new GridH2ValueCacheObject(co, idx.objectContext());
-
-            obj = co.value(idx.objectContext(), false);
-        }
-
-        switch (type) {
-            case Value.BOOLEAN:
-                return ValueBoolean.get((Boolean)obj);
-            case Value.BYTE:
-                return ValueByte.get((Byte)obj);
-            case Value.SHORT:
-                return ValueShort.get((Short)obj);
-            case Value.INT:
-                return ValueInt.get((Integer)obj);
-            case Value.FLOAT:
-                return ValueFloat.get((Float)obj);
-            case Value.LONG:
-                return ValueLong.get((Long)obj);
-            case Value.DOUBLE:
-                return ValueDouble.get((Double)obj);
-            case Value.UUID:
-                UUID uuid = (UUID)obj;
-                return ValueUuid.get(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits());
-            case Value.DATE:
-                if (LocalDateTimeUtils.isLocalDate(obj.getClass()))
-                    return LocalDateTimeUtils.localDateToDateValue(obj);
-
-                return ValueDate.get((Date)obj);
-
-            case Value.TIME:
-                if (LocalDateTimeUtils.isLocalTime(obj.getClass()))
-                    return LocalDateTimeUtils.localTimeToTimeValue(obj);
-
-                return ValueTime.get((Time)obj);
-
-            case Value.TIMESTAMP:
-                if (obj instanceof java.util.Date && !(obj instanceof Timestamp))
-                    obj = new Timestamp(((java.util.Date)obj).getTime());
-
-                if (LocalDateTimeUtils.isLocalDateTime(obj.getClass()))
-                    return LocalDateTimeUtils.localDateTimeToValue(obj);
-
-                return ValueTimestamp.get((Timestamp)obj);
-
-            case Value.DECIMAL:
-                return ValueDecimal.get((BigDecimal)obj);
-            case Value.STRING:
-                return ValueString.get(obj.toString());
-            case Value.BYTES:
-                return ValueBytes.get((byte[])obj);
-            case Value.JAVA_OBJECT:
-                return ValueJavaObject.getNoCopy(obj, null, null);
-            case Value.ARRAY:
-                Object[] arr = (Object[])obj;
-
-                Value[] valArr = new Value[arr.length];
-
-                for (int i = 0; i < arr.length; i++) {
-                    Object o = arr[i];
-
-                    valArr[i] = o == null ? ValueNull.INSTANCE : wrap(o, DataType.getTypeFromClass(o.getClass()));
-                }
-
-                return ValueArray.get(valArr);
-
-            case Value.GEOMETRY:
-                return ValueGeometry.getFromGeometry(obj);
-        }
-
-        throw new IgniteCheckedException("Failed to wrap value[type=" + type + ", value=" + obj + "]");
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridH2Row createRow(KeyCacheObject key, int partId, @Nullable CacheObject val,
-        GridCacheVersion ver, long expirationTime) throws IgniteCheckedException {
-        GridH2Row row;
-
-        try {
-            if (val == null) // Only can happen for remove operation, can create simple search row.
-                row = GridH2RowFactory.create(wrap(key, keyType));
-            else
-                row = new GridH2KeyValueRowOnheap(this, key, keyType, val, valType, ver, expirationTime);
-        }
-        catch (ClassCastException e) {
-            throw new IgniteCheckedException("Failed to convert key to SQL type. " +
-                "Please make sure that you always store each value type with the same key type " +
-                "or configure key type as common super class for all actual keys for this value type.", e);
-        }
-
-        row.ver = ver;
-
-        row.key = key;
-        row.val = val;
-        row.partId = partId;
-
-        return row;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int valueType() {
-        return valType;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int fieldsCount() {
-        return fields.length;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int fieldType(int col) {
-        return fieldTypes[col];
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object columnValue(Object key, Object val, int col) {
-        try {
-            return props[col].value(key, val);
-        }
-        catch (IgniteCheckedException e) {
-            throw DbException.convert(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setColumnValue(Object key, Object val, Object colVal, int col) {
-        try {
-            props[col].setValue(key, val, colVal);
-        }
-        catch (IgniteCheckedException e) {
-            throw DbException.convert(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isColumnKeyProperty(int col) {
-        return props[col].key();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isKeyColumn(int colId) {
-        assert colId >= 0;
-        return colId == KEY_COL || colId == keyAliasColId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isValueColumn(int colId) {
-        assert colId >= 0;
-        return colId == VAL_COL || colId == valAliasColId;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("RedundantIfStatement")
-    @Override public boolean isKeyValueOrVersionColumn(int colId) {
-        assert colId >= 0;
-
-        if (colId < DEFAULT_COLUMNS_COUNT)
-            return true;
-
-        if (colId == keyAliasColId)
-            return true;
-
-        if (colId == valAliasColId)
-            return true;
-
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean checkKeyIndexCondition(int masks[], int mask) {
-        assert masks != null;
-        assert masks.length > 0;
-
-        if (keyAliasColId < 0)
-            return (masks[KEY_COL] & mask) != 0;
-        else
-            return (masks[KEY_COL] & mask) != 0 || (masks[keyAliasColId] & mask) != 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void initValueCache(Value valCache[], Value key, Value val, Value ver) {
-        assert valCache != null;
-        assert valCache.length > 0;
-
-        valCache[KEY_COL] = key;
-        valCache[VAL_COL] = val;
-        valCache[VER_COL] = ver;
-
-        if (keyAliasColId > 0)
-            valCache[keyAliasColId] = key;
-
-        if (valAliasColId > 0)
-            valCache[valAliasColId] = val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public SearchRow prepareProxyIndexRow(SearchRow row) {
-        if (row == null)
-            return null;
-
-        Value[] data = new Value[row.getColumnCount()];
-        for (int idx = 0; idx < data.length; idx++)
-            data[idx] = row.getValue(idx);
-
-        copyAliasColumnData(data, KEY_COL, keyAliasColId);
-        copyAliasColumnData(data, VAL_COL, valAliasColId);
-
-        return new SimpleRow(data);
-    }
-
-    /**
-     * Copies data between original and alias columns
-     *
-     * @param data Array of values.
-     * @param colId Original column id.
-     * @param aliasColId Alias column id.
-     */
-    private void copyAliasColumnData(Value[] data, int colId, int aliasColId) {
-        if (aliasColId <= 0)
-            return;
-
-        if (data[aliasColId] == null && data[colId] != null)
-            data[aliasColId] = data[colId];
-
-        if (data[colId] == null && data[aliasColId] != null)
-            data[colId] = data[aliasColId];
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getAlternativeColumnId(int colId) {
-        if (keyAliasColId > 0) {
-            if (colId == KEY_COL)
-                return keyAliasColId;
-            else if (colId == keyAliasColId)
-                return KEY_COL;
-        }
-        if (valAliasColId > 0) {
-            if (colId == VAL_COL)
-                return valAliasColId;
-            else if (colId == valAliasColId)
-                return VAL_COL;
-        }
-
-        return colId;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/583e3461/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
index 391b002..899bdda 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
@@ -141,7 +141,7 @@ public class H2TableDescriptor implements GridH2SystemIndexFactory {
     /**
      * @return Cache context.
      */
-    GridCacheContext cache() {
+    public GridCacheContext cache() {
         return cctx;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/583e3461/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java
index d3e9560..c05aaf6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java
@@ -21,6 +21,7 @@ import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.Statement;
 import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.h2.api.TableEngine;
 import org.h2.command.ddl.CreateTableData;
@@ -31,7 +32,7 @@ import org.h2.table.TableBase;
  */
 public class H2TableEngine implements TableEngine {
     /** */
-    private static H2RowDescriptor rowDesc0;
+    private static GridH2RowDescriptor rowDesc0;
 
     /** */
     private static H2RowFactory rowFactory0;
@@ -53,7 +54,7 @@ public class H2TableEngine implements TableEngine {
      * @throws SQLException If failed.
      * @return Created table.
      */
-    public static synchronized GridH2Table createTable(Connection conn, String sql, H2RowDescriptor rowDesc,
+    public static synchronized GridH2Table createTable(Connection conn, String sql, GridH2RowDescriptor rowDesc,
         H2RowFactory rowFactory, H2TableDescriptor tblDesc)
         throws SQLException {
         rowDesc0 = rowDesc;

http://git-wip-us.apache.org/repos/asf/ignite/blob/583e3461/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 157e1ba..cfbb7bb 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
@@ -247,8 +247,7 @@ public class H2Utils {
      * @return Converted object.
      * @throws IgniteCheckedException if failed.
      */
-    public static Object convert(Object val, GridH2RowDescriptor desc, int type)
-        throws IgniteCheckedException {
+    public static Object convert(Object val, GridH2RowDescriptor desc, int type) throws IgniteCheckedException {
         if (val == null)
             return null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/583e3461/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 c172e65..dd35723 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
@@ -675,9 +675,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     if (expTime == 0L)
                         expTime = Long.MAX_VALUE;
 
-                    GridH2Row row = rowDesc.createRow(key, part, val, ver, expTime);
-
-                    row.link(link);
+                    GridH2Row row = rowDesc.createRow(key, part, val, ver, expTime, link);
 
                     h2Idx.put(row);
                 }
@@ -1775,7 +1773,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (log.isDebugEnabled())
             log.debug("Creating DB table with SQL: " + sql);
 
-        H2RowDescriptor rowDesc = new H2RowDescriptor(this, tbl, tbl.type());
+        GridH2RowDescriptor rowDesc = new GridH2RowDescriptor(this, tbl, tbl.type());
 
         H2RowFactory rowFactory = tbl.rowFactory(rowDesc);
 
@@ -2450,8 +2448,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         assert partInfo != null;
         assert partInfo.partition() < 0;
 
-        GridH2RowDescriptor desc = dataTable(schema(partInfo.cacheName()),
-                partInfo.tableName()).rowDescriptor();
+        GridH2RowDescriptor desc = dataTable(schema(partInfo.cacheName()), partInfo.tableName()).rowDescriptor();
 
         Object param = H2Utils.convert(params[partInfo.paramIdx()],
                 desc, partInfo.dataType());

http://git-wip-us.apache.org/repos/asf/ignite/blob/583e3461/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
index 6691485..891e59f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
@@ -197,11 +197,8 @@ public class H2PkHashIndex extends GridH2IndexBase {
             try {
                 CacheDataRow dataRow = cursor.get();
 
-                GridH2Row row = tbl.rowDescriptor().createRow(dataRow.key(), dataRow.partition(), dataRow.value(), dataRow.version(), 0);
-
-                row.link(dataRow.link());
-
-                return row;
+                return tbl.rowDescriptor().createRow(dataRow.key(), dataRow.partition(), dataRow.value(),
+                    dataRow.version(), 0, dataRow.link());
             }
             catch (IgniteCheckedException e) {
                 throw DbException.convert(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/583e3461/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java
index 2e57ca3..92ecd3d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java
@@ -22,8 +22,8 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
 
 /**
  * Data store for H2 rows.
@@ -65,9 +65,7 @@ public class H2RowFactory {
 
         try {
             row = rowDesc.createRow(rowBuilder.key(),
-                PageIdUtils.partId(link), rowBuilder.value(), rowBuilder.version(), rowBuilder.expireTime());
-
-            row.link = link;
+                PageIdUtils.partId(link), rowBuilder.value(), rowBuilder.version(), rowBuilder.expireTime(), link);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/583e3461/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java
index 390015b..63b4606 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java
@@ -61,7 +61,7 @@ public class GridH2KeyValueRowOnheap extends GridH2Row {
     private Value[] valCache;
 
     /** */
-    private Value version;
+    private Value ver;
 
     /**
      * Constructor.
@@ -79,13 +79,13 @@ public class GridH2KeyValueRowOnheap extends GridH2Row {
         this.desc = desc;
         this.expirationTime = expirationTime;
 
-        setValue(KEY_COL, desc.wrap(key, keyType));
+        this.key = desc.wrap(key, keyType);
 
-        if (val != null) // We remove by key only, so value can be null here.
-            setValue(VAL_COL, desc.wrap(val, valType));
+        if (val != null)
+            this.val = desc.wrap(val, valType);
 
         if (ver != null)
-            setValue(VER_COL, desc.wrap(ver, Value.JAVA_OBJECT));
+            this.ver = desc.wrap(ver, Value.JAVA_OBJECT);
     }
 
     /** {@inheritDoc} */
@@ -103,59 +103,39 @@ public class GridH2KeyValueRowOnheap extends GridH2Row {
         return DEFAULT_COLUMNS_COUNT + desc.fieldsCount();
     }
 
-    /**
-     * @param col Column index.
-     * @return Value if exists.
-     */
-    protected final Value peekValue(int col) {
-        if (col == KEY_COL)
-            return key;
-
-        if (col == VAL_COL)
-            return val;
-
-        assert col == VER_COL;
-
-        return version;
-    }
-
     /** {@inheritDoc} */
     @Override public Value getValue(int col) {
-        Value[] vCache = valCache;
-
-        if (vCache != null) {
-            Value v = vCache[col];
+        switch (col) {
+            case KEY_COL:
+                return key;
 
-            if (v != null)
-                return v;
-        }
+            case VAL_COL:
+                return val;
 
-        Value v;
+            case VER_COL:
+                return ver;
 
-        if (desc.isValueColumn(col)) {
-            v = peekValue(VAL_COL);
+            default:
+                if (desc.isKeyAliasColumn(col))
+                    return key;
+                else if (desc.isValueAliasColumn(col))
+                    return val;
 
-            return v;
+                return getValue0(col - DEFAULT_COLUMNS_COUNT);
         }
-        else if (desc.isKeyColumn(col)) {
-            v = peekValue(KEY_COL);
+    }
 
-            assert v != null;
+    /**
+     * Get real column value.
+     *
+     * @param col Adjusted column index (without default columns).
+     * @return Value.
+     */
+    private Value getValue0(int col) {
+        Value v = getCached(col);
 
+        if (v != null)
             return v;
-        }
-        else if (col == VER_COL)
-            return version;
-
-        col -= DEFAULT_COLUMNS_COUNT;
-
-        assert col >= 0;
-
-        Value key = getValue(KEY_COL);
-        Value val = getValue(VAL_COL);
-
-        assert key != null;
-        assert val != null;
 
         Object res = desc.columnValue(key.getObject(), val.getObject(), col);
 
@@ -170,21 +150,44 @@ public class GridH2KeyValueRowOnheap extends GridH2Row {
             }
         }
 
-        if (vCache != null)
-            vCache[col + DEFAULT_COLUMNS_COUNT] = v;
+        setCached(col, v);
 
         return v;
     }
 
     /**
-     * @param valCache Value cache.
+     * Prepare values cache.
      */
-    public void valuesCache(Value[] valCache) {
-        if (valCache != null) {
-            desc.initValueCache(valCache, key, val, version);
-        }
+    public void prepareValuesCache() {
+        this.valCache = new Value[desc.fieldsCount()];
+    }
+
+    /**
+     * Clear values cache.
+     */
+    public void clearValuesCache() {
+        this.valCache = null;
+    }
+
+    /**
+     * Get cached value (if any).
+     *
+     * @param colIdx Column index.
+     * @return Value.
+     */
+    private Value getCached(int colIdx) {
+        return valCache != null ? valCache[colIdx] : null;
+    }
 
-        this.valCache = valCache;
+    /**
+     * Set cache value.
+     *
+     * @param colIdx Column index.
+     * @param val Value.
+     */
+    private void setCached(int colIdx, Value val) {
+        if (valCache != null)
+            valCache[colIdx] = val;
     }
 
     /** {@inheritDoc} */
@@ -193,13 +196,13 @@ public class GridH2KeyValueRowOnheap extends GridH2Row {
 
         sb.a(Integer.toHexString(System.identityHashCode(this)));
 
-        Value v = peekValue(KEY_COL);
+        Value v = key;
         sb.a("[ key: ").a(v == null ? "nil" : v.getString());
 
-        v = peekValue(VAL_COL);
+        v = val;
         sb.a(", val: ").a(v == null ? "nil" : v.getString());
 
-        v = peekValue(VER_COL);
+        v = ver;
         sb.a(", ver: ").a(v == null ? "nil" : v.getString());
 
         sb.a(" ][ ");
@@ -223,54 +226,46 @@ public class GridH2KeyValueRowOnheap extends GridH2Row {
 
     /** {@inheritDoc} */
     @Override public void setKeyAndVersion(SearchRow old) {
-        throw new IllegalStateException();
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public void setKey(long key) {
-        throw new IllegalStateException();
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public Row getCopy() {
-        throw new IllegalStateException();
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public void setDeleted(boolean deleted) {
-        throw new IllegalStateException();
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public long getKey() {
-        throw new IllegalStateException();
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public void setSessionId(int sesId) {
-        throw new IllegalStateException();
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public void setVersion(int ver) {
-        throw new IllegalStateException();
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public void setValue(int idx, Value v) {
-        if (desc.isValueColumn(idx))
-            val = v;
-        else if (idx == VER_COL)
-            version = v;
-        else {
-            assert desc.isKeyColumn(idx) : idx + " " + v;
-
-            key = v;
-        }
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public final int hashCode() {
-        throw new IllegalStateException();
+        throw new UnsupportedOperationException();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/583e3461/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
index 1f6ff88..081805e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
@@ -17,41 +17,259 @@
 
 package org.apache.ignite.internal.processors.query.h2.opt;
 
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.h2.H2TableDescriptor;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.h2.message.DbException;
 import org.h2.result.SearchRow;
+import org.h2.result.SimpleRow;
+import org.h2.util.LocalDateTimeUtils;
+import org.h2.value.DataType;
 import org.h2.value.Value;
+import org.h2.value.ValueArray;
+import org.h2.value.ValueBoolean;
+import org.h2.value.ValueByte;
+import org.h2.value.ValueBytes;
+import org.h2.value.ValueDate;
+import org.h2.value.ValueDecimal;
+import org.h2.value.ValueDouble;
+import org.h2.value.ValueFloat;
+import org.h2.value.ValueGeometry;
+import org.h2.value.ValueInt;
+import org.h2.value.ValueJavaObject;
+import org.h2.value.ValueLong;
+import org.h2.value.ValueNull;
+import org.h2.value.ValueShort;
+import org.h2.value.ValueString;
+import org.h2.value.ValueTime;
+import org.h2.value.ValueTimestamp;
+import org.h2.value.ValueUuid;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.KEY_COL;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.VAL_COL;
+
 /**
  * Row descriptor.
  */
-public interface GridH2RowDescriptor {
+public class GridH2RowDescriptor {
+    /** Indexing SPI. */
+    private final IgniteH2Indexing idx;
+
+    /** Table descriptor. */
+    private final H2TableDescriptor tbl;
+
+    /** */
+    private final GridQueryTypeDescriptor type;
+
+    /** */
+    private volatile String[] fields;
+
+    /** */
+    private volatile int[] fieldTypes;
+
+    /** */
+    private final int keyType;
+
+    /** */
+    private final int valType;
+
+    /** */
+    private volatile GridQueryProperty[] props;
+
+    /** Id of user-defined key column */
+    private volatile int keyAliasColId;
+
+    /** Id of user-defined value column */
+    private volatile int valAliasColId;
+
+    /**
+     * Constructor.
+     *
+     * @param idx Indexing.
+     * @param tbl Table.
+     * @param type Type descriptor.
+     */
+    public GridH2RowDescriptor(IgniteH2Indexing idx, H2TableDescriptor tbl, GridQueryTypeDescriptor type) {
+        assert type != null;
+
+        this.idx = idx;
+        this.tbl = tbl;
+        this.type = type;
+
+        keyType = DataType.getTypeFromClass(type.keyClass());
+        valType = DataType.getTypeFromClass(type.valueClass());
+
+        refreshMetadataFromTypeDescriptor();
+    }
+
+    /**
+     * Update metadata of this row descriptor according to current state of type descriptor.
+     */
+    @SuppressWarnings("WeakerAccess")
+    public final void refreshMetadataFromTypeDescriptor() {
+        Map<String, Class<?>> allFields = new LinkedHashMap<>();
+
+        allFields.putAll(type.fields());
+
+        fields = allFields.keySet().toArray(new String[allFields.size()]);
+
+        fieldTypes = new int[fields.length];
+
+        Class[] classes = allFields.values().toArray(new Class[fields.length]);
+
+        for (int i = 0; i < fieldTypes.length; i++)
+            fieldTypes[i] = DataType.getTypeFromClass(classes[i]);
+
+        props = new GridQueryProperty[fields.length];
+
+        for (int i = 0; i < fields.length; i++) {
+            GridQueryProperty p = type.property(fields[i]);
+
+            assert p != null : fields[i];
+
+            props[i] = p;
+        }
+
+        List<String> fieldsList = Arrays.asList(fields);
+
+        keyAliasColId =
+            (type.keyFieldName() != null) ? DEFAULT_COLUMNS_COUNT + fieldsList.indexOf(type.keyFieldAlias()) : -1;
+
+        valAliasColId =
+            (type.valueFieldName() != null) ? DEFAULT_COLUMNS_COUNT + fieldsList.indexOf(type.valueFieldAlias()) : -1;
+    }
+
     /**
      * Gets indexing.
      *
      * @return indexing.
      */
-    public IgniteH2Indexing indexing();
+    public IgniteH2Indexing indexing() {
+        return idx;
+    }
 
     /**
      * Gets type descriptor.
      *
      * @return Type descriptor.
      */
-    public GridQueryTypeDescriptor type();
+    public GridQueryTypeDescriptor type() {
+        return type;
+    }
 
     /**
      * Gets cache context for this row descriptor.
      *
      * @return Cache context.
      */
-    public GridCacheContext<?, ?> context();
+    public GridCacheContext<?, ?> context() {
+        return tbl.cache();
+    }
+
+    /**
+     * Wraps object to respective {@link Value}.
+     *
+     * @param obj Object.
+     * @param type Value type.
+     * @return Value.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    public Value wrap(Object obj, int type) throws IgniteCheckedException {
+        assert obj != null;
+
+        if (obj instanceof CacheObject) { // Handle cache object.
+            CacheObject co = (CacheObject)obj;
+
+            if (type == Value.JAVA_OBJECT)
+                return new GridH2ValueCacheObject(co, idx.objectContext());
+
+            obj = co.value(idx.objectContext(), false);
+        }
+
+        switch (type) {
+            case Value.BOOLEAN:
+                return ValueBoolean.get((Boolean)obj);
+            case Value.BYTE:
+                return ValueByte.get((Byte)obj);
+            case Value.SHORT:
+                return ValueShort.get((Short)obj);
+            case Value.INT:
+                return ValueInt.get((Integer)obj);
+            case Value.FLOAT:
+                return ValueFloat.get((Float)obj);
+            case Value.LONG:
+                return ValueLong.get((Long)obj);
+            case Value.DOUBLE:
+                return ValueDouble.get((Double)obj);
+            case Value.UUID:
+                UUID uuid = (UUID)obj;
+                return ValueUuid.get(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits());
+            case Value.DATE:
+                if (LocalDateTimeUtils.isLocalDate(obj.getClass()))
+                    return LocalDateTimeUtils.localDateToDateValue(obj);
+
+                return ValueDate.get((Date)obj);
+
+            case Value.TIME:
+                if (LocalDateTimeUtils.isLocalTime(obj.getClass()))
+                    return LocalDateTimeUtils.localTimeToTimeValue(obj);
+
+                return ValueTime.get((Time)obj);
+
+            case Value.TIMESTAMP:
+                if (obj instanceof java.util.Date && !(obj instanceof Timestamp))
+                    obj = new Timestamp(((java.util.Date)obj).getTime());
+
+                if (LocalDateTimeUtils.isLocalDateTime(obj.getClass()))
+                    return LocalDateTimeUtils.localDateTimeToValue(obj);
+
+                return ValueTimestamp.get((Timestamp)obj);
+
+            case Value.DECIMAL:
+                return ValueDecimal.get((BigDecimal)obj);
+            case Value.STRING:
+                return ValueString.get(obj.toString());
+            case Value.BYTES:
+                return ValueBytes.get((byte[])obj);
+            case Value.JAVA_OBJECT:
+                return ValueJavaObject.getNoCopy(obj, null, null);
+            case Value.ARRAY:
+                Object[] arr = (Object[])obj;
+
+                Value[] valArr = new Value[arr.length];
+
+                for (int i = 0; i < arr.length; i++) {
+                    Object o = arr[i];
+
+                    valArr[i] = o == null ? ValueNull.INSTANCE : wrap(o, DataType.getTypeFromClass(o.getClass()));
+                }
+
+                return ValueArray.get(valArr);
+
+            case Value.GEOMETRY:
+                return ValueGeometry.getFromGeometry(obj);
+        }
+
+        throw new IgniteCheckedException("Failed to wrap value[type=" + type + ", value=" + obj + "]");
+    }
 
     /**
      * Creates new row.
@@ -63,18 +281,46 @@ public interface GridH2RowDescriptor {
      * @return Row.
      * @throws IgniteCheckedException If failed.
      */
-    public GridH2Row createRow(KeyCacheObject key, int part, @Nullable CacheObject val, GridCacheVersion ver,
-        long expirationTime) throws IgniteCheckedException;
+    public GridH2Row createRow(KeyCacheObject key, int partId, @Nullable CacheObject val, GridCacheVersion ver,
+        long expirationTime, long link) throws IgniteCheckedException {
+        GridH2Row row;
+
+        try {
+            if (val == null) // Only can happen for remove operation, can create simple search row.
+                row = GridH2RowFactory.create(wrap(key, keyType));
+            else
+                row = new GridH2KeyValueRowOnheap(this, key, keyType, val, valType, ver, expirationTime);
+        }
+        catch (ClassCastException e) {
+            throw new IgniteCheckedException("Failed to convert key to SQL type. " +
+                "Please make sure that you always store each value type with the same key type " +
+                "or configure key type as common super class for all actual keys for this value type.", e);
+        }
+
+        row.ver = ver;
+
+        row.key = key;
+        row.val = val;
+        row.partId = partId;
+
+        row.link(link);
+
+        return row;
+    }
 
     /**
      * @return Value type.
      */
-    public int valueType();
+    public int valueType() {
+        return valType;
+    }
 
     /**
      * @return Total fields count.
      */
-    public int fieldsCount();
+    public int fieldsCount() {
+        return fields.length;
+    }
 
     /**
      * Gets value type for column index.
@@ -82,7 +328,9 @@ public interface GridH2RowDescriptor {
      * @param col Column index.
      * @return Value type.
      */
-    public int fieldType(int col);
+    public int fieldType(int col) {
+        return fieldTypes[col];
+    }
 
     /**
      * Gets column value by column index.
@@ -92,7 +340,14 @@ public interface GridH2RowDescriptor {
      * @param col Column index.
      * @return  Column value.
      */
-    public Object columnValue(Object key, Object val, int col);
+    public Object columnValue(Object key, Object val, int col) {
+        try {
+            return props[col].value(key, val);
+        }
+        catch (IgniteCheckedException e) {
+            throw DbException.convert(e);
+        }
+    }
 
     /**
      * Gets column value by column index.
@@ -102,7 +357,14 @@ public interface GridH2RowDescriptor {
      * @param colVal Value to set to column.
      * @param col Column index.
      */
-    public void setColumnValue(Object key, Object val, Object colVal, int col);
+    public void setColumnValue(Object key, Object val, Object colVal, int col) {
+        try {
+            props[col].setValue(key, val, colVal);
+        }
+        catch (IgniteCheckedException e) {
+            throw DbException.convert(e);
+        }
+    }
 
     /**
      * Determine whether a column corresponds to a property of key or to one of value.
@@ -110,25 +372,31 @@ public interface GridH2RowDescriptor {
      * @param col Column index.
      * @return {@code true} if given column corresponds to a key property, {@code false} otherwise
      */
-    public boolean isColumnKeyProperty(int col);
+    public boolean isColumnKeyProperty(int col) {
+        return props[col].key();
+    }
 
     /**
-     * Wraps object to respective {@link Value}.
+     * Checks if provided column id matches key column or key alias.
      *
-     * @param o Object.
-     * @param type Value type.
-     * @return Value.
-     * @throws IgniteCheckedException If failed.
+     * @param colId Column id.
+     * @return Result.
      */
-    public Value wrap(Object o, int type) throws IgniteCheckedException;
+    public boolean isKeyColumn(int colId) {
+        assert colId >= 0;
+        return colId == KEY_COL || colId == keyAliasColId;
+    }
 
     /**
-     * Checks if provided column id matches key column or key alias.
+     * Checks if provided column id matches key alias column.
      *
      * @param colId Column id.
      * @return Result.
      */
-    public boolean isKeyColumn(int colId);
+    public boolean isKeyAliasColumn(int colId) {
+        assert colId >= 0;
+        return colId == keyAliasColId;
+    }
 
     /**
      * Checks if provided column id matches value column or alias.
@@ -136,7 +404,21 @@ public interface GridH2RowDescriptor {
      * @param colId Column id.
      * @return Result.
      */
-    public boolean isValueColumn(int colId);
+    public boolean isValueColumn(int colId) {
+        assert colId >= 0;
+        return colId == VAL_COL || colId == valAliasColId;
+    }
+
+    /**
+     * Checks if provided column id matches value alias column.
+     *
+     * @param colId Column id.
+     * @return Result.
+     */
+    public boolean isValueAliasColumn(int colId) {
+        assert colId >= 0;
+        return colId == valAliasColId;
+    }
 
     /**
      * Checks if provided column id matches key, key alias,
@@ -145,7 +427,21 @@ public interface GridH2RowDescriptor {
      * @param colId Column id.
      * @return Result.
      */
-    public boolean isKeyValueOrVersionColumn(int colId);
+    @SuppressWarnings("RedundantIfStatement")
+    public boolean isKeyValueOrVersionColumn(int colId) {
+        assert colId >= 0;
+
+        if (colId < DEFAULT_COLUMNS_COUNT)
+            return true;
+
+        if (colId == keyAliasColId)
+            return true;
+
+        if (colId == valAliasColId)
+            return true;
+
+        return false;
+    }
 
     /**
      * Checks if provided index condition is allowed for key column or key alias column.
@@ -154,17 +450,15 @@ public interface GridH2RowDescriptor {
      * @param mask Index Condition to check.
      * @return Result.
      */
-    public boolean checkKeyIndexCondition(int masks[], int mask);
+    public boolean checkKeyIndexCondition(int masks[], int mask) {
+        assert masks != null;
+        assert masks.length > 0;
 
-    /**
-     * Initializes value cache with key, val and version.
-     *
-     * @param valCache Value cache.
-     * @param key Key.
-     * @param value Value.
-     * @param version Version.
-     */
-    public void initValueCache(Value valCache[], Value key, Value value, Value version);
+        if (keyAliasColId < 0)
+            return (masks[KEY_COL] & mask) != 0;
+        else
+            return (masks[KEY_COL] & mask) != 0 || (masks[keyAliasColId] & mask) != 0;
+    }
 
     /**
      * Clones provided row and copies values of alias key and val columns
@@ -173,7 +467,38 @@ public interface GridH2RowDescriptor {
      * @param row Source row.
      * @return Result.
      */
-    public SearchRow prepareProxyIndexRow(SearchRow row);
+    public SearchRow prepareProxyIndexRow(SearchRow row) {
+        if (row == null)
+            return null;
+
+        Value[] data = new Value[row.getColumnCount()];
+
+        for (int idx = 0; idx < data.length; idx++)
+            data[idx] = row.getValue(idx);
+
+        copyAliasColumnData(data, KEY_COL, keyAliasColId);
+        copyAliasColumnData(data, VAL_COL, valAliasColId);
+
+        return new SimpleRow(data);
+    }
+
+    /**
+     * Copies data between original and alias columns
+     *
+     * @param data Array of values.
+     * @param colId Original column id.
+     * @param aliasColId Alias column id.
+     */
+    private void copyAliasColumnData(Value[] data, int colId, int aliasColId) {
+        if (aliasColId <= 0)
+            return;
+
+        if (data[aliasColId] == null && data[colId] != null)
+            data[aliasColId] = data[colId];
+
+        if (data[colId] == null && data[aliasColId] != null)
+            data[colId] = data[aliasColId];
+    }
 
     /**
      * Gets alternative column id that may substitute the given column id.
@@ -186,5 +511,20 @@ public interface GridH2RowDescriptor {
      * @param colId Column id.
      * @return Result.
      */
-    public int getAlternativeColumnId(int colId);
-}
\ No newline at end of file
+    public int getAlternativeColumnId(int colId) {
+        if (keyAliasColId > 0) {
+            if (colId == KEY_COL)
+                return keyAliasColId;
+            else if (colId == keyAliasColId)
+                return KEY_COL;
+        }
+        if (valAliasColId > 0) {
+            if (colId == VAL_COL)
+                return valAliasColId;
+            else if (colId == valAliasColId)
+                return VAL_COL;
+        }
+
+        return colId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/583e3461/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index 79eed12..d20b56b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -34,7 +34,6 @@ import org.apache.ignite.internal.processors.cache.query.QueryTable;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryField;
-import org.apache.ignite.internal.processors.query.h2.H2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory;
 import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
 import org.apache.ignite.internal.util.typedef.F;
@@ -55,7 +54,6 @@ import org.h2.table.IndexColumn;
 import org.h2.table.TableBase;
 import org.h2.table.TableType;
 import org.h2.value.DataType;
-import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import org.jsr166.LongAdder8;
@@ -71,7 +69,7 @@ public class GridH2Table extends TableBase {
     private final GridCacheContext cctx;
 
     /** */
-    private final H2RowDescriptor desc;
+    private final GridH2RowDescriptor desc;
 
     /** */
     private volatile ArrayList<Index> idxs;
@@ -124,7 +122,7 @@ public class GridH2Table extends TableBase {
      * @param idxsFactory Indexes factory.
      * @param cctx Cache context.
      */
-    public GridH2Table(CreateTableData createTblData, H2RowDescriptor desc, H2RowFactory rowFactory,
+    public GridH2Table(CreateTableData createTblData, GridH2RowDescriptor desc, H2RowFactory rowFactory,
         GridH2SystemIndexFactory idxsFactory, GridCacheContext cctx) {
         super(createTblData);
 
@@ -408,19 +406,21 @@ public class GridH2Table extends TableBase {
         throws IgniteCheckedException {
         assert desc != null;
 
-        GridH2Row row = desc.createRow(key, partId, val, ver, expirationTime);
+        GridH2Row row = desc.createRow(key, partId, val, ver, expirationTime, link);
 
-        row.link = link;
+        if (rmv)
+            return doUpdate(row, true);
+        else {
+            GridH2KeyValueRowOnheap row0 = (GridH2KeyValueRowOnheap)row;
 
-        if (!rmv)
-            ((GridH2KeyValueRowOnheap)row).valuesCache(new Value[getColumns().length]);
+            row0.prepareValuesCache();
 
-        try {
-            return doUpdate(row, rmv);
-        }
-        finally {
-            if (!rmv)
-                ((GridH2KeyValueRowOnheap)row).valuesCache(null);
+            try {
+                return doUpdate(row, false);
+            }
+            finally {
+                row0.clearValuesCache();
+            }
         }
     }
 


[2/7] ignite git commit: IGNITE-4723 .NET: Support REGEXP_LIKE in LINQ

Posted by sb...@apache.org.
IGNITE-4723 .NET: Support REGEXP_LIKE in LINQ

This closes #2842


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

Branch: refs/heads/ignite-3478
Commit: 5e1a22db1c1da0e5d866fd5a465dd3cdc7b3ffa0
Parents: a6e2808
Author: Alexey Popov <ta...@gmail.com>
Authored: Fri Oct 13 14:19:14 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Oct 13 14:19:14 2017 +0300

----------------------------------------------------------------------
 .../Cache/Query/Linq/CacheLinqTest.Strings.cs   | 18 +++++++
 .../Impl/CacheQueryExpressionVisitor.cs         |  5 ++
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    | 55 +++++++++++++++++++-
 3 files changed, 76 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5e1a22db/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
index 1139c4d..35996b0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
@@ -83,6 +83,24 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Linq
             Assert.Throws<NotSupportedException>(() => CheckFunc(x => x.TrimEnd(toTrimFails), strings));
 
             CheckFunc(x => Regex.Replace(x, @"son.\d", "kele!"), strings);
+            CheckFunc(x => Regex.Replace(x, @"son.\d", "kele!", RegexOptions.None), strings);
+            CheckFunc(x => Regex.Replace(x, @"person.\d", "akele!", RegexOptions.IgnoreCase), strings);
+            CheckFunc(x => Regex.Replace(x, @"person.\d", "akele!", RegexOptions.Multiline), strings);
+            CheckFunc(x => Regex.Replace(x, @"person.\d", "akele!", RegexOptions.IgnoreCase | RegexOptions.Multiline), 
+                strings);
+            var notSupportedException = Assert.Throws<NotSupportedException>(() => CheckFunc(x => 
+                Regex.IsMatch(x, @"^person\d", RegexOptions.IgnoreCase | RegexOptions.CultureInvariant), strings));
+            Assert.AreEqual("RegexOptions.CultureInvariant is not supported", notSupportedException.Message);
+
+            CheckFunc(x => Regex.IsMatch(x, @"^Person_9\d"), strings);
+            CheckFunc(x => Regex.IsMatch(x, @"^person_9\d", RegexOptions.None), strings);
+            CheckFunc(x => Regex.IsMatch(x, @"^person_9\d", RegexOptions.IgnoreCase), strings);
+            CheckFunc(x => Regex.IsMatch(x, @"^Person_9\d", RegexOptions.Multiline), strings);
+            CheckFunc(x => Regex.IsMatch(x, @"^person_9\d", RegexOptions.IgnoreCase | RegexOptions.Multiline), strings);
+            notSupportedException = Assert.Throws<NotSupportedException>(() => CheckFunc(x => 
+                Regex.IsMatch(x, @"^person_9\d",RegexOptions.IgnoreCase | RegexOptions.CultureInvariant), strings));
+            Assert.AreEqual("RegexOptions.CultureInvariant is not supported", notSupportedException.Message);
+
             CheckFunc(x => x.Replace("son", ""), strings);
             CheckFunc(x => x.Replace("son", "kele"), strings);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e1a22db/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
index d187f08..4caefe1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
@@ -474,6 +474,11 @@ namespace Apache.Ignite.Linq.Impl
         [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         protected override Expression VisitConstant(ConstantExpression expression)
         {
+            if (MethodVisitor.VisitConstantCall(expression, this))
+            {
+                return expression;
+            }
+
             AppendParameter(expression.Value);
 
             return expression;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e1a22db/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
index 8abf2a6..84bd98f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
@@ -71,8 +71,12 @@ namespace Apache.Ignite.Linq.Impl
             GetStringMethod("PadRight", "rpad", typeof (int)),
             GetStringMethod("PadRight", "rpad", typeof (int), typeof (char)),
 
-            GetMethod(typeof (Regex), "Replace", new[] {typeof (string), typeof (string), typeof (string)}, 
-                GetFunc("regexp_replace")),
+            GetRegexMethod("Replace", "regexp_replace", typeof (string), typeof (string), typeof (string)),
+            GetRegexMethod("Replace", "regexp_replace", typeof (string), typeof (string), typeof (string), 
+                typeof(RegexOptions)),
+            GetRegexMethod("IsMatch", "regexp_like", typeof (string), typeof (string)),
+            GetRegexMethod("IsMatch", "regexp_like", typeof (string), typeof (string), typeof(RegexOptions)),
+
             GetMethod(typeof (DateTime), "ToString", new[] {typeof (string)},
                 (e, v) => VisitFunc(e, v, "formatdatetime", ", 'en', 'UTC'")),
 
@@ -117,6 +121,13 @@ namespace Apache.Ignite.Linq.Impl
             GetMathMethod("Truncate", typeof (decimal)),
         }.ToDictionary(x => x.Key, x => x.Value);
 
+        /// <summary> RegexOptions transformations. </summary>
+        private static readonly Dictionary<RegexOptions, string> RegexOptionFlags = new Dictionary<RegexOptions, string>
+        {
+            { RegexOptions.IgnoreCase, "i" },
+            { RegexOptions.Multiline, "m" }
+        };
+
         /// <summary>
         /// Visits the property call expression.
         /// </summary>
@@ -153,6 +164,37 @@ namespace Apache.Ignite.Linq.Impl
         }
 
         /// <summary>
+        /// Visits the constant call expression.
+        /// </summary>
+        public static bool VisitConstantCall(ConstantExpression expression, CacheQueryExpressionVisitor visitor)
+        {
+            if (expression.Type != typeof(RegexOptions))
+            {
+                return false;
+            }
+
+            var regexOptions = expression.Value as RegexOptions? ?? RegexOptions.None;
+            var result = string.Empty;
+            foreach (var option in RegexOptionFlags)
+            {
+                if (regexOptions.HasFlag(option.Key))
+                {
+                    result += option.Value;
+                    regexOptions &= ~option.Key;
+                }
+            }
+
+            if (regexOptions != RegexOptions.None)
+            {
+                throw new NotSupportedException(string.Format("RegexOptions.{0} is not supported", regexOptions));
+            }
+
+            visitor.AppendParameter(result);
+
+            return true;
+        }
+
+        /// <summary>
         /// Gets the function.
         /// </summary>
         private static VisitMethodDelegate GetFunc(string func, params int[] adjust)
@@ -302,6 +344,15 @@ namespace Apache.Ignite.Linq.Impl
         }
 
         /// <summary>
+        /// Gets the Regex method.
+        /// </summary>
+        private static KeyValuePair<MethodInfo, VisitMethodDelegate> GetRegexMethod(string name, string sqlName,
+            params Type[] argTypes)
+        {
+            return GetMethod(typeof(Regex), name, argTypes, GetFunc(sqlName));
+        }
+
+        /// <summary>
         /// Gets string parameterized Trim(TrimStart, TrimEnd) method.
         /// </summary>
         private static KeyValuePair<MethodInfo, VisitMethodDelegate> GetParameterizedTrimMethod(string name,


[5/7] ignite git commit: IGNITE-6634: Removed IgniteDistributedJoinTestSuite. It's tests are distributed between "Query" and "Query 2" suites. This closes #2857.

Posted by sb...@apache.org.
IGNITE-6634: Removed IgniteDistributedJoinTestSuite. It's tests are distributed between "Query" and "Query 2" suites. This closes #2857.


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

Branch: refs/heads/ignite-3478
Commit: 0b304042dedab3196f9db98e51f90e8f5b72bdaa
Parents: 0323468
Author: devozerov <vo...@gridgain.com>
Authored: Mon Oct 16 13:37:11 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 16 13:37:11 2017 +0300

----------------------------------------------------------------------
 ...ributedJoinPartitionedAndReplicatedTest.java |  2 +
 .../IgniteCacheQuerySelfTestSuite.java          | 17 ++++++
 .../IgniteCacheQuerySelfTestSuite2.java         |  6 +++
 .../IgniteCacheQuerySelfTestSuite3.java         |  3 --
 .../IgniteDistributedJoinTestSuite.java         | 55 --------------------
 5 files changed, 25 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b304042/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDistributedJoinPartitionedAndReplicatedTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDistributedJoinPartitionedAndReplicatedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDistributedJoinPartitionedAndReplicatedTest.java
index 5e906af..eb568df 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDistributedJoinPartitionedAndReplicatedTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDistributedJoinPartitionedAndReplicatedTest.java
@@ -194,6 +194,8 @@ public class IgniteCacheDistributedJoinPartitionedAndReplicatedTest extends Grid
      * @throws Exception If failed.
      */
     public void testJoin2() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-5956");
+
         join(true, PARTITIONED, REPLICATED, PARTITIONED);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b304042/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 83b4689..0b1a753 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
@@ -46,6 +46,12 @@ import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgume
 import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDeleteSqlQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCollocatedAndNotTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCustomAffinityMapper;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinNoIndexTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinPartitionedAndReplicatedTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinQueryConditionsTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheFullTextQueryNodeJoiningSelfTest;
@@ -123,6 +129,7 @@ import org.apache.ignite.internal.processors.cache.query.IgniteCacheQueryCacheDe
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryTxSelfTest;
 import org.apache.ignite.internal.processors.client.ClientConnectorConfigurationValidationSelfTest;
+import org.apache.ignite.internal.processors.query.IgniteSqlDistributedJoinSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSkipReducerOnUpdateDmlFlagSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlParameterizedQueryTest;
 import org.apache.ignite.internal.processors.query.h2.IgniteSqlBigIntegerKeyTest;
@@ -325,6 +332,16 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheDistributedQueryCancelSelfTest.class);
         suite.addTestSuite(IgniteCacheLocalQueryCancelOrTimeoutSelfTest.class);
 
+        // Distributed joins.
+        suite.addTestSuite(H2CompareBigQueryDistributedJoinsTest.class);
+        suite.addTestSuite(IgniteCacheDistributedJoinCollocatedAndNotTest.class);
+        suite.addTestSuite(IgniteCacheDistributedJoinCustomAffinityMapper.class);
+        suite.addTestSuite(IgniteCacheDistributedJoinNoIndexTest.class);
+        suite.addTestSuite(IgniteCacheDistributedJoinPartitionedAndReplicatedTest.class);
+        suite.addTestSuite(IgniteCacheDistributedJoinQueryConditionsTest.class);
+        suite.addTestSuite(IgniteCacheDistributedJoinTest.class);
+        suite.addTestSuite(IgniteSqlDistributedJoinSelfTest.class);
+
         // Other.
         suite.addTestSuite(CacheIteratorScanQueryTest.class);
         suite.addTestSuite(CacheQueryNewClientSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b304042/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
index f84ddc2..91e4478 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@ -28,8 +28,10 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryMultiThrea
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheClientQueryReplicatedNodeRestartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeFailTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartDistributedJoinSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest2;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicColumnsConcurrentAtomicPartitionedSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicColumnsConcurrentAtomicReplicatedSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicColumnsConcurrentTransactionalPartitionedSelfTest;
@@ -67,6 +69,10 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite {
         suite.addTestSuite(DynamicColumnsConcurrentAtomicReplicatedSelfTest.class);
         suite.addTestSuite(DynamicColumnsConcurrentTransactionalReplicatedSelfTest.class);
 
+        // Distributed joins.
+        suite.addTestSuite(IgniteCacheQueryNodeRestartDistributedJoinSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.class);
+
         // Other tests.
         suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b304042/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
index e523cf3..4ea8bca 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
@@ -37,7 +37,6 @@ import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBin
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBinaryIterationStoreEnabledTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.ClientReconnectContinuousQueryTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.ContinuousQueryPeerClassLoadingTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.ClientReconnectContinuousQueryTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.ContinuousQueryRemoteFilterMissingInClassPathSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicNearEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicP2PDisabledSelfTest;
@@ -125,8 +124,6 @@ public class IgniteCacheQuerySelfTestSuite3 extends TestSuite {
         suite.addTestSuite(CacheContinuousQueryConcurrentPartitionUpdateTest.class);
         suite.addTestSuite(CacheContinuousQueryEventBufferTest.class);
 
-        suite.addTest(IgniteDistributedJoinTestSuite.suite());
-
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b304042/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
deleted file mode 100644
index cf67041..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.testsuites;
-
-import junit.framework.TestSuite;
-import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCollocatedAndNotTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCustomAffinityMapper;
-import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinNoIndexTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinPartitionedAndReplicatedTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinQueryConditionsTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartDistributedJoinSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest;
-import org.apache.ignite.internal.processors.query.IgniteSqlDistributedJoinSelfTest;
-import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest;
-
-/**
- *
- */
-public class IgniteDistributedJoinTestSuite extends TestSuite {
-    /**
-     * @return Suite.
-     */
-    public static TestSuite suite() {
-        TestSuite suite = new TestSuite("Distributed Joins Test Suite.");
-
-        suite.addTestSuite(H2CompareBigQueryDistributedJoinsTest.class);
-        suite.addTestSuite(IgniteCacheDistributedJoinCollocatedAndNotTest.class);
-        suite.addTestSuite(IgniteCacheDistributedJoinCustomAffinityMapper.class);
-        suite.addTestSuite(IgniteCacheDistributedJoinNoIndexTest.class);
-        suite.addTestSuite(IgniteCacheDistributedJoinPartitionedAndReplicatedTest.class);
-        suite.addTestSuite(IgniteCacheDistributedJoinQueryConditionsTest.class);
-        suite.addTestSuite(IgniteCacheDistributedJoinTest.class);
-        suite.addTestSuite(IgniteCacheQueryNodeRestartDistributedJoinSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.class);
-        suite.addTestSuite(IgniteSqlDistributedJoinSelfTest.class);
-
-        return suite;
-    }
-}