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/12/05 07:31:16 UTC

[01/11] ignite git commit: Minor changes in datastructures - Fixes #3129.

Repository: ignite
Updated Branches:
  refs/heads/ignite-zk 74526d19e -> be7ae489b


Minor changes in datastructures - Fixes #3129.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-zk
Commit: f08c9d3ada4139df6835188f327a4181a0470b61
Parents: 6101fde
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Mon Dec 4 11:04:41 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Dec 4 11:04:41 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  34 +++-
 .../AtomicDataStructureProxy.java               | 189 +++++++++++++++++++
 .../datastructures/DataStructuresProcessor.java |  20 ++
 .../datastructures/GridCacheAtomicLongImpl.java | 120 ++----------
 .../GridCacheAtomicReferenceImpl.java           | 114 ++---------
 .../GridCacheAtomicSequenceImpl.java            | 113 ++---------
 .../GridCacheAtomicStampedImpl.java             | 115 ++---------
 .../GridCacheCountDownLatchImpl.java            |  73 +------
 .../datastructures/GridCacheLockImpl.java       |  91 ++-------
 .../datastructures/GridCacheRemovable.java      |   6 +
 .../datastructures/GridCacheSemaphoreImpl.java  |  85 ++-------
 .../AtomicCacheAffinityConfigurationTest.java   |   7 +-
 12 files changed, 348 insertions(+), 619 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 99ff4cb..dcc3d13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1676,6 +1676,26 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Gets a collection of currently started public cache names.
+     *
+     * @return Collection of currently started public cache names
+     */
+    public Collection<String> publicAndDsCacheNames() {
+        return F.viewReadOnly(cacheDescriptors().values(),
+            new IgniteClosure<DynamicCacheDescriptor, String>() {
+                @Override public String apply(DynamicCacheDescriptor desc) {
+                    return desc.cacheConfiguration().getName();
+                }
+            },
+            new IgnitePredicate<DynamicCacheDescriptor>() {
+                @Override public boolean apply(DynamicCacheDescriptor desc) {
+                    return desc.cacheType().userCache() || desc.cacheType() == CacheType.DATA_STRUCTURES;
+                }
+            }
+        );
+    }
+
+    /**
      * Gets cache mode.
      *
      * @param cacheName Cache name to check.
@@ -1852,8 +1872,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         IgniteCacheProxyImpl<?, ?> proxy = jCacheProxies.get(ccfg.getName());
 
-        if (!disabledAfterStart && proxy != null && proxy.isRestarting())
+        if (!disabledAfterStart && proxy != null && proxy.isRestarting()) {
             proxy.onRestarted(cacheCtx, cache);
+
+            if (cacheCtx.dataStructuresCache())
+                ctx.dataStructures().restart(proxy.internalProxy());
+        }
     }
 
     /**
@@ -1874,6 +1898,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 caches.get(proxy.getName()).active(true);
 
                 proxy.onRestarted(cacheCtx, cacheCtx.cache());
+
+                if (cacheCtx.dataStructuresCache())
+                    ctx.dataStructures().restart(proxy.internalProxy());
             }
         }
     }
@@ -1977,6 +2004,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         // Break the proxy before exchange future is done.
         if (req.restart()) {
+            if (DataStructuresProcessor.isDataStructureCache(req.cacheName()))
+                ctx.dataStructures().suspend(req.cacheName());
+
             GridCacheAdapter<?, ?> cache = caches.get(req.cacheName());
 
             if (cache != null)
@@ -2668,6 +2698,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                         ct = CacheType.UTILITY;
                     else if (internalCaches.contains(ccfg.getName()))
                         ct = CacheType.INTERNAL;
+                    else if (DataStructuresProcessor.isDataStructureCache(ccfg.getName()))
+                        ct = CacheType.DATA_STRUCTURES;
                     else
                         ct = CacheType.USER;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/AtomicDataStructureProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/AtomicDataStructureProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/AtomicDataStructureProxy.java
new file mode 100644
index 0000000..214672a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/AtomicDataStructureProxy.java
@@ -0,0 +1,189 @@
+/*
+ * 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.datastructures;
+
+import java.io.Externalizable;
+import org.apache.ignite.IgniteCacheRestartingException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+public abstract class AtomicDataStructureProxy<V extends AtomicDataStructureValue>
+    implements GridCacheRemovable,IgniteChangeGlobalStateSupport {
+    /** Logger. */
+    protected IgniteLogger log;
+
+    /** Removed flag. */
+    protected volatile boolean rmvd;
+
+    /** Suspended future. */
+    private volatile GridFutureAdapter<Void> suspendFut;
+
+    /** Check removed flag. */
+    private boolean rmvCheck;
+
+    /** Structure name. */
+    protected String name;
+
+    /** Structure key. */
+    protected GridCacheInternalKey key;
+
+    /** Structure projection. */
+    protected IgniteInternalCache<GridCacheInternalKey, V> cacheView;
+
+    /** Cache context. */
+    protected volatile GridCacheContext<GridCacheInternalKey, V> ctx;
+
+    /**
+     * Empty constructor required by {@link Externalizable}.
+     */
+    public AtomicDataStructureProxy() {
+        // No-op.
+    }
+
+    /**
+     * Default constructor.
+     *
+     * @param name Structure name.
+     * @param key Structure key.
+     * @param cacheView Cache projection.
+     */
+    public AtomicDataStructureProxy(String name,
+        GridCacheInternalKey key,
+        IgniteInternalCache<GridCacheInternalKey, V> cacheView)
+    {
+        assert key != null;
+        assert cacheView != null;
+
+        this.ctx = cacheView.context();
+        this.key = key;
+        this.cacheView = cacheView;
+        this.name = name;
+
+        log = ctx.logger(getClass());
+    }
+
+    /** {@inheritDoc} */
+    public String name() {
+            return name;
+        }
+
+    /** {@inheritDoc} */
+    public GridCacheInternalKey key() {
+            return key;
+        }
+
+    /** {@inheritDoc} */
+    public boolean removed() {
+        return rmvd;
+    }
+
+    /**
+     * Check removed status.
+     *
+     * @throws IllegalStateException If removed.
+     */
+    protected void checkRemoved() throws IllegalStateException {
+        if (rmvd)
+            throw removedError();
+
+        GridFutureAdapter<Void> suspendFut0 = suspendFut;
+
+        if (suspendFut0 != null && !suspendFut0.isDone())
+            throw suspendedError();
+
+        if (rmvCheck) {
+            try {
+                rmvd = cacheView.get(key) == null;
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+
+            rmvCheck = false;
+
+            if (rmvd) {
+                ctx.kernalContext().dataStructures().onRemoved(key, this);
+
+                throw removedError();
+            }
+        }
+    }
+
+    /**
+     * @return Error.
+     */
+    private IllegalStateException removedError() {
+        return new IllegalStateException("Sequence was removed from cache: " + name);
+    }
+
+    /**
+     * @return Error.
+     */
+    private IllegalStateException suspendedError() {
+        throw new IgniteCacheRestartingException(new IgniteFutureImpl<>(suspendFut), "Underlying cache is restarting: " + ctx.name());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onRemoved() {
+        return rmvd = true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void needCheckNotRemoved() {
+        rmvCheck = true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void suspend() {
+        suspendFut = new GridFutureAdapter<>();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void restart(IgniteInternalCache cache) {
+        invalidateLocalState();
+
+        cacheView = cache;
+        ctx = cache.context();
+        rmvCheck = true;
+        suspendFut.onDone();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
+        this.ctx = kctx.cache().<GridCacheInternalKey, V>context().cacheContext(ctx.cacheId());
+        this.cacheView = ctx.cache();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDeActivate(GridKernalContext kctx) {
+        // No-op.
+    }
+
+    protected void invalidateLocalState() {
+        // No-op
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index b26acdd..acd8c11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.datastructures;
 
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -73,6 +74,7 @@ import org.apache.ignite.internal.util.typedef.CX1;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.GPR;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -663,6 +665,24 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
         });
     }
 
+    public void suspend(String cacheName) {
+        for (Map.Entry<GridCacheInternalKey, GridCacheRemovable> e : dsMap.entrySet()) {
+            String cacheName0 = ATOMICS_CACHE_NAME + "@" + e.getKey().groupName();
+
+            if (cacheName0.equals(cacheName))
+                e.getValue().suspend();
+        }
+    }
+
+    public void restart(IgniteInternalCache cache) {
+        for (Map.Entry<GridCacheInternalKey, GridCacheRemovable> e : dsMap.entrySet()) {
+            String cacheName0 = ATOMICS_CACHE_NAME + "@" + e.getKey().groupName();
+
+            if (cacheName0.equals(cache.name()))
+                e.getValue().restart(cache);
+        }
+    }
+
     /**
      * Gets an atomic reference from cache or creates one if it's not cached.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
index 0bc0c63..8e6f913 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
@@ -40,7 +40,8 @@ import org.apache.ignite.lang.IgniteBiTuple;
 /**
  * Cache atomic long implementation.
  */
-public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, IgniteChangeGlobalStateSupport, Externalizable {
+public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<GridCacheAtomicLongValue>
+    implements GridCacheAtomicLongEx, IgniteChangeGlobalStateSupport, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -52,24 +53,6 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
             }
         };
 
-    /** Atomic long name. */
-    private String name;
-
-    /** Removed flag.*/
-    private volatile boolean rmvd;
-
-    /** Check removed flag. */
-    private boolean rmvCheck;
-
-    /** Atomic long key. */
-    private GridCacheInternalKey key;
-
-    /** Atomic long projection. */
-    private IgniteInternalCache<GridCacheInternalKey, GridCacheAtomicLongValue> atomicView;
-
-    /** Cache context. */
-    private GridCacheContext<GridCacheInternalKey, GridCacheAtomicLongValue> ctx;
-
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -87,19 +70,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
     public GridCacheAtomicLongImpl(String name,
         GridCacheInternalKey key,
         IgniteInternalCache<GridCacheInternalKey, GridCacheAtomicLongValue> atomicView) {
-        assert key != null;
-        assert atomicView != null;
-        assert name != null;
-
-        this.ctx = atomicView.context();
-        this.key = key;
-        this.atomicView = atomicView;
-        this.name = name;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() {
-        return name;
+        super(name, key, atomicView);
     }
 
     /** {@inheritDoc} */
@@ -107,7 +78,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         checkRemoved();
 
         try {
-            GridCacheAtomicLongValue val = atomicView.get(key);
+            GridCacheAtomicLongValue val = cacheView.get(key);
 
             if (val == null)
                 throw new IgniteException("Failed to find atomic long: " + name);
@@ -124,7 +95,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         checkRemoved();
 
         try{
-            EntryProcessorResult<Long> res = atomicView.invoke(key, IncrementAndGetProcessor.INSTANCE);
+            EntryProcessorResult<Long> res = cacheView.invoke(key, IncrementAndGetProcessor.INSTANCE);
 
             assert res != null && res.get() != null : res;
 
@@ -143,7 +114,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         checkRemoved();
 
         try {
-            EntryProcessorResult<Long> res = atomicView.invoke(key, GetAndIncrementProcessor.INSTANCE);
+            EntryProcessorResult<Long> res = cacheView.invoke(key, GetAndIncrementProcessor.INSTANCE);
 
             assert res != null && res.get() != null : res;
 
@@ -162,7 +133,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         checkRemoved();
 
         try {
-            EntryProcessorResult<Long> res = atomicView.invoke(key, new AddAndGetProcessor(l));
+            EntryProcessorResult<Long> res = cacheView.invoke(key, new AddAndGetProcessor(l));
 
             assert res != null && res.get() != null : res;
 
@@ -181,7 +152,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         checkRemoved();
 
         try {
-            EntryProcessorResult<Long> res = atomicView.invoke(key, new GetAndAddProcessor(l));
+            EntryProcessorResult<Long> res = cacheView.invoke(key, new GetAndAddProcessor(l));
 
             assert res != null && res.get() != null : res;
 
@@ -200,7 +171,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         checkRemoved();
 
         try {
-            EntryProcessorResult<Long> res = atomicView.invoke(key, DecrementAndGetProcessor.INSTANCE);
+            EntryProcessorResult<Long> res = cacheView.invoke(key, DecrementAndGetProcessor.INSTANCE);
 
             assert res != null && res.get() != null : res;
 
@@ -219,7 +190,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         checkRemoved();
 
         try {
-            EntryProcessorResult<Long> res = atomicView.invoke(key, GetAndDecrementProcessor.INSTANCE);
+            EntryProcessorResult<Long> res = cacheView.invoke(key, GetAndDecrementProcessor.INSTANCE);
 
             assert res != null && res.get() != null : res;
 
@@ -238,7 +209,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         checkRemoved();
 
         try {
-            EntryProcessorResult<Long> res = atomicView.invoke(key, new GetAndSetProcessor(l));
+            EntryProcessorResult<Long> res = cacheView.invoke(key, new GetAndSetProcessor(l));
 
             assert res != null && res.get() != null : res;
 
@@ -257,7 +228,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         checkRemoved();
 
         try {
-            EntryProcessorResult<Long> res = atomicView.invoke(key, new CompareAndSetProcessor(expVal, newVal));
+            EntryProcessorResult<Long> res = cacheView.invoke(key, new CompareAndSetProcessor(expVal, newVal));
 
             assert res != null && res.get() != null : res;
 
@@ -280,7 +251,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         checkRemoved();
 
         try {
-            EntryProcessorResult<Long> res = atomicView.invoke(key, new CompareAndSetProcessor(expVal, newVal));
+            EntryProcessorResult<Long> res = cacheView.invoke(key, new CompareAndSetProcessor(expVal, newVal));
 
             assert res != null && res.get() != null : res;
 
@@ -294,60 +265,6 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
         }
     }
 
-    /**
-     * Check removed flag.
-     *
-     * @throws IllegalStateException If removed.
-     */
-    private void checkRemoved() throws IllegalStateException {
-        if (rmvd)
-            throw removedError();
-
-        if (rmvCheck) {
-            try {
-                rmvd = atomicView.get(key) == null;
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
-
-            rmvCheck = false;
-
-            if (rmvd) {
-                ctx.kernalContext().dataStructures().onRemoved(key, this);
-
-                throw removedError();
-            }
-        }
-    }
-
-    /**
-     * @return Error.
-     */
-    private IllegalStateException removedError() {
-        return new IllegalStateException("Atomic long was removed from cache: " + name);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean onRemoved() {
-        return rmvd = true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void needCheckNotRemoved() {
-        rmvCheck = true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheInternalKey key() {
-        return key;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean removed() {
-        return rmvd;
-    }
-
     /** {@inheritDoc} */
     @Override public void close() {
         if (rmvd)
@@ -362,17 +279,6 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
     }
 
     /** {@inheritDoc} */
-    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-        this.ctx = kctx.cache().<GridCacheInternalKey, GridCacheAtomicLongValue>context().cacheContext(ctx.cacheId());
-        this.atomicView = ctx.cache();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeObject(ctx.kernalContext());
         out.writeUTF(name);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
index 42f16f2..df126d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
@@ -31,7 +31,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
@@ -47,7 +46,7 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
 /**
  * Cache atomic reference implementation.
  */
-public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicReferenceEx<T>,
+public final class GridCacheAtomicReferenceImpl<T> extends AtomicDataStructureProxy<GridCacheAtomicReferenceValue<T>> implements GridCacheAtomicReferenceEx<T>,
     IgniteChangeGlobalStateSupport, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
@@ -60,24 +59,6 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
             }
         };
 
-    /** Atomic reference name. */
-    private String name;
-
-    /** Status.*/
-    private volatile boolean rmvd;
-
-    /** Check removed flag. */
-    private boolean rmvCheck;
-
-    /** Atomic reference key. */
-    private GridCacheInternalKey key;
-
-    /** Atomic reference projection. */
-    private IgniteInternalCache<GridCacheInternalKey, GridCacheAtomicReferenceValue<T>> atomicView;
-
-    /** Cache context. */
-    private GridCacheContext<GridCacheInternalKey, GridCacheAtomicReferenceValue<T>> ctx;
-
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -95,14 +76,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
     public GridCacheAtomicReferenceImpl(String name,
         GridCacheInternalKey key,
         IgniteInternalCache<GridCacheInternalKey, GridCacheAtomicReferenceValue<T>> atomicView) {
-        assert key != null;
-        assert atomicView != null;
-        assert name != null;
-
-        this.ctx = atomicView.context();
-        this.key = key;
-        this.atomicView = atomicView;
-        this.name = name;
+        super(name, key, atomicView);
     }
 
     /** {@inheritDoc} */
@@ -115,7 +89,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
         checkRemoved();
 
         try {
-            GridCacheAtomicReferenceValue<T> ref = atomicView.get(key);
+            GridCacheAtomicReferenceValue<T> ref = cacheView.get(key);
 
             if (ref == null)
                 throw new IgniteCheckedException("Failed to find atomic reference with given name: " + name);
@@ -133,17 +107,17 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
 
         try {
             if (ctx.dataStructures().knownType(val))
-                atomicView.invoke(key, new ReferenceSetEntryProcessor<>(val));
+                cacheView.invoke(key, new ReferenceSetEntryProcessor<>(val));
             else {
                 CU.retryTopologySafe(new Callable<Void>() {
                     @Override public Void call() throws Exception {
-                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ)) {
-                            GridCacheAtomicReferenceValue<T> ref = atomicView.get(key);
+                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                            GridCacheAtomicReferenceValue<T> ref = cacheView.get(key);
 
                             if (ref == null)
                                 throw new IgniteException("Failed to find atomic reference with given name: " + name);
 
-                            atomicView.put(key, new GridCacheAtomicReferenceValue<>(val));
+                            cacheView.put(key, new GridCacheAtomicReferenceValue<>(val));
 
                             tx.commit();
                         }
@@ -168,7 +142,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
         try {
             if (ctx.dataStructures().knownType(expVal) && ctx.dataStructures().knownType(newVal)) {
                 EntryProcessorResult<Boolean> res =
-                    atomicView.invoke(key, new ReferenceCompareAndSetEntryProcessor<>(expVal, newVal));
+                    cacheView.invoke(key, new ReferenceCompareAndSetEntryProcessor<>(expVal, newVal));
 
                 assert res != null && res.get() != null : res;
 
@@ -177,8 +151,8 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
             else {
                 return CU.retryTopologySafe(new Callable<Boolean>() {
                     @Override public Boolean call() throws Exception {
-                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ)) {
-                            GridCacheAtomicReferenceValue<T> ref = atomicView.get(key);
+                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                            GridCacheAtomicReferenceValue<T> ref = cacheView.get(key);
 
                             if (ref == null)
                                 throw new IgniteException("Failed to find atomic reference with given name: " + name);
@@ -188,7 +162,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
                             if (!F.eq(expVal, curVal))
                                 return false;
                             else {
-                                atomicView.put(key, new GridCacheAtomicReferenceValue<>(newVal));
+                                cacheView.put(key, new GridCacheAtomicReferenceValue<>(newVal));
 
                                 tx.commit();
 
@@ -220,7 +194,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
         try {
             if (ctx.dataStructures().knownType(expVal) && ctx.dataStructures().knownType(newVal)) {
                 EntryProcessorResult<T> res =
-                    atomicView.invoke(key, new ReferenceCompareAndSetAndGetEntryProcessor<T>(expVal, newVal));
+                    cacheView.invoke(key, new ReferenceCompareAndSetAndGetEntryProcessor<T>(expVal, newVal));
 
                 assert res != null;
 
@@ -229,8 +203,8 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
             else {
                 return CU.retryTopologySafe(new Callable<T>() {
                     @Override public T call() throws Exception {
-                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ)) {
-                            GridCacheAtomicReferenceValue<T> ref = atomicView.get(key);
+                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                            GridCacheAtomicReferenceValue<T> ref = cacheView.get(key);
 
                             if (ref == null)
                                 throw new IgniteException("Failed to find atomic reference with given name: " + name);
@@ -240,7 +214,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
                             if (!F.eq(expVal, curVal))
                                 return curVal;
                             else {
-                                atomicView.put(key, new GridCacheAtomicReferenceValue<>(newVal));
+                                cacheView.put(key, new GridCacheAtomicReferenceValue<>(newVal));
 
                                 tx.commit();
 
@@ -260,26 +234,6 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onRemoved() {
-        return rmvd = true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void needCheckNotRemoved() {
-        rmvCheck = true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheInternalKey key() {
-        return key;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean removed() {
-        return rmvd;
-    }
-
-    /** {@inheritDoc} */
     @Override public void close() {
         if (rmvd)
             return;
@@ -292,44 +246,6 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-        this.ctx = kctx.cache().<GridCacheInternalKey, GridCacheAtomicReferenceValue<T>>context().cacheContext(ctx.cacheId());
-        this.atomicView = ctx.cache();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) {
-        // No-op.
-    }
-
-    /**
-     * Check removed status.
-     *
-     * @throws IllegalStateException If removed.
-     */
-    private void checkRemoved() throws IllegalStateException {
-        if (rmvd)
-            throw removedError();
-
-        if (rmvCheck) {
-            try {
-                rmvd = atomicView.get(key) == null;
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
-
-            rmvCheck = false;
-
-            if (rmvd) {
-                ctx.kernalContext().dataStructures().onRemoved(key, this);
-
-                throw removedError();
-            }
-        }
-    }
-
     /**
      * @return Error.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
index 019de3c..fd4db4a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
@@ -26,6 +26,7 @@ import java.io.ObjectStreamException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCacheRestartingException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -34,6 +35,8 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -49,7 +52,8 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
 /**
  * Cache sequence implementation.
  */
-public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenceEx, IgniteChangeGlobalStateSupport, Externalizable {
+public final class GridCacheAtomicSequenceImpl extends AtomicDataStructureProxy<GridCacheAtomicSequenceValue>
+    implements GridCacheAtomicSequenceEx, IgniteChangeGlobalStateSupport, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -61,27 +65,6 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
             }
         };
 
-    /** Logger. */
-    private IgniteLogger log;
-
-    /** Sequence name. */
-    private String name;
-
-    /** Removed flag. */
-    private volatile boolean rmvd;
-
-    /** Check removed flag. */
-    private boolean rmvCheck;
-
-    /** Sequence key. */
-    private GridCacheInternalKey key;
-
-    /** Sequence projection. */
-    private IgniteInternalCache<GridCacheInternalKey, GridCacheAtomicSequenceValue> seqView;
-
-    /** Cache context. */
-    private volatile GridCacheContext<GridCacheInternalKey, GridCacheAtomicSequenceValue> ctx;
-
     /** Local value of sequence. */
     @GridToStringInclude(sensitive = true)
     private volatile long locVal;
@@ -131,24 +114,13 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
         long locVal,
         long upBound)
     {
-        assert key != null;
-        assert seqView != null;
+        super(name, key, seqView);
+
         assert locVal <= upBound;
 
         this.batchSize = batchSize;
-        this.ctx = seqView.context();
-        this.key = key;
-        this.seqView = seqView;
         this.upBound = upBound;
         this.locVal = locVal;
-        this.name = name;
-
-        log = ctx.logger(getClass());
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() {
-        return name;
     }
 
     /** {@inheritDoc} */
@@ -291,58 +263,10 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
         }
     }
 
-    /**
-     * Check removed status.
-     *
-     * @throws IllegalStateException If removed.
-     */
-    private void checkRemoved() throws IllegalStateException {
-        if (rmvd)
-            throw removedError();
-
-        if (rmvCheck) {
-            try {
-                rmvd = seqView.get(key) == null;
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
-
-            rmvCheck = false;
-
-            if (rmvd) {
-                ctx.kernalContext().dataStructures().onRemoved(key, this);
-
-                throw removedError();
-            }
-        }
-    }
-
-    /**
-     * @return Error.
-     */
-    private IllegalStateException removedError() {
-        return new IllegalStateException("Sequence was removed from cache: " + name);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean onRemoved() {
-        return rmvd = true;
-    }
-
     /** {@inheritDoc} */
-    @Override public void needCheckNotRemoved() {
-        rmvCheck = true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheInternalKey key() {
-        return key;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean removed() {
-        return rmvd;
+    @Override protected void invalidateLocalState() {
+        locVal = 0;
+        upBound = -1;
     }
 
     /** {@inheritDoc} */
@@ -371,8 +295,8 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
             @Override public Long call() throws Exception {
                 assert distUpdateFreeTop.isHeldByCurrentThread() || distUpdateLockedTop.isHeldByCurrentThread();
 
-                try (GridNearTxLocal tx = CU.txStartInternal(ctx, seqView, PESSIMISTIC, REPEATABLE_READ)) {
-                    GridCacheAtomicSequenceValue seq = seqView.get(key);
+                try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                    GridCacheAtomicSequenceValue seq = cacheView.get(key);
 
                     checkRemoved();
 
@@ -428,7 +352,7 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
                     // Global counter must be more than reserved upper bound.
                     seq.set(newUpBound + 1);
 
-                    seqView.put(key, seq);
+                    cacheView.put(key, seq);
 
                     tx.commit();
 
@@ -444,17 +368,6 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
     }
 
     /** {@inheritDoc} */
-    @Override public void onActivate(GridKernalContext kctx) {
-        ctx = kctx.cache().<GridCacheInternalKey, GridCacheAtomicSequenceValue>context().cacheContext(ctx.cacheId());
-        seqView = ctx.cache();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeObject(ctx.kernalContext());
         out.writeUTF(name);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
index ed7a225..70f3b48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
@@ -47,7 +47,8 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
 /**
  * Cache atomic stamped implementation.
  */
-public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicStampedEx<T, S>, IgniteChangeGlobalStateSupport, Externalizable {
+public final class GridCacheAtomicStampedImpl<T, S> extends AtomicDataStructureProxy<GridCacheAtomicStampedValue<T, S>>
+    implements GridCacheAtomicStampedEx<T, S>, IgniteChangeGlobalStateSupport, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -59,24 +60,6 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
             }
         };
 
-    /** Atomic stamped name. */
-    private String name;
-
-    /** Removed flag.*/
-    private volatile boolean rmvd;
-
-    /** Check removed flag. */
-    private boolean rmvCheck;
-
-    /** Atomic stamped key. */
-    private GridCacheInternalKey key;
-
-    /** Atomic stamped projection. */
-    private IgniteInternalCache<GridCacheInternalKey, GridCacheAtomicStampedValue<T, S>> atomicView;
-
-    /** Cache context. */
-    private GridCacheContext<GridCacheInternalKey, GridCacheAtomicStampedValue<T, S>> ctx;
-
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -94,19 +77,7 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
     public GridCacheAtomicStampedImpl(String name,
         GridCacheInternalKey key,
         IgniteInternalCache<GridCacheInternalKey, GridCacheAtomicStampedValue<T, S>> atomicView) {
-        assert key != null;
-        assert atomicView != null;
-        assert name != null;
-
-        this.ctx = atomicView.context();
-        this.key = key;
-        this.atomicView = atomicView;
-        this.name = name;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() {
-        return name;
+        super(name, key, atomicView);
     }
 
     /** {@inheritDoc} */
@@ -114,7 +85,7 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
         checkRemoved();
 
         try {
-            GridCacheAtomicStampedValue<T, S> stmp = atomicView.get(key);
+            GridCacheAtomicStampedValue<T, S> stmp = cacheView.get(key);
 
             if (stmp == null)
                 throw new IgniteCheckedException("Failed to find atomic stamped with given name: " + name);
@@ -132,17 +103,17 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
 
         try {
             if (ctx.dataStructures().knownType(val) && ctx.dataStructures().knownType(stamp))
-                atomicView.invoke(key, new StampedSetEntryProcessor<>(val, stamp));
+                cacheView.invoke(key, new StampedSetEntryProcessor<>(val, stamp));
             else {
                 CU.retryTopologySafe(new Callable<Void>() {
                     @Override public Void call() throws Exception {
-                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ)) {
-                            GridCacheAtomicStampedValue<T, S> ref = atomicView.get(key);
+                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                            GridCacheAtomicStampedValue<T, S> ref = cacheView.get(key);
 
                             if (ref == null)
                                 throw new IgniteException("Failed to find atomic stamped with given name: " + name);
 
-                            atomicView.put(key, new GridCacheAtomicStampedValue<>(val, stamp));
+                            cacheView.put(key, new GridCacheAtomicStampedValue<>(val, stamp));
 
                             tx.commit();
                         }
@@ -170,7 +141,7 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
                 ctx.dataStructures().knownType(expStamp) &&
                 ctx.dataStructures().knownType(newStamp)) {
                 EntryProcessorResult<Boolean> res =
-                    atomicView.invoke(key, new StampedCompareAndSetEntryProcessor<>(expVal, expStamp, newVal, newStamp));
+                    cacheView.invoke(key, new StampedCompareAndSetEntryProcessor<>(expVal, expStamp, newVal, newStamp));
 
                 assert res != null && res.get() != null : res;
 
@@ -179,14 +150,14 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
             else {
                 return CU.retryTopologySafe(new Callable<Boolean>() {
                     @Override public Boolean call() throws Exception {
-                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ)) {
-                            GridCacheAtomicStampedValue<T, S> val = atomicView.get(key);
+                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                            GridCacheAtomicStampedValue<T, S> val = cacheView.get(key);
 
                             if (val == null)
                                 throw new IgniteException("Failed to find atomic stamped with given name: " + name);
 
                             if (F.eq(expVal, val.value()) && F.eq(expStamp, val.stamp())) {
-                                atomicView.put(key, new GridCacheAtomicStampedValue<>(newVal, newStamp));
+                                cacheView.put(key, new GridCacheAtomicStampedValue<>(newVal, newStamp));
 
                                 tx.commit();
 
@@ -212,7 +183,7 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
         checkRemoved();
 
         try {
-            GridCacheAtomicStampedValue<T, S> stmp = atomicView.get(key);
+            GridCacheAtomicStampedValue<T, S> stmp = cacheView.get(key);
 
             if (stmp == null)
                 throw new IgniteCheckedException("Failed to find atomic stamped with given name: " + name);
@@ -229,7 +200,7 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
         checkRemoved();
 
         try {
-            GridCacheAtomicStampedValue<T, S> stmp = atomicView.get(key);
+            GridCacheAtomicStampedValue<T, S> stmp = cacheView.get(key);
 
             if (stmp == null)
                 throw new IgniteCheckedException("Failed to find atomic stamped with given name: " + name);
@@ -242,26 +213,6 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onRemoved() {
-        return rmvd = true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void needCheckNotRemoved() {
-        rmvCheck = true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheInternalKey key() {
-        return key;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean removed() {
-        return rmvd;
-    }
-
-    /** {@inheritDoc} */
     @Override public void close() {
         if (rmvd)
             return;
@@ -310,44 +261,6 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
     }
 
     /**
-     * Check removed status.
-     *
-     * @throws IllegalStateException If removed.
-     */
-    private void checkRemoved() throws IllegalStateException {
-        if (rmvd)
-            throw removedError();
-
-        if (rmvCheck) {
-            try {
-                rmvd = atomicView.get(key) == null;
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
-
-            rmvCheck = false;
-
-            if (rmvd) {
-                ctx.kernalContext().dataStructures().onRemoved(key, this);
-
-                throw removedError();
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-        this.ctx = kctx.cache().<GridCacheInternalKey, GridCacheAtomicStampedValue<T, S>>context().cacheContext(ctx.cacheId());
-        this.atomicView = ctx.cache();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) {
-        // No-op.
-    }
-
-    /**
      * @return Error.
      */
     private IllegalStateException removedError() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
index 7f331c3..72311c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
@@ -47,7 +47,8 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
 /**
  * Cache count down latch implementation.
  */
-public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatchEx, IgniteChangeGlobalStateSupport, Externalizable {
+public final class GridCacheCountDownLatchImpl extends AtomicDataStructureProxy<GridCacheCountDownLatchValue>
+    implements GridCacheCountDownLatchEx, IgniteChangeGlobalStateSupport, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -68,24 +69,6 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
             }
         };
 
-    /** Logger. */
-    private IgniteLogger log;
-
-    /** Latch name. */
-    private String name;
-
-    /** Removed flag.*/
-    private volatile boolean rmvd;
-
-    /** Latch key. */
-    private GridCacheInternalKey key;
-
-    /** Latch projection. */
-    private IgniteInternalCache<GridCacheInternalKey, GridCacheCountDownLatchValue> latchView;
-
-    /** Cache context. */
-    private GridCacheContext<GridCacheInternalKey, GridCacheCountDownLatchValue> ctx;
-
     /** Initial count. */
     private int initCnt;
 
@@ -126,30 +109,20 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
         GridCacheInternalKey key,
         IgniteInternalCache<GridCacheInternalKey, GridCacheCountDownLatchValue> latchView)
     {
+        super(name, key, latchView);
+
         assert name != null;
-        assert initCnt >= 0;
         assert key != null;
         assert latchView != null;
 
-        this.name = name;
         this.initCnt = initCnt;
         this.autoDel = autoDel;
-        this.key = key;
-        this.latchView = latchView;
-        this.ctx = latchView.context();
-
-        log = ctx.logger(getClass());
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() {
-        return name;
     }
 
     /** {@inheritDoc} */
     @Override public int count() {
         try {
-            GridCacheCountDownLatchValue latchVal = latchView.get(key);
+            GridCacheCountDownLatchValue latchVal = cacheView.get(key);
 
             return latchVal == null ? 0 : latchVal.get();
         }
@@ -225,26 +198,11 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onRemoved() {
-        return rmvd = true;
-    }
-
-    /** {@inheritDoc} */
     @Override public void needCheckNotRemoved() {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheInternalKey key() {
-        return key;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean removed() {
-        return rmvd;
-    }
-
-    /** {@inheritDoc} */
     @Override public void onUpdate(int cnt) {
         assert cnt >= 0;
 
@@ -280,8 +238,8 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
             try {
                 internalLatch = retryTopologySafe(new Callable<CountDownLatch>() {
                     @Override public CountDownLatch call() throws Exception {
-                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, latchView, PESSIMISTIC, REPEATABLE_READ)) {
-                            GridCacheCountDownLatchValue val = latchView.get(key);
+                        try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                            GridCacheCountDownLatchValue val = cacheView.get(key);
 
                             if (val == null) {
                                 if (log.isDebugEnabled())
@@ -334,17 +292,6 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
     }
 
     /** {@inheritDoc} */
-    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-        this.ctx = kctx.cache().<GridCacheInternalKey, GridCacheCountDownLatchValue>context().cacheContext(ctx.cacheId());
-        this.latchView = ctx.cache();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeObject(ctx.kernalContext());
         out.writeUTF(name);
@@ -402,8 +349,8 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
 
         /** {@inheritDoc} */
         @Override public Integer call() throws Exception {
-            try (GridNearTxLocal tx = CU.txStartInternal(ctx, latchView, PESSIMISTIC, REPEATABLE_READ)) {
-                GridCacheCountDownLatchValue latchVal = latchView.get(key);
+            try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                GridCacheCountDownLatchValue latchVal = cacheView.get(key);
 
                 if (latchVal == null) {
                     if (log.isDebugEnabled())
@@ -425,7 +372,7 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
 
                 latchVal.set(retVal);
 
-                latchView.put(key, latchVal);
+                cacheView.put(key, latchVal);
 
                 tx.commit();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
index fac7eaf..f677ff5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
@@ -67,31 +67,14 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
 /**
  * Cache reentrant lock implementation based on AbstractQueuedSynchronizer.
  */
-public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlobalStateSupport, Externalizable {
+public final class GridCacheLockImpl extends AtomicDataStructureProxy<GridCacheLockState>
+    implements GridCacheLockEx, IgniteChangeGlobalStateSupport, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** Deserialization stash. */
     private static final ThreadLocal<String> stash = new ThreadLocal<>();
 
-    /** Logger. */
-    private IgniteLogger log;
-
-    /** Reentrant lock name. */
-    private String name;
-
-    /** Removed flag. */
-    private volatile boolean rmvd;
-
-    /** Reentrant lock key. */
-    private GridCacheInternalKey key;
-
-    /** Reentrant lock projection. */
-    private IgniteInternalCache<GridCacheInternalKey, GridCacheLockState> lockView;
-
-    /** Cache context. */
-    private GridCacheContext<GridCacheInternalKey, GridCacheLockState> ctx;
-
     /** Initialization guard. */
     private final AtomicBoolean initGuard = new AtomicBoolean();
 
@@ -513,7 +496,7 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
         }
 
         final boolean isLocked() throws IgniteCheckedException {
-            return getState() != 0 || lockView.get(key).get() != 0;
+            return getState() != 0 || cacheView.get(key).get() != 0;
         }
 
         /**
@@ -524,8 +507,8 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
             try {
                 return retryTopologySafe(new Callable<Boolean>() {
                         @Override public Boolean call() throws Exception {
-                            try (GridNearTxLocal tx = CU.txStartInternal(ctx, lockView, PESSIMISTIC, REPEATABLE_READ)) {
-                                GridCacheLockState val = lockView.get(key);
+                            try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                                GridCacheLockState val = cacheView.get(key);
 
                                 if (val == null)
                                     throw new IgniteCheckedException("Failed to find reentrant lock with given name: " + name);
@@ -555,7 +538,7 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
 
                                         val.setChanged(true);
 
-                                        lockView.put(key, val);
+                                        cacheView.put(key, val);
 
                                         tx.commit();
 
@@ -609,8 +592,8 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
             try {
                 return retryTopologySafe(new Callable<Boolean>() {
                         @Override public Boolean call() throws Exception {
-                            try (GridNearTxLocal tx = CU.txStartInternal(ctx, lockView, PESSIMISTIC, REPEATABLE_READ)) {
-                                GridCacheLockState val = lockView.get(key);
+                            try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                                GridCacheLockState val = cacheView.get(key);
 
                                 if (val == null)
                                     throw new IgniteCheckedException("Failed to find reentrant lock with given name: " + name);
@@ -622,7 +605,7 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
 
                                     val.setChanged(false);
 
-                                    lockView.put(key, val);
+                                    cacheView.put(key, val);
 
                                     tx.commit();
 
@@ -640,7 +623,7 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
 
                                         nodes.removeLastOccurrence(thisNode);
 
-                                        lockView.put(key, val);
+                                        cacheView.put(key, val);
 
                                         tx.commit();
 
@@ -705,8 +688,8 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
             try {
                 return retryTopologySafe(new Callable<Boolean>() {
                         @Override public Boolean call() throws Exception {
-                            try (GridNearTxLocal tx = CU.txStartInternal(ctx, lockView, PESSIMISTIC, REPEATABLE_READ)) {
-                                GridCacheLockState val = lockView.get(key);
+                            try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                                GridCacheLockState val = cacheView.get(key);
 
                                 if (val == null)
                                     throw new IgniteCheckedException("Failed to find reentrant lock with given name: " + name);
@@ -797,7 +780,7 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
 
                                 val.setConditionMap(condMap);
 
-                                lockView.put(key, val);
+                                cacheView.put(key, val);
 
                                 tx.commit();
 
@@ -1054,16 +1037,7 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
     public GridCacheLockImpl(String name,
         GridCacheInternalKey key,
         IgniteInternalCache<GridCacheInternalKey, GridCacheLockState> lockView) {
-        assert name != null;
-        assert key != null;
-        assert lockView != null;
-
-        this.name = name;
-        this.key = key;
-        this.lockView = lockView;
-        this.ctx = lockView.context();
-
-        log = ctx.logger(getClass());
+        super(name, key, lockView);
     }
 
     /**
@@ -1074,8 +1048,8 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
             try {
                 sync = retryTopologySafe(new Callable<Sync>() {
                         @Override public Sync call() throws Exception {
-                            try (GridNearTxLocal tx = CU.txStartInternal(ctx, lockView, PESSIMISTIC, REPEATABLE_READ)) {
-                                GridCacheLockState val = lockView.get(key);
+                            try (GridNearTxLocal tx = CU.txStartInternal(ctx, cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                                GridCacheLockState val = cacheView.get(key);
 
                                 if (val == null) {
                                     if (log.isDebugEnabled())
@@ -1184,11 +1158,6 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
     }
 
     /** {@inheritDoc} */
-    @Override public String name() {
-        return name;
-    }
-
-    /** {@inheritDoc} */
     @Override public void lock() {
         ctx.kernalContext().gateway().readLock();
 
@@ -1454,34 +1423,8 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheInternalKey key() {
-        return key;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean removed() {
-        return rmvd;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean onRemoved() {
-        return rmvd = true;
-    }
-
-    /** {@inheritDoc} */
     @Override public void needCheckNotRemoved() {
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-        this.ctx = kctx.cache().<GridCacheInternalKey, GridCacheLockState>context().cacheContext(ctx.cacheId());
-        this.lockView = ctx.cache();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) {
-        // No-op.
+        // no-op
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheRemovable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheRemovable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheRemovable.java
index e222e57..d26a153 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheRemovable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheRemovable.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.datastructures;
 
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+
 /**
  * Provides callback for marking object as removed.
  */
@@ -32,4 +34,8 @@ public interface GridCacheRemovable {
      *
      */
     public void needCheckNotRemoved();
+
+    public void suspend();
+
+    public void restart(IgniteInternalCache cache);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
index 4abefc9..9502a6f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
@@ -59,7 +59,8 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  * acquired by the failing node. In case this parameter is false, IgniteInterruptedException is called on every node
  * waiting on this semaphore.
  */
-public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, IgniteChangeGlobalStateSupport, Externalizable {
+public final class GridCacheSemaphoreImpl extends AtomicDataStructureProxy<GridCacheSemaphoreState>
+    implements GridCacheSemaphoreEx, IgniteChangeGlobalStateSupport, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -71,24 +72,6 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
             }
         };
 
-    /** Logger. */
-    private IgniteLogger log;
-
-    /** Semaphore name. */
-    private String name;
-
-    /** Removed flag. */
-    private volatile boolean rmvd;
-
-    /** Semaphore key. */
-    private GridCacheInternalKey key;
-
-    /** Semaphore projection. */
-    private IgniteInternalCache<GridCacheInternalKey, GridCacheSemaphoreState> semView;
-
-    /** Cache context. */
-    private GridCacheContext<GridCacheInternalKey, GridCacheSemaphoreState> ctx;
-
     /** Initialization guard. */
     private final AtomicBoolean initGuard = new AtomicBoolean();
 
@@ -290,10 +273,10 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
                 return retryTopologySafe(new Callable<Boolean>() {
                     @Override public Boolean call() throws Exception {
                         try (GridNearTxLocal tx = CU.txStartInternal(ctx,
-                            semView,
+                            cacheView,
                             PESSIMISTIC, REPEATABLE_READ)
                         ) {
-                            GridCacheSemaphoreState val = semView.get(key);
+                            GridCacheSemaphoreState val = cacheView.get(key);
 
                             if (val == null)
                                 throw new IgniteCheckedException("Failed to find semaphore with given name: " +
@@ -328,7 +311,7 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
 
                                 val.setCount(newVal);
 
-                                semView.put(key, val);
+                                cacheView.put(key, val);
 
                                 tx.commit();
                             }
@@ -370,10 +353,10 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
                     @Override public Boolean call() throws Exception {
                         try (
                             GridNearTxLocal tx = CU.txStartInternal(ctx,
-                                semView,
+                                cacheView,
                                 PESSIMISTIC, REPEATABLE_READ)
                         ) {
-                            GridCacheSemaphoreState val = semView.get(key);
+                            GridCacheSemaphoreState val = cacheView.get(key);
 
                             if (val == null)
                                 throw new IgniteCheckedException("Failed to find semaphore with given name: " +
@@ -391,7 +374,7 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
                             if (broken) {
                                 val.setBroken(true);
 
-                                semView.put(key, val);
+                                cacheView.put(key, val);
 
                                 tx.commit();
 
@@ -415,7 +398,7 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
 
                             val.setWaiters(map);
 
-                            semView.put(key, val);
+                            cacheView.put(key, val);
 
                             sync.nodeMap = map;
 
@@ -457,16 +440,7 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
         GridCacheInternalKey key,
         IgniteInternalCache<GridCacheInternalKey, GridCacheSemaphoreState> semView
     ) {
-        assert name != null;
-        assert key != null;
-        assert semView != null;
-
-        this.name = name;
-        this.key = key;
-        this.semView = semView;
-        this.ctx = semView.context();
-
-        log = ctx.logger(getClass());
+        super(name, key, semView);
     }
 
     /**
@@ -478,8 +452,8 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
                 sync = retryTopologySafe(new Callable<Sync>() {
                     @Override public Sync call() throws Exception {
                         try (GridNearTxLocal tx = CU.txStartInternal(ctx,
-                            semView, PESSIMISTIC, REPEATABLE_READ)) {
-                            GridCacheSemaphoreState val = semView.get(key);
+                            cacheView, PESSIMISTIC, REPEATABLE_READ)) {
+                            GridCacheSemaphoreState val = cacheView.get(key);
 
                             if (val == null) {
                                 if (log.isDebugEnabled())
@@ -521,26 +495,6 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
     }
 
     /** {@inheritDoc} */
-    @Override public String name() {
-        return name;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheInternalKey key() {
-        return key;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean removed() {
-        return rmvd;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean onRemoved() {
-        return rmvd = true;
-    }
-
-    /** {@inheritDoc} */
     @Override public void onUpdate(GridCacheSemaphoreState val) {
         if (sync == null)
             return;
@@ -722,9 +676,9 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
                 @Override public Integer call() throws Exception {
                     try (
                         GridNearTxLocal tx = CU.txStartInternal(ctx,
-                            semView, PESSIMISTIC, REPEATABLE_READ)
+                            cacheView, PESSIMISTIC, REPEATABLE_READ)
                     ) {
-                        GridCacheSemaphoreState val = semView.get(key);
+                        GridCacheSemaphoreState val = cacheView.get(key);
 
                         if (val == null)
                             throw new IgniteException("Failed to find semaphore with given name: " + name);
@@ -962,17 +916,6 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
     }
 
     /** {@inheritDoc} */
-    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-        this.ctx = kctx.cache().<GridCacheInternalKey, GridCacheSemaphoreState>context().cacheContext(ctx.cacheId());
-        this.semView = ctx.cache();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeObject(ctx.kernalContext());
         out.writeUTF(name);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f08c9d3a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/AtomicCacheAffinityConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/AtomicCacheAffinityConfigurationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/AtomicCacheAffinityConfigurationTest.java
index 623b076..3fd4e1f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/AtomicCacheAffinityConfigurationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/AtomicCacheAffinityConfigurationTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.datastructures.AtomicDataStructureProxy;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -58,7 +59,7 @@ public class AtomicCacheAffinityConfigurationTest extends GridCommonAbstractTest
 
                 IgniteAtomicLong atomic = igniteEx.atomicLong("test", 0, true);
 
-                GridCacheContext cctx = GridTestUtils.getFieldValue(atomic, "ctx");
+                GridCacheContext cctx = GridTestUtils.getFieldValue(atomic, AtomicDataStructureProxy.class, "ctx");
 
                 AffinityFunction aff = cctx.config().getAffinity();
 
@@ -90,7 +91,7 @@ public class AtomicCacheAffinityConfigurationTest extends GridCommonAbstractTest
 
                 IgniteAtomicLong atomic = igniteEx.atomicLong("test", 0, true);
 
-                GridCacheContext cctx = GridTestUtils.getFieldValue(atomic, "ctx");
+                GridCacheContext cctx = GridTestUtils.getFieldValue(atomic, AtomicDataStructureProxy.class, "ctx");
 
                 TestAffinityFunction aff = (TestAffinityFunction) cctx.config().getAffinity();
 
@@ -122,7 +123,7 @@ public class AtomicCacheAffinityConfigurationTest extends GridCommonAbstractTest
 
                 IgniteAtomicLong atomic = igniteEx.atomicLong("test", 0, true);
 
-                GridCacheContext cctx = GridTestUtils.getFieldValue(atomic, "ctx");
+                GridCacheContext cctx = GridTestUtils.getFieldValue(atomic, AtomicDataStructureProxy.class, "ctx");
 
                 AffinityFunction aff = cctx.config().getAffinity();
 


[10/11] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-zk

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


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

Branch: refs/heads/ignite-zk
Commit: 017f8cbbde00437bef1a7064d56a4b5b360cdd48
Parents: 0bddcde ff3712c
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 5 10:11:16 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 5 10:11:16 2017 +0300

----------------------------------------------------------------------
 .../internal/jdbc2/JdbcLocalCachesSelfTest.java |  28 +
 .../processors/cache/GridCacheProcessor.java    |  34 +-
 .../cache/persistence/tree/BPlusTree.java       | 141 +++-
 .../AtomicDataStructureProxy.java               | 189 +++++
 .../datastructures/DataStructuresProcessor.java |  20 +
 .../datastructures/GridCacheAtomicLongImpl.java | 120 +--
 .../GridCacheAtomicReferenceImpl.java           | 114 +--
 .../GridCacheAtomicSequenceImpl.java            | 113 +--
 .../GridCacheAtomicStampedImpl.java             | 115 +--
 .../GridCacheCountDownLatchImpl.java            |  73 +-
 .../datastructures/GridCacheLockImpl.java       |  91 +--
 .../datastructures/GridCacheRemovable.java      |   6 +
 .../datastructures/GridCacheSemaphoreImpl.java  |  85 +-
 .../AtomicCacheAffinityConfigurationTest.java   |   7 +-
 .../processors/database/BPlusTreeSelfTest.java  | 819 ++++++++++++++++++-
 .../query/h2/database/H2TreeIndex.java          |  92 ++-
 .../cache/index/AbstractSchemaSelfTest.java     | 248 ++----
 ...ynamicColumnsAbstractConcurrentSelfTest.java |  40 +-
 .../cache/index/DynamicColumnsAbstractTest.java |  24 +-
 .../DynamicIndexAbstractBasicSelfTest.java      |  58 +-
 .../DynamicIndexAbstractConcurrentSelfTest.java |   5 +-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  |  15 +-
 ...lexClientAtomicPartitionedNoBackupsTest.java |  34 +
 ...exingComplexClientAtomicPartitionedTest.java |   2 +-
 ...dexingComplexClientAtomicReplicatedTest.java |   2 +-
 ...ntTransactionalPartitionedNoBackupsTest.java |  34 +
 ...mplexClientTransactionalPartitionedTest.java |   2 +-
 ...omplexClientTransactionalReplicatedTest.java |   2 +-
 ...lexServerAtomicPartitionedNoBackupsTest.java |  34 +
 ...exingComplexServerAtomicPartitionedTest.java |   2 +-
 ...dexingComplexServerAtomicReplicatedTest.java |   2 +-
 ...erTransactionalPartitionedNoBackupsTest.java |  34 +
 ...mplexServerTransactionalPartitionedTest.java |   2 +-
 ...omplexServerTransactionalReplicatedTest.java |   2 +-
 .../index/H2DynamicIndexingComplexTest.java     |  22 +-
 .../cache/index/H2DynamicTableSelfTest.java     |  57 +-
 .../query/IgniteSqlSegmentedIndexSelfTest.java  | 124 ++-
 .../cpp/odbc-test/src/sql_get_info_test.cpp     |   4 +-
 .../Apache.Ignite.Core.Tests.DotNetCore.csproj  |   2 +
 .../Apache.Ignite.Core.Tests.csproj             |  13 +
 .../ApiParity/BinaryParityTest.cs               |  48 ++
 .../ApiParity/CacheAffinityParityTest.cs        |  39 +
 .../ApiParity/CacheParityTest.cs                |  76 ++
 .../ApiParity/ClusterParityTest.cs              |  50 ++
 .../ApiParity/ComputeParityTest.cs              |  51 ++
 .../ApiParity/EventsParityTest.cs               |  50 ++
 .../ApiParity/IgniteParityTest.cs               |  79 ++
 .../ApiParity/MessagingParityTest.cs            |  39 +
 .../ApiParity/ParityTest.cs                     |  90 +-
 .../ApiParity/ServicesParityTest.cs             |  39 +
 .../ApiParity/StreamerParityTest.cs             |  54 ++
 .../ApiParity/TransactionsParityTest.cs         |  39 +
 .../Query/Linq/CacheLinqTest.Introspection.cs   |   8 +-
 .../Client/Cache/CreateCacheTest.cs             |   1 -
 .../Client/Cache/LinqTest.cs                    | 117 +++
 .../Client/Cache/SqlQueryTest.cs                |  43 +-
 .../Client/Cache/SqlQueryTestBase.cs            |  68 ++
 .../Client/RawSocketTest.cs                     |   1 -
 .../TestUtils.Windows.cs                        |   2 -
 .../Impl/Cache/ICacheInternal.cs                |   6 +
 .../Impl/Client/Cache/CacheClient.cs            |  64 +-
 .../Client/Cache/Query/ClientQueryCursorBase.cs |   4 +-
 .../Apache.Ignite.Linq.csproj                   |   3 +-
 .../CacheClientLinqExtensions.cs                | 131 +++
 .../Apache.Ignite.Linq/CacheExtensions.cs       | 195 -----
 .../Apache.Ignite.Linq/CacheLinqExtensions.cs   | 197 +++++
 .../Apache.Ignite.Linq/ICacheQueryable.cs       |   1 +
 .../Impl/CacheFieldsQueryProvider.cs            |   2 +-
 .../Apache.Ignite.Linq/Impl/CacheQueryable.cs   |   8 +-
 .../Impl/CacheQueryableBase.cs                  |   1 +
 .../dotnet/Apache.Ignite/IgniteRunner.cs        |   1 -
 modules/web-console/backend/middlewares/api.js  |   5 +
 .../frontend/app/browserUpdate/index.js         |  34 +
 .../frontend/app/browserUpdate/style.scss       |  36 +
 .../frontend/app/helpers/jade/mixins.pug        |   2 +-
 modules/web-console/frontend/package.json       |   5 +-
 .../frontend/webpack/webpack.common.js          |   3 +-
 77 files changed, 3157 insertions(+), 1246 deletions(-)
----------------------------------------------------------------------



[03/11] ignite git commit: IGNITE-7001: SQL: now DDL tests mostly use public API. This closes #3096.

Posted by sb...@apache.org.
IGNITE-7001: SQL: now DDL tests mostly use public API. This closes #3096.


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

Branch: refs/heads/ignite-zk
Commit: 8292335e419a19a90a0bfcd302a937cbf7440f21
Parents: 1f43375
Author: Alexander Paschenko <al...@gmail.com>
Authored: Mon Dec 4 15:14:12 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 4 15:14:12 2017 +0300

----------------------------------------------------------------------
 .../cache/index/AbstractSchemaSelfTest.java     | 248 ++++++-------------
 ...ynamicColumnsAbstractConcurrentSelfTest.java |  40 +--
 .../cache/index/DynamicColumnsAbstractTest.java |  24 +-
 .../DynamicIndexAbstractBasicSelfTest.java      |  58 ++---
 .../DynamicIndexAbstractConcurrentSelfTest.java |   5 +-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  |  15 +-
 .../cache/index/H2DynamicTableSelfTest.java     |  57 +++--
 7 files changed, 188 insertions(+), 259 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8292335e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
index 01d1f36..5310f46 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
@@ -17,21 +17,28 @@
 
 package org.apache.ignite.internal.processors.cache.index;
 
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
-import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor;
+import org.apache.ignite.internal.processors.port.GridPortRecord;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl;
@@ -39,25 +46,17 @@ import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.GridStringBuilder;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
 /**
  * Tests for dynamic schema changes.
  */
 @SuppressWarnings("unchecked")
-public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
+public abstract class AbstractSchemaSelfTest extends GridCommonAbstractTest {
     /** Cache. */
     protected static final String CACHE_NAME = "cache";
 
@@ -105,7 +104,7 @@ public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
      * @param tblName Table name.
      * @return Type.
      */
-    protected static QueryTypeDescriptorImpl typeExisting(IgniteEx node, String cacheName, String tblName) {
+    static QueryTypeDescriptorImpl typeExisting(IgniteEx node, String cacheName, String tblName) {
         QueryTypeDescriptorImpl res = type(node, cacheName, tblName);
 
         assertNotNull(res);
@@ -155,107 +154,61 @@ public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
      * @param inlineSize Inline size.
      * @param fields Fields.
      */
-    protected static void assertIndex(String cacheName, String tblName, String idxName,
-        int inlineSize, IgniteBiTuple<String, Boolean>... fields) {
-        assertIndex(cacheName, false, tblName, idxName, inlineSize, fields);
-    }
-
-    /**
-     * Assert index state on all nodes.
-     *
-     * @param cacheName Cache name.
-     * @param checkNonAffinityNodes Whether existence of {@link GridQueryIndexDescriptor} must be checked on non
-     *     affinity nodes as well.
-     * @param tblName Table name.
-     * @param idxName Index name.
-     * @param inlineSize Inline size.
-     * @param fields Fields.
-     */
-    protected static void assertIndex(String cacheName, boolean checkNonAffinityNodes, String tblName, String idxName,
+    static void assertIndex(String cacheName, String tblName, String idxName,
         int inlineSize, IgniteBiTuple<String, Boolean>... fields) {
         for (Ignite node : Ignition.allGrids())
-            assertIndex(node, checkNonAffinityNodes, cacheName, tblName, idxName, inlineSize, fields);
+            assertIndex(node, cacheName, tblName, idxName, inlineSize, fields);
     }
 
     /**
      * Assert index state on particular node.
      *
      * @param node Node.
-     * @param checkNonAffinityNode Whether existence of {@link GridQueryIndexDescriptor} must be checked regardless of
-     * whether this node is affinity node or not.
      * @param cacheName Cache name.
      * @param tblName Table name.
      * @param idxName Index name.
      * @param inlineSize Inline size.
      * @param fields Fields.
      */
-    protected static void assertIndex(Ignite node, boolean checkNonAffinityNode, String cacheName, String tblName,
+    static void assertIndex(Ignite node, String cacheName, String tblName,
         String idxName, int inlineSize, IgniteBiTuple<String, Boolean>... fields) {
-        IgniteEx node0 = (IgniteEx)node;
-
-        assertIndexDescriptor(node0, cacheName, tblName, idxName, fields);
-
-        if (checkNonAffinityNode || affinityNode(node0, cacheName)) {
-            QueryTypeDescriptorImpl typeDesc = typeExisting(node0, cacheName, tblName);
+        node.cache(cacheName);
 
-            assertIndex(typeDesc, idxName, inlineSize, fields);
-        }
-    }
-
-    /**
-     * Make sure index exists in cache descriptor.
-     *
-     * @param node Node.
-     * @param cacheName Cache name.
-     * @param tblName Table name.
-     * @param idxName Index name.
-     * @param fields Fields.
-     */
-    protected static void assertIndexDescriptor(IgniteEx node, String cacheName, String tblName, String idxName,
-        IgniteBiTuple<String, Boolean>... fields) {
-        awaitCompletion();
-
-        DynamicCacheDescriptor desc = node.context().cache().cacheDescriptor(cacheName);
-
-        assert desc != null;
-
-        for (QueryEntity entity : desc.schema().entities()) {
-            if (F.eq(tblName, entity.getTableName())) {
-                for (QueryIndex idx : entity.getIndexes()) {
-                    if (F.eq(QueryUtils.indexName(entity, idx), idxName)) {
-                        LinkedHashMap<String, Boolean> idxFields = idx.getFields();
-
-                        assertEquals(idxFields.size(), fields.length);
-
-                        int i = 0;
-
-                        for (String idxField : idxFields.keySet()) {
-                            assertEquals(idxField.toLowerCase(), fields[i].get1().toLowerCase());
-                            assertEquals(idxFields.get(idxField), fields[i].get2());
+        IgniteEx node0 = (IgniteEx)node;
 
-                            i++;
-                        }
+        ArrayList<IgniteBiTuple<String, Boolean>> res = new ArrayList<>();
 
-                        return;
+        try {
+            try (Connection c = connect(node0)) {
+                try (ResultSet rs = c.getMetaData().getIndexInfo(null, cacheName, tblName, false, false)) {
+                    while (rs.next()) {
+                        if (F.eq(idxName, rs.getString("INDEX_NAME")))
+                            res.add(new T2<>(rs.getString("COLUMN_NAME"), F.eq("A", rs.getString("ASC_OR_DESC"))));
                     }
                 }
             }
+
+            assertTrue("Index not found: " + idxName, res.size() > 0);
+
+            assertEquals(Arrays.asList(fields), res);
         }
+        catch (SQLException e) {
+            throw new AssertionError(e);
+        }
+
+        // Also, let's check internal stuff not visible via JDBC - like inline size.
+        QueryTypeDescriptorImpl typeDesc = typeExisting(node0, cacheName, tblName);
 
-        fail("Index not found [node=" + node.name() + ", cacheName=" + cacheName + ", tlbName=" + tblName +
-            ", idxName=" + idxName + ']');
+        assertInternalIndexParams(typeDesc, idxName, inlineSize);
     }
 
     /**
-     * Assert index state.
-     *
+     * Assert index details not available via JDBC.
      * @param typeDesc Type descriptor.
      * @param idxName Index name.
      * @param inlineSize Inline size.
-     * @param fields Fields (order is important).
      */
-    protected static void assertIndex(QueryTypeDescriptorImpl typeDesc, String idxName,
-        int inlineSize, IgniteBiTuple<String, Boolean>... fields) {
+    private static void assertInternalIndexParams(QueryTypeDescriptorImpl typeDesc, String idxName, int inlineSize) {
         QueryIndexDescriptorImpl idxDesc = typeDesc.index(idxName);
 
         assertNotNull(idxDesc);
@@ -264,22 +217,32 @@ public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
         assertEquals(typeDesc, idxDesc.typeDescriptor());
         assertEquals(QueryIndexType.SORTED, idxDesc.type());
         assertEquals(inlineSize, idxDesc.inlineSize());
+    }
 
-        List<String> fieldNames = new ArrayList<>(idxDesc.fields());
+    /**
+     * @param node Node to connect to.
+     * @return Thin JDBC connection to specified node.
+     */
+    static Connection connect(IgniteEx node) {
+        Collection<GridPortRecord> recs = node.context().ports().records();
 
-        assertEquals(fields.length, fieldNames.size());
+        GridPortRecord cliLsnrRec = null;
 
-        for (int i = 0; i < fields.length; i++) {
-            String expFieldName = fields[i].get1();
-            boolean expFieldAsc = fields[i].get2();
+        for (GridPortRecord rec : recs) {
+            if (rec.clazz() == ClientListenerProcessor.class) {
+                cliLsnrRec = rec;
 
-            assertEquals("Index field mismatch [pos=" + i + ", expField=" + expFieldName + ", actualField=" +
-                fieldNames.get(i) + ']', expFieldName.toLowerCase(), fieldNames.get(i).toLowerCase());
+                break;
+            }
+        }
 
-            boolean fieldAsc = !idxDesc.descending(expFieldName);
+        assertNotNull(cliLsnrRec);
 
-            assertEquals("Index field sort mismatch [pos=" + i + ", field=" + expFieldName +
-                ", expAsc=" + expFieldAsc + ", actualAsc=" + fieldAsc + ']', expFieldAsc, fieldAsc);
+        try {
+            return DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1:" + cliLsnrRec.port());
+        }
+        catch (SQLException e) {
+            throw new AssertionError(e);
         }
     }
 
@@ -290,7 +253,7 @@ public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
      * @param tblName Table name.
      * @param idxName Index name.
      */
-    protected static void assertNoIndex(String cacheName, String tblName, String idxName) {
+    static void assertNoIndex(String cacheName, String tblName, String idxName) {
         for (Ignite node : Ignition.allGrids())
             assertNoIndex(node, cacheName, tblName, idxName);
     }
@@ -303,82 +266,25 @@ public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
      * @param tblName Table name.
      * @param idxName Index name.
      */
-    protected static void assertNoIndex(Ignite node, String cacheName, String tblName, String idxName) {
-        IgniteEx node0 = (IgniteEx)node;
-
-        assertNoIndexDescriptor(node0, cacheName, idxName);
+    static void assertNoIndex(Ignite node, String cacheName, String tblName, String idxName) {
+        node.cache(cacheName);
 
-        if (affinityNode(node0, cacheName)) {
-            QueryTypeDescriptorImpl typeDesc = typeExisting(node0, cacheName, tblName);
-
-            assertNoIndex(typeDesc, idxName);
-        }
-    }
-
-    /**
-     * Assert index doesn't exist in particular node's cache descriptor.
-     *
-     * @param node Node.
-     * @param cacheName Cache name.
-     * @param idxName Index name.
-     */
-    protected static void assertNoIndexDescriptor(IgniteEx node, String cacheName, String idxName) {
-        awaitCompletion();
-
-        DynamicCacheDescriptor desc = node.context().cache().cacheDescriptor(cacheName);
-
-        if (desc == null)
-            return;
-
-        for (QueryEntity entity : desc.schema().entities()) {
-            for (QueryIndex idx : entity.getIndexes()) {
-                if (F.eq(idxName, QueryUtils.indexName(entity, idx)))
-                    fail("Index exists: " + idxName);
-            }
-        }
-    }
-
-    /**
-     * Await completion (hopefully) of pending operations.
-     */
-    private static void awaitCompletion() {
         try {
-            U.sleep(100);
+            try (Connection c = connect((IgniteEx)node)) {
+                try (ResultSet rs = c.getMetaData().getIndexInfo(null, cacheName, tblName, false, false)) {
+                    while (rs.next()) {
+                        assertFalse("Index exists, although shouldn't: " + tblName + '.' + idxName,
+                            F.eq(idxName, rs.getString("INDEX_NAME")));
+                    }
+                }
+            }
         }
-        catch (IgniteInterruptedCheckedException e) {
-            fail();
+        catch (SQLException e) {
+            throw new AssertionError(e);
         }
     }
 
     /**
-     * Assert index doesn't exist.
-     *
-     * @param typeDesc Type descriptor.
-     * @param idxName Index name.
-     */
-    protected static void assertNoIndex(QueryTypeDescriptorImpl typeDesc, String idxName) {
-        assertNull(typeDesc.index(idxName));
-    }
-
-    /**
-     * Check whether this is affinity node for cache.
-     *
-     * @param node Node.
-     * @param cacheName Cache name.
-     * @return {@code True} if affinity node.
-     */
-    private static boolean affinityNode(IgniteEx node, String cacheName) {
-        if (node.configuration().isClientMode())
-            return false;
-
-        DynamicCacheDescriptor cacheDesc = node.context().cache().cacheDescriptor(cacheName);
-
-        IgnitePredicate<ClusterNode> filter = cacheDesc.cacheConfiguration().getNodeFilter();
-
-        return filter == null || filter.apply(node.localNode());
-    }
-
-    /**
      * Get table name for class.
      *
      * @param cls Class.
@@ -394,7 +300,7 @@ public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
      * @param cls Class.
      * @return Table name.
      */
-    protected static String tableName(Class cls) {
+    static String tableName(Class cls) {
         return QueryUtils.normalizeObjectName(typeName(cls), true);
     }
 
@@ -426,7 +332,7 @@ public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
      * @param node Node.
      * @return Query processor.
      */
-    protected static GridQueryProcessor queryProcessor(Ignite node) {
+    static GridQueryProcessor queryProcessor(Ignite node) {
         return ((IgniteEx)node).context().query();
     }
 
@@ -448,7 +354,7 @@ public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
      * @return Field.
      */
     protected static IgniteBiTuple<String, Boolean> field(String name, boolean asc) {
-        return F.t(name, asc);
+        return F.t(name.toUpperCase(), asc);
     }
 
     /**
@@ -470,7 +376,7 @@ public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
      * @param parallel Parallelism level.
      * @throws Exception If failed.
      */
-    protected void dynamicIndexCreate(Ignite node, String cacheName, String tblName, QueryIndex idx,
+    void dynamicIndexCreate(Ignite node, String cacheName, String tblName, QueryIndex idx,
         boolean ifNotExists, int parallel)
         throws Exception {
         GridStringBuilder sql = new SB("CREATE INDEX ")

http://git-wip-us.apache.org/repos/asf/ignite/blob/8292335e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
index 0263f5c..145947a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
@@ -147,15 +147,15 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         CountDownLatch finishLatch = new CountDownLatch(2);
 
         // Start servers.
-        Ignite srv1 = ignitionStart(serverConfiguration(1), null);
-        Ignite srv2 = ignitionStart(serverConfiguration(2), null);
-        Ignite srv3 = ignitionStart(serverConfiguration(3, true), finishLatch);
+        IgniteEx srv1 = ignitionStart(serverConfiguration(1), null);
+        IgniteEx srv2 = ignitionStart(serverConfiguration(2), null);
+        IgniteEx srv3 = ignitionStart(serverConfiguration(3, true), finishLatch);
 
         UUID srv1Id = srv1.cluster().localNode().id();
         UUID srv2Id = srv2.cluster().localNode().id();
 
         // Start client which will execute operations.
-        IgniteEx cli = (IgniteEx)ignitionStart(clientConfiguration(4), finishLatch);
+        IgniteEx cli = ignitionStart(clientConfiguration(4), finishLatch);
 
         createSqlCache(cli);
 
@@ -176,7 +176,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         colFut1.get();
 
         // Port number is for srv2.
-        checkTableState(QueryUtils.DFLT_SCHEMA, TBL_NAME, 10801, c("age", Integer.class.getName()));
+        checkTableState(srv2, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("age", Integer.class.getName()));
 
         // Test migration from normal server to non-affinity server.
         idxLatch = blockIndexing(srv2Id);
@@ -196,7 +196,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         srv3.cache(QueryUtils.createTableCacheName(QueryUtils.DFLT_SCHEMA, "PERSON"));
 
         // Port number is for srv3.
-        checkTableState(QueryUtils.DFLT_SCHEMA, TBL_NAME, 10802, c("city", String.class.getName()));
+        checkTableState(srv3, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("city", String.class.getName()));
     }
 
     /**
@@ -208,7 +208,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         // 7 nodes * 2 columns = 14 latch countdowns.
         CountDownLatch finishLatch = new CountDownLatch(14);
 
-        Ignite srv1 = ignitionStart(serverConfiguration(1), finishLatch);
+        IgniteEx srv1 = ignitionStart(serverConfiguration(1), finishLatch);
 
         ignitionStart(serverConfiguration(2), finishLatch);
         ignitionStart(serverConfiguration(3, true), finishLatch);
@@ -244,7 +244,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         U.await(finishLatch);
 
-        checkTableState(QueryUtils.DFLT_SCHEMA, TBL_NAME, c1, c2);
+        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c1, c2);
     }
 
     /**
@@ -255,7 +255,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     public void testNodeJoinOnPendingOperation() throws Exception {
         CountDownLatch finishLatch = new CountDownLatch(4);
 
-        Ignite srv1 = ignitionStart(serverConfiguration(1), finishLatch);
+        IgniteEx srv1 = ignitionStart(serverConfiguration(1), finishLatch);
 
         createSqlCache(srv1);
 
@@ -281,7 +281,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         U.await(finishLatch);
 
-        checkTableState(QueryUtils.DFLT_SCHEMA, TBL_NAME, c);
+        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c);
     }
 
     /**
@@ -293,7 +293,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         CountDownLatch finishLatch = new CountDownLatch(4);
 
         // Start several nodes.
-        Ignite srv1 = ignitionStart(serverConfiguration(1), finishLatch);
+        IgniteEx srv1 = ignitionStart(serverConfiguration(1), finishLatch);
         ignitionStart(serverConfiguration(2), finishLatch);
         ignitionStart(serverConfiguration(3), finishLatch);
         ignitionStart(serverConfiguration(4), finishLatch);
@@ -340,7 +340,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         finishLatch.await();
 
         // Make sure new column is there.
-        checkTableState(QueryUtils.DFLT_SCHEMA, TBL_NAME, c("v", Integer.class.getName()));
+        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("v", Integer.class.getName()));
 
         run(srv1, "update person set \"v\" = case when mod(id, 2) <> 0 then substring(name, 7, length(name) - 6) " +
             "else null end");
@@ -419,7 +419,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      */
     public void testConcurrentRebalance() throws Exception {
         // Start cache and populate it with data.
-        Ignite srv1 = ignitionStart(serverConfiguration(1));
+        IgniteEx srv1 = ignitionStart(serverConfiguration(1));
         Ignite srv2 = ignitionStart(serverConfiguration(2));
 
         createSqlCache(srv1);
@@ -454,7 +454,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         // Validate index state.
         idxFut.get();
 
-        checkTableState(QueryUtils.DFLT_SCHEMA, TBL_NAME, c);
+        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c);
     }
 
     /**
@@ -636,7 +636,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      */
     private void checkClientReconnect(final boolean restartCache, boolean dynamicCache) throws Exception {
         // Start complex topology.
-        final Ignite srv = ignitionStart(serverConfiguration(1));
+        final IgniteEx srv = ignitionStart(serverConfiguration(1));
         ignitionStart(serverConfiguration(2));
         ignitionStart(serverConfiguration(3, true));
 
@@ -660,7 +660,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
             }
         });
 
-        checkTableState(schemaName, TBL_NAME, cols);
+        checkTableState(srv, schemaName, TBL_NAME, cols);
     }
 
     /**
@@ -724,7 +724,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         ignitionStart(serverConfiguration(2));
         ignitionStart(serverConfiguration(3, true));
 
-        final Ignite cli = ignitionStart(clientConfiguration(4));
+        final IgniteEx cli = ignitionStart(clientConfiguration(4));
 
         createSqlCache(cli);
 
@@ -849,7 +849,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         idxQry += ')';
 
-        checkTableState(QueryUtils.DFLT_SCHEMA, TBL_NAME, expCols);
+        checkTableState(cli, QueryUtils.DFLT_SCHEMA, TBL_NAME, expCols);
 
         put(cli, 0, 500);
 
@@ -987,7 +987,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      * @param cfg Node configuration.
      * @return New node.
      */
-    private static Ignite ignitionStart(IgniteConfiguration cfg) {
+    private static IgniteEx ignitionStart(IgniteConfiguration cfg) {
         return ignitionStart(cfg, null);
     }
 
@@ -997,7 +997,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      * @param latch Latch to await for ultimate completion of DDL operations.
      * @return New node.
      */
-    private static Ignite ignitionStart(IgniteConfiguration cfg, final CountDownLatch latch) {
+    private static IgniteEx ignitionStart(IgniteConfiguration cfg, final CountDownLatch latch) {
         // Have to do this for each starting node - see GridQueryProcessor ctor, it nulls
         // idxCls static field on each call.
         GridQueryProcessor.idxCls = BlockingIndexing.class;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8292335e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
index 34be76e..611f857 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.index;
 
 import java.sql.Connection;
-import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -32,7 +31,6 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.ClientConnectorConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -47,6 +45,8 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.h2.value.DataType;
 
+import static org.apache.ignite.internal.processors.cache.index.AbstractSchemaSelfTest.connect;
+
 /**
  * Common stuff for dynamic columns tests.
  */
@@ -62,25 +62,16 @@ public abstract class DynamicColumnsAbstractTest extends GridCommonAbstractTest
 
     /**
      * Check that given columns are seen by client.
+     * @param node Node to check.
      * @param schemaName Schema name to look for the table in.
      * @param tblName Table name to check.
      * @param cols Columns whose presence must be checked.
      */
-    static void checkTableState(String schemaName, String tblName, QueryField... cols) throws SQLException {
-        checkTableState(schemaName, tblName, ClientConnectorConfiguration.DFLT_PORT, cols);
-    }
-
-    /**
-     * Check that given columns are seen by client.
-     * @param schemaName Schema name to look for the table in.
-     * @param tblName Table name to check.
-     * @param port Port number.
-     * @param cols Columns whose presence must be checked.
-     */
-    static void checkTableState(String schemaName, String tblName, int port, QueryField... cols) throws SQLException {
+    static void checkTableState(IgniteEx node, String schemaName, String tblName, QueryField... cols)
+        throws SQLException {
         List<QueryField> flds = new ArrayList<>();
 
-        try (Connection c = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1:" + port)) {
+        try (Connection c = connect(node)) {
             try (ResultSet rs = c.getMetaData().getColumns(null, schemaName, tblName, "%")) {
                 while (rs.next()) {
                     String name = rs.getString("COLUMN_NAME");
@@ -110,8 +101,7 @@ public abstract class DynamicColumnsAbstractTest extends GridCommonAbstractTest
 
             assertEquals(exp.typeName(), act.typeName());
 
-            // TODO uncomment after IGNITE-6529 is implemented.
-            //assertEquals(exp.isNullable(), act.isNullable());
+            assertEquals(exp.isNullable(), act.isNullable());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8292335e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
index bf469f1..db874c2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
@@ -295,7 +295,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoCachePartitionedAtomic() throws Exception {
+    public void testCreateIndexNoCachePartitionedAtomic() throws Exception {
         checkCreateNotCache(PARTITIONED, ATOMIC, false);
     }
 
@@ -304,7 +304,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoCachePartitionedAtomicNear() throws Exception {
+    public void testCreateIndexNoCachePartitionedAtomicNear() throws Exception {
         checkCreateNotCache(PARTITIONED, ATOMIC, true);
     }
 
@@ -313,7 +313,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoCachePartitionedTransactional() throws Exception {
+    public void testCreateIndexNoCachePartitionedTransactional() throws Exception {
         checkCreateNotCache(PARTITIONED, TRANSACTIONAL, false);
     }
 
@@ -322,7 +322,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoCachePartitionedTransactionalNear() throws Exception {
+    public void testCreateIndexNoCachePartitionedTransactionalNear() throws Exception {
         checkCreateNotCache(PARTITIONED, TRANSACTIONAL, true);
     }
 
@@ -331,7 +331,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoCacheReplicatedAtomic() throws Exception {
+    public void testCreateIndexNoCacheReplicatedAtomic() throws Exception {
         checkCreateNotCache(REPLICATED, ATOMIC, false);
     }
 
@@ -340,7 +340,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoCacheReplicatedTransactional() throws Exception {
+    public void testCreateIndexNoCacheReplicatedTransactional() throws Exception {
         checkCreateNotCache(REPLICATED, TRANSACTIONAL, false);
     }
 
@@ -457,8 +457,8 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoColumnPartitionedAtomic() throws Exception {
-        checkCreateNoColumn(PARTITIONED, ATOMIC, false);
+    public void testCreateIndexNoColumnPartitionedAtomic() throws Exception {
+        checkCreateIndexNoColumn(PARTITIONED, ATOMIC, false);
     }
 
     /**
@@ -466,8 +466,8 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoColumnPartitionedAtomicNear() throws Exception {
-        checkCreateNoColumn(PARTITIONED, ATOMIC, true);
+    public void testCreateIndexNoColumnPartitionedAtomicNear() throws Exception {
+        checkCreateIndexNoColumn(PARTITIONED, ATOMIC, true);
     }
 
     /**
@@ -475,8 +475,8 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoColumnPartitionedTransactional() throws Exception {
-        checkCreateNoColumn(PARTITIONED, TRANSACTIONAL, false);
+    public void testCreateIndexNoColumnPartitionedTransactional() throws Exception {
+        checkCreateIndexNoColumn(PARTITIONED, TRANSACTIONAL, false);
     }
 
     /**
@@ -484,8 +484,8 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoColumnPartitionedTransactionalNear() throws Exception {
-        checkCreateNoColumn(PARTITIONED, TRANSACTIONAL, true);
+    public void testCreateIndexNoColumnPartitionedTransactionalNear() throws Exception {
+        checkCreateIndexNoColumn(PARTITIONED, TRANSACTIONAL, true);
     }
 
     /**
@@ -493,8 +493,8 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoColumnReplicatedAtomic() throws Exception {
-        checkCreateNoColumn(REPLICATED, ATOMIC, false);
+    public void testCreateIndexNoColumnReplicatedAtomic() throws Exception {
+        checkCreateIndexNoColumn(REPLICATED, ATOMIC, false);
     }
 
     /**
@@ -502,8 +502,8 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateNoColumnReplicatedTransactional() throws Exception {
-        checkCreateNoColumn(REPLICATED, TRANSACTIONAL, false);
+    public void testCreateIndexNoColumnReplicatedTransactional() throws Exception {
+        checkCreateIndexNoColumn(REPLICATED, TRANSACTIONAL, false);
     }
 
     /**
@@ -514,7 +514,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      * @param near Near flag.
      * @throws Exception If failed.
      */
-    private void checkCreateNoColumn(CacheMode mode, CacheAtomicityMode atomicityMode, boolean near) throws Exception {
+    private void checkCreateIndexNoColumn(CacheMode mode, CacheAtomicityMode atomicityMode, boolean near) throws Exception {
         initialize(mode, atomicityMode, near);
 
         final QueryIndex idx = index(IDX_NAME_1, field(randomString()));
@@ -533,8 +533,8 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateColumnWithAliasPartitionedAtomic() throws Exception {
-        checkCreateColumnWithAlias(PARTITIONED, ATOMIC, false);
+    public void testCreateIndexOnColumnWithAliasPartitionedAtomic() throws Exception {
+        checkCreateIndexOnColumnWithAlias(PARTITIONED, ATOMIC, false);
     }
 
     /**
@@ -542,8 +542,8 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateColumnWithAliasPartitionedAtomicNear() throws Exception {
-        checkCreateColumnWithAlias(PARTITIONED, ATOMIC, true);
+    public void testCreateIndexOnColumnWithAliasPartitionedAtomicNear() throws Exception {
+        checkCreateIndexOnColumnWithAlias(PARTITIONED, ATOMIC, true);
     }
 
     /**
@@ -551,8 +551,8 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      *
      * @throws Exception If failed.
      */
-    public void testCreateColumnWithAliasPartitionedTransactional() throws Exception {
-        checkCreateColumnWithAlias(PARTITIONED, TRANSACTIONAL, false);
+    public void testCreateIndexOnColumnWithAliasPartitionedTransactional() throws Exception {
+        checkCreateIndexOnColumnWithAlias(PARTITIONED, TRANSACTIONAL, false);
     }
 
     /**
@@ -561,7 +561,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      * @throws Exception If failed.
      */
     public void testCreateColumnWithAliasPartitionedTransactionalNear() throws Exception {
-        checkCreateColumnWithAlias(PARTITIONED, TRANSACTIONAL, true);
+        checkCreateIndexOnColumnWithAlias(PARTITIONED, TRANSACTIONAL, true);
     }
 
     /**
@@ -570,7 +570,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      * @throws Exception If failed.
      */
     public void testCreateColumnWithAliasReplicatedAtomic() throws Exception {
-        checkCreateColumnWithAlias(REPLICATED, ATOMIC, false);
+        checkCreateIndexOnColumnWithAlias(REPLICATED, ATOMIC, false);
     }
 
     /**
@@ -579,7 +579,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      * @throws Exception If failed.
      */
     public void testCreateColumnWithAliasReplicatedTransactional() throws Exception {
-        checkCreateColumnWithAlias(REPLICATED, TRANSACTIONAL, false);
+        checkCreateIndexOnColumnWithAlias(REPLICATED, TRANSACTIONAL, false);
     }
 
     /**
@@ -590,7 +590,7 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      * @param near Near flag.
      * @throws Exception If failed.
      */
-    private void checkCreateColumnWithAlias(CacheMode mode, CacheAtomicityMode atomicityMode, boolean near)
+    private void checkCreateIndexOnColumnWithAlias(CacheMode mode, CacheAtomicityMode atomicityMode, boolean near)
         throws Exception {
         initialize(mode, atomicityMode, near);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8292335e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
index 65ceb24..f8bb0cf 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
@@ -670,7 +670,7 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
             }
         });
 
-        assertIndex(cli, true, CACHE_NAME, TBL_NAME, IDX_NAME_1,
+        assertIndex(cli, CACHE_NAME, TBL_NAME, IDX_NAME_1,
             QueryIndex.DFLT_INLINE_SIZE, field(FIELD_NAME_1));
         assertIndexUsed(IDX_NAME_1, SQL_SIMPLE_FIELD_1, SQL_ARG_1);
 
@@ -690,8 +690,9 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
 
         queryProcessor(srv).dynamicIndexCreate(CACHE_NAME, CACHE_NAME, TBL_NAME, idx, false, 0).get();
 
-        assertIndex(cli, true, CACHE_NAME, TBL_NAME, IDX_NAME_2, QueryIndex.DFLT_INLINE_SIZE,
+        assertIndex(cli, CACHE_NAME, TBL_NAME, IDX_NAME_2, QueryIndex.DFLT_INLINE_SIZE,
             field(aliasUnescaped(FIELD_NAME_2)));
+
         assertIndexUsed(IDX_NAME_2, SQL_SIMPLE_FIELD_2, SQL_ARG_2);
 
         reconnectClientNode(srv, cli, restartCache, new RunnableX() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8292335e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
index e4681ae..eb6d7e6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
@@ -94,6 +94,18 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
     }
 
     /**
+     * Check table state on default node.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param cols Columns to look for.
+     * @throws SQLException if failed.
+     */
+    private void checkTableState(String schemaName, String tblName, QueryField... cols) throws SQLException {
+        checkTableState(grid(nodeIndex()), schemaName, tblName, cols);
+    }
+
+    /**
      * Test column addition to the end of the columns list.
      */
     public void testAddColumnSimple() throws SQLException {
@@ -114,7 +126,8 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
 
         doSleep(500);
 
-        checkTableState(QueryUtils.DFLT_SCHEMA, "PERSON", c("AGE", Integer.class.getName()), c("city", String.class.getName()));
+        checkTableState(QueryUtils.DFLT_SCHEMA, "PERSON", c("AGE", Integer.class.getName()),
+            c("city", String.class.getName()));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8292335e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
index 71b0b5f..f3deec9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
@@ -491,7 +491,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
      * @param additionalParams Supplemental parameters to append to {@code CREATE TABLE} SQL.
      */
     private void doTestCreateTable(String tplCacheName, String cacheGrp, CacheMode cacheMode,
-        CacheWriteSynchronizationMode writeSyncMode, String... additionalParams) {
+        CacheWriteSynchronizationMode writeSyncMode, String... additionalParams) throws SQLException {
         doTestCreateTable(tplCacheName, cacheGrp, cacheMode, writeSyncMode, false, additionalParams);
     }
 
@@ -506,7 +506,8 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
      * @param additionalParams Supplemental parameters to append to {@code CREATE TABLE} SQL.
      */
     private void doTestCreateTable(String tplCacheName, String cacheGrp, CacheMode cacheMode,
-        CacheWriteSynchronizationMode writeSyncMode, boolean useLegacyCacheGrpParamName, String... additionalParams) {
+        CacheWriteSynchronizationMode writeSyncMode, boolean useLegacyCacheGrpParamName, String... additionalParams)
+        throws SQLException {
         String cacheGrpParamName = useLegacyCacheGrpParamName ? "cacheGroup" : "cache_group";
 
         String sql = "CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar," +
@@ -547,30 +548,48 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
             if (writeSyncMode != null)
                 assertEquals(writeSyncMode, cacheDesc.cacheConfiguration().getWriteSynchronizationMode());
 
-            QueryTypeDescriptorImpl desc = typeExisting(node, cacheName, "Person");
+            List<String> colNames = new ArrayList<>(5);
 
-            assertEquals(Object.class, desc.keyClass());
-            assertEquals(Object.class, desc.valueClass());
+            List<Class<?>> colTypes = new ArrayList<>(5);
 
-            assertTrue(desc.valueTypeName(), desc.valueTypeName().contains("Person"));
+            List<String> pkColNames = new ArrayList<>(2);
 
-            assertTrue(desc.keyTypeName(), desc.keyTypeName().startsWith(desc.valueTypeName()));
-            assertTrue(desc.keyTypeName(), desc.keyTypeName().endsWith("KEY"));
+            try (Connection c = connect(node)) {
+                try (ResultSet rs = c.getMetaData().getColumns(null, QueryUtils.DFLT_SCHEMA, "Person", null)) {
+                    for (int j = 0; j < 5; j++) {
+                        assertTrue(rs.next());
 
-            assertEquals(
-                F.asList("id", "city", "name", "surname", "age"),
-                new ArrayList<>(desc.fields().keySet())
-            );
+                        colNames.add(rs.getString("COLUMN_NAME"));
 
-            assertProperty(desc, "id", Integer.class, true);
-            assertProperty(desc, "city", String.class, true);
-            assertProperty(desc, "name", String.class, false);
-            assertProperty(desc, "surname", String.class, false);
-            assertProperty(desc, "age", Integer.class, false);
+                        try {
+                            colTypes.add(Class.forName(DataType.getTypeClassName(DataType
+                                .convertSQLTypeToValueType(rs.getInt("DATA_TYPE")))));
+                        }
+                        catch (ClassNotFoundException e) {
+                            throw new AssertionError(e);
+                        }
+                    }
+
+                    assertFalse(rs.next());
+                }
+
+                try (ResultSet rs = c.getMetaData().getPrimaryKeys(null, QueryUtils.DFLT_SCHEMA, "Person")) {
+                    for (int j = 0; j < 2; j++) {
+                        assertTrue(rs.next());
+
+                        pkColNames.add(rs.getString("COLUMN_NAME"));
+                    }
+
+                    assertFalse(rs.next());
+                }
+            }
+
+            assertEqualsCollections(F.asList("id", "city", "name", "surname", "age"), colNames);
 
-            GridH2Table tbl = ((IgniteH2Indexing)node.context().query().getIndexing()).dataTable("PUBLIC", "Person");
+            assertEqualsCollections(F.<Class<?>>asList(Integer.class, String.class, String.class, String.class,
+                Integer.class), colTypes);
 
-            assertNotNull(tbl);
+            assertEqualsCollections(F.asList("id", "city"), pkColNames);
         }
     }
 


[09/11] ignite git commit: IGNITE-6873 Notify users about outdated browser.

Posted by sb...@apache.org.
IGNITE-6873 Notify users about outdated browser.


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

Branch: refs/heads/ignite-zk
Commit: ff3712ce830decb09fd68f71e56dd5eee4c51115
Parents: f418563
Author: Ilya Borisov <ib...@gridgain.com>
Authored: Tue Dec 5 14:00:01 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Tue Dec 5 14:00:01 2017 +0700

----------------------------------------------------------------------
 .../frontend/app/browserUpdate/index.js         | 34 ++++++++++++++++++
 .../frontend/app/browserUpdate/style.scss       | 36 ++++++++++++++++++++
 modules/web-console/frontend/package.json       |  5 +--
 .../frontend/webpack/webpack.common.js          |  3 +-
 4 files changed, 75 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ff3712ce/modules/web-console/frontend/app/browserUpdate/index.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/browserUpdate/index.js b/modules/web-console/frontend/app/browserUpdate/index.js
new file mode 100644
index 0000000..85df64e
--- /dev/null
+++ b/modules/web-console/frontend/app/browserUpdate/index.js
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import browserUpdate from 'browser-update';
+import './style.scss';
+
+browserUpdate({
+    notify: {
+        i: 10,
+        f: '-18m',
+        s: 9,
+        c: '-18m',
+        o: '-18m',
+        e: '-6m'
+    },
+    l: 'en',
+    mobile: false,
+    api: 5,
+    reminder: 0
+});

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff3712ce/modules/web-console/frontend/app/browserUpdate/style.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/browserUpdate/style.scss b/modules/web-console/frontend/app/browserUpdate/style.scss
new file mode 100644
index 0000000..5842f5f
--- /dev/null
+++ b/modules/web-console/frontend/app/browserUpdate/style.scss
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+@import "./../primitives/btn/index.scss";
+
+#buorg.buorg {
+    background-color: $brand-warning;
+    line-height: 16px;
+    font-family: Roboto, sans-serif;
+
+    a {
+        @extend .btn-ignite;
+
+        &#buorgul {
+            @extend .btn-ignite--success;
+        }
+
+        &#buorgig {
+            @extend .btn-ignite--primary;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff3712ce/modules/web-console/frontend/package.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json
index 18635f7..5b1734f 100644
--- a/modules/web-console/frontend/package.json
+++ b/modules/web-console/frontend/package.json
@@ -61,6 +61,7 @@
     "babel-runtime": "6.25.0",
     "bootstrap-sass": "3.3.7",
     "brace": "0.10.0",
+    "browser-update": "2.1.9",
     "copy-webpack-plugin": "4.0.1",
     "css-loader": "0.28.7",
     "eslint": "4.3.0",
@@ -102,7 +103,6 @@
   },
   "devDependencies": {
     "chai": "4.1.0",
-    "type-detect": "4.0.3",
     "jasmine-core": "2.6.4",
     "karma": "1.7.0",
     "karma-babel-preprocessor": "6.0.1",
@@ -115,6 +115,7 @@
     "mocha": "3.4.2",
     "mocha-teamcity-reporter": "1.1.1",
     "phantomjs-prebuilt": "2.1.14",
-    "sinon": "2.3.8"
+    "sinon": "2.3.8",
+    "type-detect": "4.0.3"
   }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff3712ce/modules/web-console/frontend/webpack/webpack.common.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/webpack/webpack.common.js b/modules/web-console/frontend/webpack/webpack.common.js
index 48e1e9b..a0d6d0c 100644
--- a/modules/web-console/frontend/webpack/webpack.common.js
+++ b/modules/web-console/frontend/webpack/webpack.common.js
@@ -44,7 +44,8 @@ export default {
     entry: {
         polyfill: 'babel-polyfill',
         vendor: path.join(app, 'vendor.js'),
-        app: path.join(app, 'app.js')
+        app: path.join(app, 'app.js'),
+        browserUpdate: path.join(app, 'browserUpdate', 'index.js')
     },
 
     // Output system.


[07/11] ignite git commit: IGNITE-6264 .NET: Verify Java API parity with a test

Posted by sb...@apache.org.
IGNITE-6264 .NET: Verify Java API parity with a test


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

Branch: refs/heads/ignite-zk
Commit: 4963e2d54fc509f59cc80138fc66571507a029ec
Parents: 06c4068
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Dec 4 19:39:30 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Dec 4 19:39:30 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.csproj             | 11 +++
 .../ApiParity/BinaryParityTest.cs               | 48 +++++++++++
 .../ApiParity/CacheAffinityParityTest.cs        | 39 +++++++++
 .../ApiParity/CacheParityTest.cs                | 76 +++++++++++++++++
 .../ApiParity/ClusterParityTest.cs              | 50 +++++++++++
 .../ApiParity/ComputeParityTest.cs              | 51 +++++++++++
 .../ApiParity/EventsParityTest.cs               | 50 +++++++++++
 .../ApiParity/IgniteParityTest.cs               | 79 +++++++++++++++++
 .../ApiParity/MessagingParityTest.cs            | 39 +++++++++
 .../ApiParity/ParityTest.cs                     | 90 +++++++++++++++++---
 .../ApiParity/ServicesParityTest.cs             | 39 +++++++++
 .../ApiParity/StreamerParityTest.cs             | 54 ++++++++++++
 .../ApiParity/TransactionsParityTest.cs         | 39 +++++++++
 13 files changed, 652 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 77b2e6e..648df7e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -69,13 +69,24 @@
     <Reference Include="System.Xml.Linq" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="ApiParity\BinaryParityTest.cs" />
+    <Compile Include="ApiParity\CacheAffinityParityTest.cs" />
+    <Compile Include="ApiParity\CacheParityTest.cs" />
     <Compile Include="ApiParity\ClientConnectorConfigurationParityTest.cs" />
+    <Compile Include="ApiParity\ClusterParityTest.cs" />
+    <Compile Include="ApiParity\ComputeParityTest.cs" />
     <Compile Include="ApiParity\DataRegionConfigurationParityTest.cs" />
     <Compile Include="ApiParity\DataStorageConfigurationParityTest.cs" />
+    <Compile Include="ApiParity\EventsParityTest.cs" />
     <Compile Include="ApiParity\IgniteConfigurationParityTest.cs" />
+    <Compile Include="ApiParity\IgniteParityTest.cs" />
+    <Compile Include="ApiParity\MessagingParityTest.cs" />
     <Compile Include="ApiParity\ParityTest.cs" />
     <Compile Include="ApiParity\CacheConfigurationParityTest.cs" />
     <Compile Include="ApiParity\QueryEntityConfigurationParityTest.cs" />
+    <Compile Include="ApiParity\ServicesParityTest.cs" />
+    <Compile Include="ApiParity\StreamerParityTest.cs" />
+    <Compile Include="ApiParity\TransactionsParityTest.cs" />
     <Compile Include="AssertExtensions.cs" />
     <Compile Include="Binary\BinaryBuilderSelfTestSimpleName.cs" />
     <Compile Include="Binary\BinaryDateTimeTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/BinaryParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/BinaryParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/BinaryParityTest.cs
new file mode 100644
index 0000000..87d1f91
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/BinaryParityTest.cs
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Binary;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="IBinary"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class BinaryParityTest
+    {
+        /** Known name mappings. */
+        private static readonly Dictionary<string, string> KnownMappings = new Dictionary<string, string>
+        {
+            {"type", "GetBinaryType"},
+            {"types", "GetBinaryTypes"}
+        };
+
+        /// <summary>
+        /// Tests the API parity.
+        /// </summary>
+        [Test]
+        public void TestBinary()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\IgniteBinary.java",
+                typeof(IBinary),
+                knownMappings: KnownMappings);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheAffinityParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheAffinityParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheAffinityParityTest.cs
new file mode 100644
index 0000000..f0ae286
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheAffinityParityTest.cs
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using Apache.Ignite.Core.Cache;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="ICacheAffinity"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class CacheAffinityParityTest
+    {
+        /// <summary>
+        /// Tests the API parity.
+        /// </summary>
+        [Test]
+        public void TestCacheAffinity()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\cache\affinity\Affinity.java",
+                typeof(ICacheAffinity));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheParityTest.cs
new file mode 100644
index 0000000..7b568bd
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheParityTest.cs
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using Apache.Ignite.Core.Cache;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="ICache{TK,TV}"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class CacheParityTest
+    {
+        /** Members that are not needed on .NET side. */
+        private static readonly string[] UnneededMembers =
+        {
+            // Java-specific.
+            "close",
+            "mxBean",
+            "localMxBean",
+            "indexReadyFuture",
+
+            // No needed, see IIgnite.DestroyCache
+            "destroy",
+
+            // The following look pointless, same as get, getAll, etc:
+            "Entry",
+            "EntryAsync",
+            "Entries",
+            "EntriesAsync",
+
+            // Questionable methods:
+            "AllOutTx",
+            "AllOutTxAsync"
+        };
+
+        /** Members that are missing on .NET side and should be added in future. */
+        private static readonly string[] MissingMembers =
+        {
+            "queryMetrics",  // IGNITE-6681
+            "resetQueryMetrics",  // IGNITE-6681
+            "queryDetailMetrics",  // IGNITE-6680
+            "resetQueryDetailMetrics",  // IGNITE-6680
+            "sizeLong",  // 	IGNITE-6563
+            "sizeLongAsync",  // IGNITE-6563
+            "localSizeLong",  // IGNITE-6563
+        };
+
+        /// <summary>
+        /// Tests the API parity.
+        /// </summary>
+        [Test]
+        public void TestCache()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\IgniteCache.java",
+                typeof(ICache<,>),
+                UnneededMembers,
+                MissingMembers);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ClusterParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ClusterParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ClusterParityTest.cs
new file mode 100644
index 0000000..59a80c3
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ClusterParityTest.cs
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+ 
+ namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using Apache.Ignite.Core.Cluster;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="ICluster"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class ClusterParityTest
+    {
+        /** Members that are not needed on .NET side. */
+        private static readonly string[] UnneededMembers =
+        {
+            "nodeLocalMap",
+            "startNodes",
+            "startNodesAsync",
+            "stopNodes",
+            "restartNodes"
+        };
+        
+        /// <summary>
+        /// Tests the API parity.
+        /// </summary>
+        [Test]
+        public void TestCluster()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\IgniteCluster.java",
+                typeof(ICluster),
+                UnneededMembers);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ComputeParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ComputeParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ComputeParityTest.cs
new file mode 100644
index 0000000..e3f47ae
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ComputeParityTest.cs
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using Apache.Ignite.Core.Compute;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="ICompute"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class ComputeParityTest
+    {
+        /** Methods that are not needed on .NET side. */
+        private static readonly string[] UnneededMethods =
+        {
+            "withExecutor",  // Java-specific.
+            "localDeployTask",  // Java-specific (classloaders)
+            "localTasks",  // Java-specific (classloaders)
+            "undeployTask",  // Java-specific (classloaders)
+            "withName",  // Java-specific (classloaders)
+            "activeTaskFutures"
+        };
+
+        /// <summary>
+        /// Tests API parity.
+        /// </summary>
+        [Test]
+        public void TestCompute()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\IgniteCompute.java",
+                typeof(ICompute),
+                UnneededMethods);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/EventsParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/EventsParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/EventsParityTest.cs
new file mode 100644
index 0000000..98e9e25
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/EventsParityTest.cs
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using Apache.Ignite.Core.Events;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="IEvents"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class EventsParityTest
+    {
+        /** Members that are missing on .NET side and should be added in future. */
+        private static readonly string[] MissingMembers =
+        {
+            // IGNITE-1683
+            "remoteListen",
+            "remoteListenAsync",
+            "stopRemoteListen",
+            "stopRemoteListenAsync",
+        };
+
+        /// <summary>
+        /// Tests the API parity.
+        /// </summary>
+        [Test]
+        public void TestEvents()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\IgniteEvents.java",
+                typeof(IEvents),
+                knownMissingMembers: MissingMembers);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteParityTest.cs
new file mode 100644
index 0000000..29cfda3
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteParityTest.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using System.Collections.Generic;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="IIgnite"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class IgniteParityTest
+    {
+        /** Methods that are not needed on .NET side. */
+        private static readonly string[] UnneededMembers =
+        {
+            "scheduler",
+            "close",
+            "executorService",
+            "fileSystem",
+            "fileSystems"
+        };
+
+        /** Members that are missing on .NET side and should be added in future. */
+        private static readonly string[] MissingMembers =
+        {
+            "version", // IGNITE-7101
+            
+            "createCaches",   // IGNITE-7100
+            "orCreateCaches", // IGNITE-7100
+            "destroyCaches",  // IGNITE-7100
+
+            "addCacheConfiguration", // IGNITE-7102
+
+            // Data structures.
+            "atomicStamped", // IGNITE-7104
+            "countDownLatch", // IGNITE-1418
+            "semaphore", // IGNITE-7103
+            "reentrantLock", // IGNITE-7105
+            "queue", // IGNITE-1417
+            "set" // IGNITE-6834
+        };
+
+        /** Known name mappings. */
+        private static readonly Dictionary<string, string> KnownMappings = new Dictionary<string, string>
+        {
+            {"message", "GetMessaging"},
+            {"log", "Logger"}
+        };
+
+        /// <summary>
+        /// Tests the IIgnite parity.
+        /// </summary>
+        [Test]
+        public void TestIgnite()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\Ignite.java",
+                typeof(IIgnite),
+                UnneededMembers,
+                MissingMembers,
+                KnownMappings);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/MessagingParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/MessagingParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/MessagingParityTest.cs
new file mode 100644
index 0000000..1d940fc
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/MessagingParityTest.cs
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using Apache.Ignite.Core.Messaging;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="IMessaging"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class MessagingParityTest
+    {
+        /// <summary>
+        /// Tests the API parity.
+        /// </summary>
+        [Test]
+        public void TestMessaging()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\IgniteMessaging.java",
+                typeof(IMessaging));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ParityTest.cs
index 068d22e..63aa96f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ParityTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ParityTest.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Tests.ApiParity
     using System.Collections.Generic;
     using System.IO;
     using System.Linq;
+    using System.Reflection;
     using System.Text;
     using System.Text.RegularExpressions;
     using Apache.Ignite.Core.Impl.Common;
@@ -35,8 +36,13 @@ namespace Apache.Ignite.Core.Tests.ApiParity
         private static readonly Regex JavaPropertyRegex = 
             new Regex("(@Deprecated)?\\s+public [^=^\r^\n]+ (\\w+)\\(\\) {", RegexOptions.Compiled);
 
+        /** Interface method regex. */
+        private static readonly Regex JavaInterfaceMethodRegex = 
+            new Regex("(@Deprecated)?\\s+(@Override)?\\s+public [^=^\r^\n]+ (\\w+)\\(.*?\\)",
+                RegexOptions.Compiled | RegexOptions.Singleline);
+
         /** Properties that are not needed on .NET side. */
-        private static readonly string[] UnneededProperties =
+        private static readonly string[] UnneededMethods =
         {
             "toString",
             "hashCode",
@@ -52,38 +58,78 @@ namespace Apache.Ignite.Core.Tests.ApiParity
             IEnumerable<string> knownMissingProperties = null,
             Dictionary<string, string> knownMappings = null)
         {
-            var path = Path.Combine(IgniteHome.Resolve(null), javaFilePath);
-
-            Assert.IsTrue(File.Exists(path));
+            var path = GetFullPath(javaFilePath);
 
             var dotNetProperties = type.GetProperties()
-                .ToDictionary(x => x.Name, x => x, StringComparer.OrdinalIgnoreCase);
+                .ToDictionary(x => x.Name, x => (MemberInfo) x, StringComparer.OrdinalIgnoreCase);
 
             var javaProperties = GetJavaProperties(path)
                 .Except(excludedProperties ?? Enumerable.Empty<string>());
 
-            var missingProperties = javaProperties
-                .Where(jp => !GetNameVariants(jp, knownMappings).Any(dotNetProperties.ContainsKey))
+            CheckParity(type, knownMissingProperties, knownMappings, javaProperties, dotNetProperties);
+        }
+
+        /// <summary>
+        /// Tests the configuration parity.
+        /// </summary>
+        public static void CheckInterfaceParity(string javaFilePath, 
+            Type type, 
+            IEnumerable<string> excludedMembers = null,
+            IEnumerable<string> knownMissingMembers = null,
+            Dictionary<string, string> knownMappings = null)
+        {
+            var path = GetFullPath(javaFilePath);
+
+            var dotNetMembers = type.GetMembers()
+                .GroupBy(x => x.Name)
+                .ToDictionary(x => x.Key, x => x.First(), StringComparer.OrdinalIgnoreCase);
+
+            var javaMethods = GetJavaInterfaceMethods(path)
+                .Except(excludedMembers ?? Enumerable.Empty<string>());
+
+            CheckParity(type, knownMissingMembers, knownMappings, javaMethods, dotNetMembers);
+        }
+
+        /// <summary>
+        /// Gets the full path.
+        /// </summary>
+        private static string GetFullPath(string javaFilePath)
+        {
+            var path = Path.Combine(IgniteHome.Resolve(null), javaFilePath);
+            Assert.IsTrue(File.Exists(path));
+
+            return path;
+        }
+
+        /// <summary>
+        /// Checks the parity.
+        /// </summary>
+        private static void CheckParity(Type type, IEnumerable<string> knownMissingMembers, 
+            IDictionary<string, string> knownMappings, IEnumerable<string> javaMethods, 
+            IDictionary<string, MemberInfo> dotNetMembers)
+        {
+            var missingMembers = javaMethods
+                .Where(jp => !GetNameVariants(jp, knownMappings).Any(dotNetMembers.ContainsKey))
                 .ToDictionary(x => x, x => x, StringComparer.OrdinalIgnoreCase);
 
-            var knownMissing = (knownMissingProperties ?? Enumerable.Empty<string>())
+            var knownMissing = (knownMissingMembers ?? Enumerable.Empty<string>())
                 .ToDictionary(x => x, x => x, StringComparer.OrdinalIgnoreCase);
 
             var sb = new StringBuilder();
 
-            foreach (var javaMissingProp in missingProperties)
+            foreach (var javaMissingProp in missingMembers)
             {
                 if (!knownMissing.ContainsKey(javaMissingProp.Key))
                 {
-                    sb.AppendFormat("{0}.{1} property is missing in .NET.\n", type.Name, javaMissingProp.Key);
+                    sb.AppendFormat("{0}.{1} member is missing in .NET.\n", type.Name, javaMissingProp.Key);
                 }
             }
 
             foreach (var dotnetMissingProp in knownMissing)
             {
-                if (!missingProperties.ContainsKey(dotnetMissingProp.Key))
+                if (!missingMembers.ContainsKey(dotnetMissingProp.Key))
                 {
-                    sb.AppendFormat("{0}.{1} property is missing in Java, but is specified as known in .NET.\n", 
+                    sb.AppendFormat("{0}.{1} member is missing in Java, but is specified as known in .NET.\n",
                         type.Name, dotnetMissingProp.Key);
                 }
             }
@@ -106,7 +152,21 @@ namespace Apache.Ignite.Core.Tests.ApiParity
                 .Where(m => m.Groups[1].Value == string.Empty)
                 .Select(m => m.Groups[2].Value.Replace("get", ""))
                 .Where(x => !x.Contains(" void "))
-                .Except(UnneededProperties);
+                .Except(UnneededMethods);
+        }
+
+        /// <summary>
+        /// Gets the java interface methods from file.
+        /// </summary>
+        private static IEnumerable<string> GetJavaInterfaceMethods(string path)
+        {
+            var text = File.ReadAllText(path);
+
+            return JavaInterfaceMethodRegex.Matches(text)
+                .OfType<Match>()
+                .Where(m => m.Groups[1].Value == string.Empty)
+                .Select(m => m.Groups[3].Value.Replace("get", ""))
+                .Except(UnneededMethods);
         }
 
         /// <summary>
@@ -116,6 +176,10 @@ namespace Apache.Ignite.Core.Tests.ApiParity
             IDictionary<string, string> knownMappings)
         {
             yield return javaPropertyName;
+            
+            yield return "get" + javaPropertyName;
+            
+            yield return "is" + javaPropertyName;
 
             yield return javaPropertyName.Replace("PoolSize", "ThreadPoolSize");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ServicesParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ServicesParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ServicesParityTest.cs
new file mode 100644
index 0000000..d8c0938
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/ServicesParityTest.cs
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using Apache.Ignite.Core.Services;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="IServices"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class ServicesParityTest
+    {
+        /// <summary>
+        /// Tests the API parity.
+        /// </summary>
+        [Test]
+        public void TestServices()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\IgniteServices.java",
+                typeof(IServices));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/StreamerParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/StreamerParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/StreamerParityTest.cs
new file mode 100644
index 0000000..8e795e5
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/StreamerParityTest.cs
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Datastream;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="IDataStreamer{TK,TV}"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class StreamerParityTest
+    {
+        /** Members that are not needed on .NET side. */
+        private static readonly string[] UnneededMembers =
+        {
+            "deployClass"
+        };
+
+        /** Known name mappings. */
+        private static readonly Dictionary<string, string> KnownMappings = new Dictionary<string, string>
+        {
+            {"keepBinary", "WithKeepBinary"},
+            {"future", "Task"}
+        };
+
+        /// <summary>
+        /// Tests the API parity.
+        /// </summary>
+        [Test]
+        public void TestStreamer()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\IgniteDataStreamer.java",
+                typeof(IDataStreamer<,>),
+                UnneededMembers, knownMappings: KnownMappings);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4963e2d5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/TransactionsParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/TransactionsParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/TransactionsParityTest.cs
new file mode 100644
index 0000000..7c19569
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/TransactionsParityTest.cs
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.ApiParity
+{
+    using Apache.Ignite.Core.Transactions;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that <see cref="ITransactions"/> has all APIs from Java Ignite interface.
+    /// </summary>
+    public class TransactionsParityTest
+    {
+        /// <summary>
+        /// Tests the API parity.
+        /// </summary>
+        [Test]
+        public void TestTransactions()
+        {
+            ParityTest.CheckInterfaceParity(
+                @"modules\core\src\main\java\org\apache\ignite\IgniteTransactions.java",
+                typeof(ITransactions));
+        }
+    }
+}
\ No newline at end of file


[08/11] ignite git commit: IGNITE-4398. Prevent $http caching in IE11.

Posted by sb...@apache.org.
IGNITE-4398. Prevent $http caching in IE11.


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

Branch: refs/heads/ignite-zk
Commit: f41856309f83fc3003a975a13bc893942a10b24d
Parents: 4963e2d
Author: alexdel <ve...@yandex.ru>
Authored: Tue Dec 5 11:40:35 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Tue Dec 5 11:40:35 2017 +0700

----------------------------------------------------------------------
 modules/web-console/backend/middlewares/api.js           | 5 +++++
 modules/web-console/frontend/app/helpers/jade/mixins.pug | 2 +-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f4185630/modules/web-console/backend/middlewares/api.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/middlewares/api.js b/modules/web-console/backend/middlewares/api.js
index aa599ae..23fd7ae 100644
--- a/modules/web-console/backend/middlewares/api.js
+++ b/modules/web-console/backend/middlewares/api.js
@@ -25,6 +25,11 @@ module.exports = {
 
 module.exports.factory = () => {
     return (req, res, next) => {
+        // Set headers to avoid API caching in browser (esp. IE)
+        res.header('Cache-Control', 'must-revalidate');
+        res.header('Expires', '-1');
+        res.header('Last-Modified', new Date().toUTCString());
+
         res.api = {
             error(err) {
                 if (err.name === 'MongoError')

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4185630/modules/web-console/frontend/app/helpers/jade/mixins.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/mixins.pug b/modules/web-console/frontend/app/helpers/jade/mixins.pug
index 3e39077..7eb4d89 100644
--- a/modules/web-console/frontend/app/helpers/jade/mixins.pug
+++ b/modules/web-console/frontend/app/helpers/jade/mixins.pug
@@ -40,7 +40,7 @@ mixin main-table(title, rows, focusId, click, rowTemplate, searchField)
             thead
                 tr
                     th
-                        lable.labelHeader.labelFormField #{title}:
+                        label.labelHeader.labelFormField #{title}:
                         .col-sm-3.pull-right(style='padding: 0')
                             input.form-control(type='text' st-search=`${searchField}` placeholder=`Filter ${title}...`)
             tbody


[06/11] ignite git commit: Fixed C++ test version fail.

Posted by sb...@apache.org.
Fixed C++ test version fail.


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

Branch: refs/heads/ignite-zk
Commit: 06c4068bb891567853904d81c67ae6d3d8e0427d
Parents: 08ab9af
Author: Igor Sapego <is...@gridgain.com>
Authored: Mon Dec 4 19:30:31 2017 +0300
Committer: Igor Sapego <is...@gridgain.com>
Committed: Mon Dec 4 19:30:31 2017 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/odbc-test/src/sql_get_info_test.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/06c4068b/modules/platforms/cpp/odbc-test/src/sql_get_info_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_get_info_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_get_info_test.cpp
index e628fb6..ad1236c 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_get_info_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_get_info_test.cpp
@@ -185,8 +185,8 @@ BOOST_AUTO_TEST_CASE(TestValues)
     CheckStrInfo(SQL_DRIVER_NAME, "Apache Ignite");
     CheckStrInfo(SQL_DBMS_NAME, "Apache Ignite");
     CheckStrInfo(SQL_DRIVER_ODBC_VER, "03.00");
-    CheckStrInfo(SQL_DRIVER_VER, "02.03.0000");
-    CheckStrInfo(SQL_DBMS_VER, "02.03.0000");
+    CheckStrInfo(SQL_DRIVER_VER, "02.04.0000");
+    CheckStrInfo(SQL_DBMS_VER, "02.04.0000");
     CheckStrInfo(SQL_COLUMN_ALIAS, "Y");
     CheckStrInfo(SQL_IDENTIFIER_QUOTE_CHAR, "");
     CheckStrInfo(SQL_CATALOG_NAME_SEPARATOR, ".");


[05/11] ignite git commit: zk

Posted by sb...@apache.org.
zk


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

Branch: refs/heads/ignite-zk
Commit: 0bddcdef32646c407e879fbe39765f877928d44c
Parents: 74526d1
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 4 18:09:43 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 4 18:09:43 2017 +0300

----------------------------------------------------------------------
 .../zk/internal/ZkInternalFailNodeMessage.java  |  55 --------
 .../ZkInternalForceNodeFailMessage.java         |  55 ++++++++
 .../zk/internal/ZkInternalJoinErrorMessage.java |  43 +++++++
 .../zk/internal/ZookeeperDiscoveryImpl.java     | 129 +++++++++++++------
 .../ZookeeperDiscoverySpiBasicTest.java         |  17 +++
 5 files changed, 204 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0bddcdef/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalFailNodeMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalFailNodeMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalFailNodeMessage.java
deleted file mode 100644
index a97289d..0000000
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalFailNodeMessage.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.spi.discovery.zk.internal;
-
-import java.util.UUID;
-import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
-import org.jetbrains.annotations.Nullable;
-
-/**
- *
- */
-public class ZkInternalFailNodeMessage implements ZkInternalMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    final UUID nodeId;
-
-    /** */
-    final String warning;
-
-    /**
-     * @param nodeId Node ID.
-     * @param warning Warning to be displayed on all nodes.
-     */
-    ZkInternalFailNodeMessage(UUID nodeId, String warning) {
-        this.nodeId = nodeId;
-        this.warning = warning;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isMutable() {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0bddcdef/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalForceNodeFailMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalForceNodeFailMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalForceNodeFailMessage.java
new file mode 100644
index 0000000..fafcafc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalForceNodeFailMessage.java
@@ -0,0 +1,55 @@
+/*
+ * 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.discovery.zk.internal;
+
+import java.util.UUID;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class ZkInternalForceNodeFailMessage implements ZkInternalMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final UUID nodeId;
+
+    /** */
+    final String warning;
+
+    /**
+     * @param nodeId Node ID.
+     * @param warning Warning to be displayed on all nodes.
+     */
+    ZkInternalForceNodeFailMessage(UUID nodeId, String warning) {
+        this.nodeId = nodeId;
+        this.warning = warning;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0bddcdef/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java
new file mode 100644
index 0000000..7e06858
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+class ZkInternalJoinErrorMessage implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final int nodeInternalId;
+
+    /** */
+    private final String err;
+
+    /**
+     * @param nodeInternalId Joining node internal ID.
+     * @param err Error message.
+     */
+    ZkInternalJoinErrorMessage(int nodeInternalId, String err) {
+        this.nodeInternalId = nodeInternalId;
+        this.err = err;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0bddcdef/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
index 796310f..ef67ec4 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
@@ -243,7 +243,7 @@ public class ZookeeperDiscoveryImpl {
             return;
         }
 
-        sendCustomMessage(new ZkInternalFailNodeMessage(nodeId, warning));
+        sendCustomMessage(new ZkInternalForceNodeFailMessage(nodeId, warning));
     }
 
     /**
@@ -618,27 +618,19 @@ public class ZookeeperDiscoveryImpl {
 
         TreeMap<Integer, String> alives = new TreeMap<>();
 
-        Integer locInternalId = null;
+        int locInternalId = ZkIgnitePaths.aliveInternalId(state.locNodeZkPath);
 
         for (String aliveNodePath : aliveNodes) {
             Integer internalId = ZkIgnitePaths.aliveInternalId(aliveNodePath);
 
             alives.put(internalId, aliveNodePath);
-
-            if (locInternalId == null) {
-                UUID nodeId = ZkIgnitePaths.aliveNodeId(aliveNodePath);
-
-                if (locNode.id().equals(nodeId))
-                    locInternalId = internalId;
-            }
         }
 
         assert !alives.isEmpty();
-        assert locInternalId != null;
 
         Map.Entry<Integer, String> crdE = alives.firstEntry();
 
-        if (locInternalId.equals(crdE.getKey()))
+        if (locInternalId == crdE.getKey())
             onBecomeCoordinator(aliveNodes, locInternalId);
         else {
             assert alives.size() > 1;
@@ -746,9 +738,7 @@ public class ZookeeperDiscoveryImpl {
             assert old == null;
 
             if (!state.top.nodesByInternalId.containsKey(internalId)) {
-                generateNodeJoin(curTop, internalId, child);
-
-                watchAliveNodeData(child);
+                processJoinOnCoordinator(curTop, internalId, child);
 
                 newEvts = true;
             }
@@ -779,6 +769,81 @@ public class ZookeeperDiscoveryImpl {
     }
 
     /**
+     * @param curTop Current nodes.
+     * @param internalId Joined node internal ID.
+     * @param aliveNodePath Joined node path.
+     * @throws Exception If failed.
+     */
+    private void processJoinOnCoordinator(TreeMap<Long, ZookeeperClusterNode> curTop,
+        int internalId,
+        String aliveNodePath) throws Exception {
+        UUID nodeId = ZkIgnitePaths.aliveNodeId(aliveNodePath);
+
+        String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, aliveNodePath);
+        byte[] joinData;
+
+        try {
+            joinData = state.zkClient.getData(joinDataPath);
+        }
+        catch (KeeperException.NoNodeException e) {
+            U.warn(log, "Failed to read joining node data, node left before join process finished: " + nodeId);
+
+            return;
+        }
+
+        String err = null;
+
+        ZkJoiningNodeData joiningNodeData = null;
+
+        try {
+            joiningNodeData = unmarshalZip(joinData);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to unmarshal joining node data [nodePath=" + aliveNodePath + "']", e);
+
+            err = "Failed to unmarshal join data: " + e;
+        }
+
+        if (err == null) {
+            assert joiningNodeData != null;
+
+            err = validateJoiningNode(joiningNodeData.node());
+        }
+
+        if (err == null) {
+            ZookeeperClusterNode joinedNode = joiningNodeData.node();
+
+            assert nodeId.equals(joinedNode.id()) : joiningNodeData.node();
+
+            generateNodeJoin(curTop, joinData, joiningNodeData, internalId);
+
+            watchAliveNodeData(aliveNodePath);
+        }
+        else {
+            ZkInternalJoinErrorMessage msg = new ZkInternalJoinErrorMessage(internalId, err);
+
+            // IgniteNodeValidationResult err = spi.getSpiContext().validateNode(node);
+        }
+    }
+
+    /**
+     * @param node Joining node.
+     * @return
+     */
+    @Nullable private String validateJoiningNode(ZookeeperClusterNode node) {
+        ZookeeperClusterNode node0 = state.top.nodesById.get(node.id());
+
+        if (node0 != null) {
+            U.error(log, "Failed to include node in cluster, node with the same ID already exists [joiningNode=" + node +
+                ", existingNode=" + node0 + ']');
+
+            return "Node with the same ID already exists";
+        }
+
+        return null;
+    }
+
+    /**
      * @throws Exception If failed.
      */
     private void saveAndProcessNewEvents() throws Exception {
@@ -827,34 +892,18 @@ public class ZookeeperDiscoveryImpl {
     /**
      * @param curTop Current nodes.
      * @param internalId Joined node internal ID.
-     * @param aliveNodePath Joined node path.
      * @throws Exception If failed.
      */
-    private void generateNodeJoin(TreeMap<Long, ZookeeperClusterNode> curTop,
-        int internalId,
-        String aliveNodePath)
+    private void generateNodeJoin(
+        TreeMap<Long, ZookeeperClusterNode> curTop,
+        byte[] joinData,
+        ZkJoiningNodeData joiningNodeData,
+        int internalId)
         throws Exception
     {
-        UUID nodeId = ZkIgnitePaths.aliveNodeId(aliveNodePath);
-
-        String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, aliveNodePath);
-        byte[] joinData;
-
-        try {
-            joinData = state.zkClient.getData(joinDataPath);
-        }
-        catch (KeeperException.NoNodeException e) {
-            U.warn(log, "Failed to read joining node data, node left before join process finished: " + nodeId);
-
-            return;
-        }
-
-        // TODO ZK: fail node if can not unmarshal.
-        ZkJoiningNodeData joiningNodeData = unmarshalZip(joinData);
-
         ZookeeperClusterNode joinedNode = joiningNodeData.node();
 
-        assert nodeId.equals(joinedNode.id()) : joiningNodeData.node();
+        UUID nodeId = joinedNode.id();
 
         state.evtsData.topVer++;
         state.evtsData.evtIdGen++;
@@ -1061,8 +1110,8 @@ public class ZookeeperDiscoveryImpl {
 
                         state.evtsData.evtIdGen++;
 
-                        if (msg instanceof ZkInternalFailNodeMessage) {
-                            ZkInternalFailNodeMessage msg0 = (ZkInternalFailNodeMessage)msg;
+                        if (msg instanceof ZkInternalForceNodeFailMessage) {
+                            ZkInternalForceNodeFailMessage msg0 = (ZkInternalForceNodeFailMessage)msg;
 
                             if (alives == null)
                                 alives = new HashSet<>(state.top.nodesById.keySet());
@@ -1358,8 +1407,8 @@ public class ZookeeperDiscoveryImpl {
      * @param msg
      */
     private void processInternalMessage(ZkDiscoveryCustomEventData evtData, ZkInternalMessage msg) throws Exception {
-        if (msg instanceof ZkInternalFailNodeMessage) {
-            ZkInternalFailNodeMessage msg0 = (ZkInternalFailNodeMessage)msg;
+        if (msg instanceof ZkInternalForceNodeFailMessage) {
+            ZkInternalForceNodeFailMessage msg0 = (ZkInternalForceNodeFailMessage)msg;
 
             ClusterNode creatorNode = state.top.nodesById.get(evtData.sndNodeId);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0bddcdef/modules/core/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiBasicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiBasicTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiBasicTest.java
index 875d264..2c6890f 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiBasicTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiBasicTest.java
@@ -120,6 +120,9 @@ public class ZookeeperDiscoverySpiBasicTest extends GridCommonAbstractTest {
     /** */
     private boolean dfltConsistenId;
 
+    /** */
+    private UUID nodeId;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         if (testSockNio)
@@ -127,6 +130,9 @@ public class ZookeeperDiscoverySpiBasicTest extends GridCommonAbstractTest {
 
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
+        if (nodeId != null)
+            cfg.setNodeId(nodeId);
+
         if (!dfltConsistenId)
             cfg.setConsistentId(igniteInstanceName);
 
@@ -1397,6 +1403,17 @@ public class ZookeeperDiscoverySpiBasicTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testDuplicatedNodeId() throws Exception {
+        nodeId = UUID.randomUUID();
+
+        startGrid(0);
+
+        startGrid(1);
+    }
+
+    /**
      * @param clients Clients.
      * @param c Closure to run.
      * @throws Exception If failed.


[02/11] ignite git commit: IGNITE-6338 .NET: Thin client: LINQ

Posted by sb...@apache.org.
IGNITE-6338 .NET: Thin client: LINQ

This closes #3125


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

Branch: refs/heads/ignite-zk
Commit: 1f4337527ee2f75b90b46a297b5056064e78eb40
Parents: f08c9d3
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Dec 4 11:14:32 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Dec 4 11:14:32 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.DotNetCore.csproj  |   2 +
 .../Apache.Ignite.Core.Tests.csproj             |   2 +
 .../Query/Linq/CacheLinqTest.Introspection.cs   |   8 +-
 .../Client/Cache/CreateCacheTest.cs             |   1 -
 .../Client/Cache/LinqTest.cs                    | 117 +++++++++++
 .../Client/Cache/SqlQueryTest.cs                |  43 +---
 .../Client/Cache/SqlQueryTestBase.cs            |  68 +++++++
 .../Client/RawSocketTest.cs                     |   1 -
 .../TestUtils.Windows.cs                        |   2 -
 .../Impl/Cache/ICacheInternal.cs                |   6 +
 .../Impl/Client/Cache/CacheClient.cs            |  64 +++++-
 .../Client/Cache/Query/ClientQueryCursorBase.cs |   4 +-
 .../Apache.Ignite.Linq.csproj                   |   3 +-
 .../CacheClientLinqExtensions.cs                | 131 ++++++++++++
 .../Apache.Ignite.Linq/CacheExtensions.cs       | 195 ------------------
 .../Apache.Ignite.Linq/CacheLinqExtensions.cs   | 197 +++++++++++++++++++
 .../Apache.Ignite.Linq/ICacheQueryable.cs       |   1 +
 .../Impl/CacheFieldsQueryProvider.cs            |   2 +-
 .../Apache.Ignite.Linq/Impl/CacheQueryable.cs   |   8 +-
 .../Impl/CacheQueryableBase.cs                  |   1 +
 .../dotnet/Apache.Ignite/IgniteRunner.cs        |   1 -
 21 files changed, 598 insertions(+), 259 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
index 5d735eb..8e4de7f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
@@ -64,7 +64,9 @@
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\EmptyObject.cs" Link="ThinClient\Cache\EmptyObject.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\Person.cs" Link="ThinClient\Cache\Person.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\ScanQueryTest.cs" Link="ThinClient\Cache\ScanQueryTest.cs" />
+    <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\SqlQueryTestBase.cs" Link="ThinClient\Cache\SqlQueryTestBase.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\SqlQueryTest.cs" Link="ThinClient\Cache\SqlQueryTest.cs" />
+    <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\LinqTest.cs" Link="ThinClient\Cache\LinqTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\ClientConnectionTest.cs" Link="ThinClient\ClientConnectionTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\ClientTestBase.cs" Link="ThinClient\ClientTestBase.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\IgniteClientConfigurationTest.cs" Link="ThinClient\IgniteClientConfigurationTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 8bd8f28..77b2e6e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -113,9 +113,11 @@
     <Compile Include="Client\Cache\ClientCacheConfigurationTest.cs" />
     <Compile Include="Client\Cache\EmptyObject.cs" />
     <Compile Include="Client\Cache\CreateCacheTest.cs" />
+    <Compile Include="Client\Cache\LinqTest.cs" />
     <Compile Include="Client\Cache\ScanQueryTest.cs" />
     <Compile Include="Client\Cache\Person.cs" />
     <Compile Include="Client\Cache\SqlQueryTest.cs" />
+    <Compile Include="Client\Cache\SqlQueryTestBase.cs" />
     <Compile Include="Client\ClientTestBase.cs" />
     <Compile Include="Client\RawSocketTest.cs" />
     <Compile Include="Client\ClientConnectionTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
index a13131b..6e7483e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
@@ -58,7 +58,9 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Linq
             }).Where(x => x.Key > 10).ToCacheQueryable();
 
             Assert.AreEqual(cache.Name, query.CacheName);
+#pragma warning disable 618 // Type or member is obsolete
             Assert.AreEqual(cache.Ignite, query.Ignite);
+#pragma warning restore 618 // Type or member is obsolete
 
             var fq = query.GetFieldsQuery();
 
@@ -93,10 +95,12 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Linq
                   "Timeout=00:00:02.5000000, ReplicatedOnly=True, Colocated=True, Schema=, Lazy=True]]", str);
 
             // Check fields query
-            var fieldsQuery = (ICacheQueryable)cache.AsCacheQueryable().Select(x => x.Value.Name);
+            var fieldsQuery = cache.AsCacheQueryable().Select(x => x.Value.Name).ToCacheQueryable();
 
             Assert.AreEqual(cache.Name, fieldsQuery.CacheName);
-            Assert.AreEqual(cache.Ignite, fieldsQuery.Ignite);
+#pragma warning disable 618 // Type or member is obsolete
+            Assert.AreEqual(cache.Ignite, query.Ignite);
+#pragma warning restore 618 // Type or member is obsolete
 
             fq = fieldsQuery.GetFieldsQuery();
             Assert.AreEqual(GetSqlEscapeAll()

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CreateCacheTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CreateCacheTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CreateCacheTest.cs
index 4489462..94ae6cd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CreateCacheTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CreateCacheTest.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
     using Apache.Ignite.Core.Client;
     using Apache.Ignite.Core.Client.Cache;
     using Apache.Ignite.Core.Configuration;
-    using Apache.Ignite.Core.Impl.Client;
     using Apache.Ignite.Core.Tests.Cache;
     using NUnit.Framework;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/LinqTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/LinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/LinqTest.cs
new file mode 100644
index 0000000..e61f712
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/LinqTest.cs
@@ -0,0 +1,117 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Client.Cache
+{
+    using System;
+    using System.Linq;
+    using Apache.Ignite.Linq;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests LINQ in thin client.
+    /// </summary>
+    public class LinqTest : SqlQueryTestBase
+    {
+        /// <summary>
+        /// Tests basic queries.
+        /// </summary>
+        [Test]
+        public void TestBasicQueries()
+        {
+            var cache = GetClientCache<Person>();
+
+            // All items.
+            var qry = cache.AsCacheQueryable();
+            Assert.AreEqual(Count, qry.Count());
+
+            // Filter.
+            qry = cache.AsCacheQueryable().Where(x => x.Value.Name.EndsWith("7"));
+            Assert.AreEqual(7, qry.Single().Key);
+            Assert.AreEqual("select _T0._KEY, _T0._VAL from \"cache\".PERSON as _T0 where (_T0.NAME like '%' || ?) ",
+                qry.ToCacheQueryable().GetFieldsQuery().Sql);
+
+            // DateTime.
+            var arg = DateTime.UtcNow.AddDays(Count - 1);
+            var qry2 = cache.AsCacheQueryable(false, "Person")
+                .Where(x => x.Value.DateTime > arg).Select(x => x.Key);
+            Assert.AreEqual(Count, qry2.Single());
+        }
+
+        /// <summary>
+        /// Tests joins.
+        /// </summary>
+        [Test]
+        public void TestJoins()
+        {
+            var cache1 = Client.GetCache<int, Person>(CacheName);
+            var cache2 = Client.GetCache<int, Person>(CacheName2);
+
+            // Non-distributed join returns incomplete results.
+            var persons1 = cache1.AsCacheQueryable(false);
+            var persons2 = cache2.AsCacheQueryable();
+
+            var qry = persons1
+                .Join(persons2, p1 => p1.Value.Id, p2 => Count + 1 - p2.Value.Id, (p1, p2) => p2.Value.Name);
+
+            Assert.Greater(Count, qry.ToArray().Length);
+
+
+            // Distributed join fixes the problem.
+            persons1 = cache1.AsCacheQueryable(new QueryOptions {EnableDistributedJoins = true});
+            persons2 = cache2.AsCacheQueryable(new QueryOptions {EnableDistributedJoins = true});
+
+            var qry2 =
+                from p1 in persons1
+                join p2 in persons2 on p1.Value.Id equals Count + 1 - p2.Value.Id
+                select p2.Value.DateTime;
+
+            Assert.AreEqual(Count, qry2.ToArray().Length);
+        }
+
+        /// <summary>
+        /// Tests DML via LINQ.
+        /// </summary>
+        [Test]
+        public void TestDml()
+        {
+            var cache = GetClientCache<Person>();
+
+            Assert.AreEqual(Count, cache.GetSize());
+
+            var res = cache.AsCacheQueryable().Where(x => x.Key % 3 == 0).RemoveAll();
+            Assert.AreEqual(Count / 3, res);
+
+            Assert.AreEqual(Count - res, cache.GetSize());
+        }
+
+        /// <summary>
+        /// Tests the compiled query.
+        /// </summary>
+        [Test]
+        public void TestCompiledQuery()
+        {
+            var cache = GetClientCache<Person>();
+            var persons = cache.AsCacheQueryable();
+
+            var qry = CompiledQuery.Compile((int id) => persons.Where(x => x.Value.Id == id));
+
+            Assert.AreEqual(1, qry(1).Single().Key);
+            Assert.AreEqual(3, qry(3).Single().Key);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
index 720a71b..6f6df11 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
@@ -19,7 +19,6 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
 {
     using System;
     using System.Linq;
-    using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cache.Query;
     using Apache.Ignite.Core.Client;
     using NUnit.Framework;
@@ -27,36 +26,9 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
     /// <summary>
     /// Tests SQL queries via thin client.
     /// </summary>
-    public class SqlQueryTest : ClientTestBase
+    public class SqlQueryTest : SqlQueryTestBase
     {
         /// <summary>
-        /// Cache item count.
-        /// </summary>
-        private const int Count = 10;
-
-        /// <summary>
-        /// Second cache name.
-        /// </summary>
-        private const string CacheName2 = CacheName + "2";
-
-        /// <summary>
-        /// Initializes a new instance of the <see cref="ScanQueryTest"/> class.
-        /// </summary>
-        public SqlQueryTest() : base(2)
-        {
-            // No-op.
-        }
-
-        /// <summary>
-        /// Sets up the test.
-        /// </summary>
-        public override void TestSetUp()
-        {
-            InitCache(CacheName);
-            InitCache(CacheName2);
-        }
-
-        /// <summary>
         /// Tests the SQL query.
         /// </summary>
         [Test]
@@ -251,18 +223,5 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
             Assert.AreEqual(1, res[0][0]);
             Assert.AreEqual("baz", cache[-10].Name);
         }
-
-        /// <summary>
-        /// Initializes the cache.
-        /// </summary>
-        private static void InitCache(string cacheName)
-        {
-            var cache = Ignition.GetIgnite().GetOrCreateCache<int, Person>(
-                new CacheConfiguration(cacheName, new QueryEntity(typeof(int), typeof(Person))));
-
-            cache.RemoveAll();
-
-            cache.PutAll(Enumerable.Range(1, Count).ToDictionary(x => x, x => new Person(x)));
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTestBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTestBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTestBase.cs
new file mode 100644
index 0000000..7efcb9c
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTestBase.cs
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Client.Cache
+{
+    using System.Linq;
+    using Apache.Ignite.Core.Cache.Configuration;
+
+    /// <summary>
+    /// Base class for SQL tests.
+    /// </summary>
+    public class SqlQueryTestBase : ClientTestBase
+    {
+        /// <summary>
+        /// Cache item count.
+        /// </summary>
+        protected const int Count = 10;
+
+        /// <summary>
+        /// Second cache name.
+        /// </summary>
+        protected const string CacheName2 = CacheName + "2";
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ScanQueryTest"/> class.
+        /// </summary>
+        public SqlQueryTestBase() : base(2)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Sets up the test.
+        /// </summary>
+        public override void TestSetUp()
+        {
+            InitCache(CacheName);
+            InitCache(CacheName2);
+        }
+
+        /// <summary>
+        /// Initializes the cache.
+        /// </summary>
+        private static void InitCache(string cacheName)
+        {
+            var cache = Ignition.GetIgnite().GetOrCreateCache<int, Person>(
+                new CacheConfiguration(cacheName, new QueryEntity(typeof(int), typeof(Person))));
+
+            cache.RemoveAll();
+
+            cache.PutAll(Enumerable.Range(1, Count).ToDictionary(x => x, x => new Person(x)));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
index b088bb6..8ab110f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
@@ -25,7 +25,6 @@ namespace Apache.Ignite.Core.Tests.Client
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
-    using Apache.Ignite.Core.Impl.Client;
     using NUnit.Framework;
 
     /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.Windows.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.Windows.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.Windows.cs
index 14b58f2..2169630 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.Windows.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.Windows.cs
@@ -20,10 +20,8 @@ namespace Apache.Ignite.Core.Tests
     using System;
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
-    using System.Linq;
     using Apache.Ignite.Core.Configuration;
     using Apache.Ignite.Core.Impl;
-    using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Tests.Process;
     using NUnit.Framework;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheInternal.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheInternal.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheInternal.cs
index 0349db8..1ec5341 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheInternal.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheInternal.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Impl.Cache
 {
     using System;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cache.Query;
 
     /// <summary>
@@ -50,5 +51,10 @@ namespace Apache.Ignite.Core.Impl.Cache
         /// </returns>
         T DoOutInOpExtension<T>(int extensionId, int opCode, Action<IBinaryRawWriter> writeAction, 
             Func<IBinaryRawReader, T> readFunc);
+
+        /// <summary>
+        /// Gets the cache configuration.
+        /// </summary>
+        CacheConfiguration GetConfiguration();
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
index 45c0b0f..93829c2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
@@ -20,9 +20,11 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
     using System;
     using System.Collections.Generic;
     using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using System.IO;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cache.Query;
     using Apache.Ignite.Core.Client;
     using Apache.Ignite.Core.Client.Cache;
@@ -37,7 +39,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
     /// <summary>
     /// Client cache implementation.
     /// </summary>
-    internal sealed class CacheClient<TK, TV> : ICacheClient<TK, TV>
+    internal sealed class CacheClient<TK, TV> : ICacheClient<TK, TV>, ICacheInternal
     {
         /** Scan query filter platform code: .NET filter. */
         private const byte FilterPlatformDotnet = 2;
@@ -188,10 +190,17 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
             IgniteArgumentCheck.NotNull(sqlFieldsQuery, "sqlFieldsQuery");
             IgniteArgumentCheck.NotNull(sqlFieldsQuery.Sql, "sqlFieldsQuery.Sql");
 
-            return DoOutInOp(ClientOp.QuerySqlFields, w => WriteSqlFieldsQuery(w, sqlFieldsQuery),
-                s => new ClientFieldsQueryCursor(
-                    _ignite, s.ReadLong(), _keepBinary, s, ClientOp.QuerySqlFieldsCursorGetPage,
-                    ClientFieldsQueryCursor.ReadColumns(_marsh.StartUnmarshal(s))));
+            return DoOutInOp(ClientOp.QuerySqlFields,
+                w => WriteSqlFieldsQuery(w, sqlFieldsQuery),
+                s => GetFieldsCursor(s));
+        }
+
+        /** <inheritDoc /> */
+        public IQueryCursor<T> QueryFields<T>(SqlFieldsQuery sqlFieldsQuery, Func<IBinaryRawReader, int, T> readerFunc)
+        {
+            return DoOutInOp(ClientOp.QuerySqlFields, 
+                w => WriteSqlFieldsQuery(w, sqlFieldsQuery, false),
+                s => GetFieldsCursorNoColumnNames(s, readerFunc));
         }
 
         /** <inheritDoc /> */
@@ -361,6 +370,12 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
         }
 
         /** <inheritDoc /> */
+        CacheConfiguration ICacheInternal.GetConfiguration()
+        {
+            return GetConfiguration().ToCacheConfiguration();
+        }
+
+        /** <inheritDoc /> */
         public ICacheClient<TK1, TV1> WithKeepBinary<TK1, TV1>()
         {
             if (_keepBinary)
@@ -380,6 +395,15 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
             return new CacheClient<TK1, TV1>(_ignite, _name, true);
         }
 
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public T DoOutInOpExtension<T>(int extensionId, int opCode, Action<IBinaryRawWriter> writeAction,
+            Func<IBinaryRawReader, T> readFunc)
+        {
+            // Should not be called, there are no usages for thin client.
+            throw IgniteClient.GetClientNotSupportedException();
+        }
+
         /// <summary>
         /// Does the out in op.
         /// </summary>
@@ -492,7 +516,8 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
         /// <summary>
         /// Writes the SQL fields query.
         /// </summary>
-        private static void WriteSqlFieldsQuery(IBinaryRawWriter writer, SqlFieldsQuery qry)
+        private static void WriteSqlFieldsQuery(IBinaryRawWriter writer, SqlFieldsQuery qry,
+            bool includeColumns = true)
         {
             Debug.Assert(qry != null);
 
@@ -513,10 +538,33 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
             writer.WriteBoolean(qry.Colocated);
             writer.WriteBoolean(qry.Lazy);
             writer.WriteTimeSpanAsLong(qry.Timeout);
+            writer.WriteBoolean(includeColumns);
 
-            // Always include field names.
-            writer.WriteBoolean(true);
+        }
+
+        /// <summary>
+        /// Gets the fields cursor.
+        /// </summary>
+        private ClientFieldsQueryCursor GetFieldsCursor(IBinaryStream s)
+        {
+            var cursorId = s.ReadLong();
+            var columnNames = ClientFieldsQueryCursor.ReadColumns(_marsh.StartUnmarshal(s));
+
+            return new ClientFieldsQueryCursor(_ignite, cursorId, _keepBinary, s,
+                ClientOp.QuerySqlFieldsCursorGetPage, columnNames);
+        }
+
+        /// <summary>
+        /// Gets the fields cursor.
+        /// </summary>
+        private ClientQueryCursorBase<T> GetFieldsCursorNoColumnNames<T>(IBinaryStream s,
+            Func<IBinaryRawReader, int, T> readerFunc)
+        {
+            var cursorId = s.ReadLong();
+            var columnCount = s.ReadInt();
 
+            return new ClientQueryCursorBase<T>(_ignite, cursorId, _keepBinary, s,
+                ClientOp.QuerySqlFieldsCursorGetPage, r => readerFunc(r, columnCount));
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursorBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursorBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursorBase.cs
index 5123537..5a0a1f6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursorBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursorBase.cs
@@ -26,7 +26,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cache.Query
     /// <summary>
     /// Client query cursor base.
     /// </summary>
-    internal abstract class ClientQueryCursorBase<T> : QueryCursorBase<T>
+    internal class ClientQueryCursorBase<T> : QueryCursorBase<T>
     {
         /** Ignite. */
         private readonly IgniteClient _ignite;
@@ -46,7 +46,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cache.Query
         /// <param name="initialBatchStream">Optional stream with initial batch.</param>
         /// <param name="getPageOp">The get page op.</param>
         /// <param name="readFunc">Read func.</param>
-        protected ClientQueryCursorBase(IgniteClient ignite, long cursorId, bool keepBinary, 
+        public ClientQueryCursorBase(IgniteClient ignite, long cursorId, bool keepBinary, 
             IBinaryStream initialBatchStream, ClientOp getPageOp, Func<BinaryReader, T> readFunc) 
             : base(ignite.Marshaller, keepBinary, readFunc, initialBatchStream)
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
index 735e4f2..fc13914 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
@@ -51,7 +51,8 @@
     <Reference Include="System.Core" />
   </ItemGroup>
   <ItemGroup>
-    <Compile Include="CacheExtensions.cs" />
+    <Compile Include="CacheClientLinqExtensions.cs" />
+    <Compile Include="CacheLinqExtensions.cs" />
     <Compile Include="CompiledQuery.cs" />
     <Compile Include="ICacheQueryable.cs" />
     <Compile Include="Impl\AliasDictionary.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Linq/CacheClientLinqExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/CacheClientLinqExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/CacheClientLinqExtensions.cs
new file mode 100644
index 0000000..0c3544b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/CacheClientLinqExtensions.cs
@@ -0,0 +1,131 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Linq
+{
+    using System.Linq;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Client.Cache;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Linq.Impl;
+
+    /// <summary>
+    /// Extensions methods for <see cref="ICacheClient{TK,TV}"/>.
+    /// </summary>
+    public static class CacheClientLinqExtensions
+    {
+        /// <summary>
+        /// Gets an <see cref="IQueryable{T}"/> instance over this cache.
+        /// <para />
+        /// Resulting query will be translated to cache SQL query and executed over the cache instance 
+        /// via either <see cref="ICacheClient{TK,TV}.Query(SqlFieldsQuery)"/>.
+        /// <para />
+        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable"/>
+        /// for introspection, or converted with <see cref="CacheLinqExtensions.ToCacheQueryable{T}"/>
+        /// extension method.
+        /// </summary>
+        /// <typeparam name="TKey">The type of the key.</typeparam>
+        /// <typeparam name="TValue">The type of the value.</typeparam>
+        /// <param name="cache">The cache.</param>
+        /// <returns><see cref="IQueryable{T}"/> instance over this cache.</returns>
+        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
+            this ICacheClient<TKey, TValue> cache)
+        {
+            IgniteArgumentCheck.NotNull(cache, "cache");
+
+            return AsCacheQueryable(cache, false, null);
+        }
+
+        /// <summary>
+        /// Gets an <see cref="IQueryable{T}"/> instance over this cache.
+        /// <para />
+        /// Resulting query will be translated to cache SQL query and executed over the cache instance 
+        /// via either <see cref="ICacheClient{TK,TV}.Query(SqlFieldsQuery)"/>.
+        /// <para />
+        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable"/> for introspection.
+        /// </summary>
+        /// <typeparam name="TKey">The type of the key.</typeparam>
+        /// <typeparam name="TValue">The type of the value.</typeparam>
+        /// <param name="cache">The cache.</param>
+        /// <param name="local">Local flag. When set query will be executed only on local node, so only local 
+        /// entries will be returned as query result.</param>
+        /// <returns><see cref="IQueryable{T}"/> instance over this cache.</returns>
+        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
+            this ICacheClient<TKey, TValue> cache, bool local)
+        {
+            IgniteArgumentCheck.NotNull(cache, "cache");
+
+            return AsCacheQueryable(cache, local, null);
+        }
+
+        /// <summary>
+        /// Gets an <see cref="IQueryable{T}" /> instance over this cache.
+        /// <para />
+        /// Resulting query will be translated to cache SQL query and executed over the cache instance 
+        /// via either <see cref="ICacheClient{TK,TV}.Query(SqlFieldsQuery)"/>.
+        /// <para />
+        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable" /> for introspection.
+        /// </summary>
+        /// <typeparam name="TKey">The type of the key.</typeparam>
+        /// <typeparam name="TValue">The type of the value.</typeparam>
+        /// <param name="cache">The cache.</param>
+        /// <param name="local">Local flag. When set query will be executed only on local node, so only local 
+        /// entries will be returned as query result.</param>
+        /// <param name="tableName">
+        /// Name of the table.
+        /// <para />
+        /// Table name is equal to short class name of a cache value.
+        /// When a cache has only one type of values, or only one <see cref="QueryEntity"/> defined, 
+        /// table name will be inferred and can be omitted.
+        /// </param>
+        /// <returns><see cref="IQueryable{T}" /> instance over this cache.</returns>
+        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
+            this ICacheClient<TKey, TValue> cache, bool local, string tableName)
+        {
+            IgniteArgumentCheck.NotNull(cache, "cache");
+
+            return AsCacheQueryable(cache, new QueryOptions {Local = local, TableName = tableName});
+        }
+
+        /// <summary>
+        /// Gets an <see cref="IQueryable{T}" /> instance over this cache.
+        /// <para />
+        /// Resulting query will be translated to cache SQL query and executed over the cache instance 
+        /// via either <see cref="ICacheClient{TK,TV}.Query(SqlFieldsQuery)"/>.
+        /// <para />
+        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable" /> for introspection.
+        /// </summary>
+        /// <typeparam name="TKey">The type of the key.</typeparam>
+        /// <typeparam name="TValue">The type of the value.</typeparam>
+        /// <param name="cache">The cache.</param>
+        /// <param name="queryOptions">The query options.</param>
+        /// <returns>
+        ///   <see cref="IQueryable{T}" /> instance over this cache.
+        /// </returns>
+        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
+            this ICacheClient<TKey, TValue> cache, QueryOptions queryOptions)
+        {
+            IgniteArgumentCheck.NotNull(cache, "cache");
+            IgniteArgumentCheck.NotNull(queryOptions, "queryOptions");
+
+            return new CacheQueryable<TKey, TValue>((ICacheInternal) cache, queryOptions);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Linq/CacheExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/CacheExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/CacheExtensions.cs
deleted file mode 100644
index f759dbb..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/CacheExtensions.cs
+++ /dev/null
@@ -1,195 +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.
- */
-
-namespace Apache.Ignite.Linq
-{
-    using System;
-    using System.Diagnostics.CodeAnalysis;
-    using System.Linq;
-    using System.Linq.Expressions;
-    using Apache.Ignite.Core.Cache;
-    using Apache.Ignite.Core.Cache.Configuration;
-    using Apache.Ignite.Core.Impl.Common;
-    using Apache.Ignite.Linq.Impl;
-    using Apache.Ignite.Linq.Impl.Dml;
-
-    /// <summary>
-    /// Extensions methods for <see cref="ICache{TK,TV}"/>.
-    /// </summary>
-    public static class CacheLinqExtensions
-    {
-        /// <summary>
-        /// Gets an <see cref="IQueryable{T}"/> instance over this cache.
-        /// <para />
-        /// Resulting query will be translated to cache SQL query and executed over the cache instance 
-        /// via either <see cref="ICache{TK,TV}.Query"/> or <see cref="ICache{TK,TV}.QueryFields"/>,
-        /// depending on requested result. 
-        /// <para />
-        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable"/> for introspection.
-        /// </summary>
-        /// <typeparam name="TKey">The type of the key.</typeparam>
-        /// <typeparam name="TValue">The type of the value.</typeparam>
-        /// <param name="cache">The cache.</param>
-        /// <returns><see cref="IQueryable{T}"/> instance over this cache.</returns>
-        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
-            this ICache<TKey, TValue> cache)
-        {
-            IgniteArgumentCheck.NotNull(cache, "cache");
-
-            return cache.AsCacheQueryable(false, null);
-        }
-
-        /// <summary>
-        /// Gets an <see cref="IQueryable{T}"/> instance over this cache.
-        /// <para />
-        /// Resulting query will be translated to cache SQL query and executed over the cache instance 
-        /// via either <see cref="ICache{TK,TV}.Query"/> or <see cref="ICache{TK,TV}.QueryFields"/>,
-        /// depending on requested result. 
-        /// <para />
-        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable"/> for introspection.
-        /// </summary>
-        /// <typeparam name="TKey">The type of the key.</typeparam>
-        /// <typeparam name="TValue">The type of the value.</typeparam>
-        /// <param name="cache">The cache.</param>
-        /// <param name="local">Local flag. When set query will be executed only on local node, so only local 
-        /// entries will be returned as query result.</param>
-        /// <returns><see cref="IQueryable{T}"/> instance over this cache.</returns>
-        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
-            this ICache<TKey, TValue> cache, bool local)
-        {
-            IgniteArgumentCheck.NotNull(cache, "cache");
-
-            return cache.AsCacheQueryable(local, null);
-        }
-
-        /// <summary>
-        /// Gets an <see cref="IQueryable{T}" /> instance over this cache.
-        /// <para />
-        /// Resulting query will be translated to cache SQL query and executed over the cache instance
-        /// via either <see cref="ICache{TK,TV}.Query" /> or <see cref="ICache{TK,TV}.QueryFields" />,
-        /// depending on requested result.
-        /// <para />
-        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable" /> for introspection.
-        /// </summary>
-        /// <typeparam name="TKey">The type of the key.</typeparam>
-        /// <typeparam name="TValue">The type of the value.</typeparam>
-        /// <param name="cache">The cache.</param>
-        /// <param name="local">Local flag. When set query will be executed only on local node, so only local 
-        /// entries will be returned as query result.</param>
-        /// <param name="tableName">
-        /// Name of the table.
-        /// <para />
-        /// Table name is equal to short class name of a cache value.
-        /// When a cache has only one type of values, or only one <see cref="QueryEntity"/> defined, 
-        /// table name will be inferred and can be omitted.
-        /// </param>
-        /// <returns><see cref="IQueryable{T}" /> instance over this cache.</returns>
-        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
-            this ICache<TKey, TValue> cache, bool local, string tableName)
-        {
-            IgniteArgumentCheck.NotNull(cache, "cache");
-
-            return cache.AsCacheQueryable(new QueryOptions {Local = local, TableName = tableName});
-        }
-
-        /// <summary>
-        /// Gets an <see cref="IQueryable{T}" /> instance over this cache.
-        /// <para />
-        /// Resulting query will be translated to cache SQL query and executed over the cache instance
-        /// via either <see cref="ICache{TK,TV}.Query" /> or <see cref="ICache{TK,TV}.QueryFields" />,
-        /// depending on requested result.
-        /// <para />
-        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable" /> for introspection.
-        /// </summary>
-        /// <typeparam name="TKey">The type of the key.</typeparam>
-        /// <typeparam name="TValue">The type of the value.</typeparam>
-        /// <param name="cache">The cache.</param>
-        /// <param name="queryOptions">The query options.</param>
-        /// <returns>
-        ///   <see cref="IQueryable{T}" /> instance over this cache.
-        /// </returns>
-        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
-            this ICache<TKey, TValue> cache, QueryOptions queryOptions)
-        {
-            IgniteArgumentCheck.NotNull(cache, "cache");
-            IgniteArgumentCheck.NotNull(queryOptions, "queryOptions");
-
-            return new CacheQueryable<TKey, TValue>(cache, queryOptions);
-        }
-
-        /// <summary>
-        /// Casts this query to <see cref="ICacheQueryable"/>.
-        /// </summary>
-        public static ICacheQueryable ToCacheQueryable<T>(this IQueryable<T> query)
-        {
-            IgniteArgumentCheck.NotNull(query, "query");
-
-            return (ICacheQueryable) query;
-        }
-
-        /// <summary>
-        /// Removes all rows that are matched by the specified query.
-        /// <para />
-        /// This method results in "DELETE FROM" distributed SQL query, performing bulk delete 
-        /// (as opposed to fetching all rows locally).
-        /// </summary>
-        /// <typeparam name="TKey">Key type.</typeparam>
-        /// <typeparam name="TValue">Value type.</typeparam>
-        /// <param name="query">The query.</param>
-        /// <returns>Affected row count.</returns>
-        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods",
-            Justification = "Validation is present.")]
-        public static int RemoveAll<TKey, TValue>(this IQueryable<ICacheEntry<TKey, TValue>> query)
-        {
-            IgniteArgumentCheck.NotNull(query, "query");
-
-            var method = RemoveAllExpressionNode.RemoveAllMethodInfo.MakeGenericMethod(typeof(TKey), typeof(TValue));
-
-            return query.Provider.Execute<int>(Expression.Call(null, method, query.Expression));
-        }
-
-        /// <summary>
-        /// Deletes all rows that are matched by the specified query.
-        /// <para />
-        /// This method results in "DELETE FROM" distributed SQL query, performing bulk delete
-        /// (as opposed to fetching all rows locally).
-        /// </summary>
-        /// <typeparam name="TKey">Key type.</typeparam>
-        /// <typeparam name="TValue">Value type.</typeparam>
-        /// <param name="query">The query.</param>
-        /// <param name="predicate">The predicate.</param>
-        /// <returns>
-        /// Affected row count.
-        /// </returns>
-        [SuppressMessage("Microsoft.Design", "CA1011:ConsiderPassingBaseTypesAsParameters",
-            Justification = "Only specified type of predicate is valid.")]
-        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods",
-            Justification = "Validation is present.")]
-        public static int RemoveAll<TKey, TValue>(this IQueryable<ICacheEntry<TKey, TValue>> query, 
-            Expression<Func<ICacheEntry<TKey, TValue>, bool>> predicate)
-        {
-            IgniteArgumentCheck.NotNull(query, "query");
-            IgniteArgumentCheck.NotNull(predicate, "predicate");
-
-            var method = RemoveAllExpressionNode.RemoveAllPredicateMethodInfo
-                .MakeGenericMethod(typeof(TKey), typeof(TValue));
-
-            return query.Provider.Execute<int>(Expression.Call(null, method, query.Expression,
-                Expression.Quote(predicate)));
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Linq/CacheLinqExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/CacheLinqExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/CacheLinqExtensions.cs
new file mode 100644
index 0000000..940b23b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/CacheLinqExtensions.cs
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Linq
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using System.Linq.Expressions;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Linq.Impl;
+    using Apache.Ignite.Linq.Impl.Dml;
+
+    /// <summary>
+    /// Extensions methods for <see cref="ICache{TK,TV}"/>.
+    /// </summary>
+    public static class CacheLinqExtensions
+    {
+        /// <summary>
+        /// Gets an <see cref="IQueryable{T}"/> instance over this cache.
+        /// <para />
+        /// Resulting query will be translated to cache SQL query and executed over the cache instance 
+        /// via either <see cref="ICache{TK,TV}.Query"/> or <see cref="ICache{TK,TV}.QueryFields"/>,
+        /// depending on requested result. 
+        /// <para />
+        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable"/>
+        /// for introspection, or converted with <see cref="ToCacheQueryable{T}"/> extension method.
+        /// </summary>
+        /// <typeparam name="TKey">The type of the key.</typeparam>
+        /// <typeparam name="TValue">The type of the value.</typeparam>
+        /// <param name="cache">The cache.</param>
+        /// <returns><see cref="IQueryable{T}"/> instance over this cache.</returns>
+        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
+            this ICache<TKey, TValue> cache)
+        {
+            IgniteArgumentCheck.NotNull(cache, "cache");
+
+            return cache.AsCacheQueryable(false, null);
+        }
+
+        /// <summary>
+        /// Gets an <see cref="IQueryable{T}"/> instance over this cache.
+        /// <para />
+        /// Resulting query will be translated to cache SQL query and executed over the cache instance 
+        /// via either <see cref="ICache{TK,TV}.Query"/> or <see cref="ICache{TK,TV}.QueryFields"/>,
+        /// depending on requested result. 
+        /// <para />
+        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable"/> for introspection.
+        /// </summary>
+        /// <typeparam name="TKey">The type of the key.</typeparam>
+        /// <typeparam name="TValue">The type of the value.</typeparam>
+        /// <param name="cache">The cache.</param>
+        /// <param name="local">Local flag. When set query will be executed only on local node, so only local 
+        /// entries will be returned as query result.</param>
+        /// <returns><see cref="IQueryable{T}"/> instance over this cache.</returns>
+        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
+            this ICache<TKey, TValue> cache, bool local)
+        {
+            IgniteArgumentCheck.NotNull(cache, "cache");
+
+            return cache.AsCacheQueryable(local, null);
+        }
+
+        /// <summary>
+        /// Gets an <see cref="IQueryable{T}" /> instance over this cache.
+        /// <para />
+        /// Resulting query will be translated to cache SQL query and executed over the cache instance
+        /// via either <see cref="ICache{TK,TV}.Query" /> or <see cref="ICache{TK,TV}.QueryFields" />,
+        /// depending on requested result.
+        /// <para />
+        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable" /> for introspection.
+        /// </summary>
+        /// <typeparam name="TKey">The type of the key.</typeparam>
+        /// <typeparam name="TValue">The type of the value.</typeparam>
+        /// <param name="cache">The cache.</param>
+        /// <param name="local">Local flag. When set query will be executed only on local node, so only local 
+        /// entries will be returned as query result.</param>
+        /// <param name="tableName">
+        /// Name of the table.
+        /// <para />
+        /// Table name is equal to short class name of a cache value.
+        /// When a cache has only one type of values, or only one <see cref="QueryEntity"/> defined, 
+        /// table name will be inferred and can be omitted.
+        /// </param>
+        /// <returns><see cref="IQueryable{T}" /> instance over this cache.</returns>
+        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
+            this ICache<TKey, TValue> cache, bool local, string tableName)
+        {
+            IgniteArgumentCheck.NotNull(cache, "cache");
+
+            return cache.AsCacheQueryable(new QueryOptions {Local = local, TableName = tableName});
+        }
+
+        /// <summary>
+        /// Gets an <see cref="IQueryable{T}" /> instance over this cache.
+        /// <para />
+        /// Resulting query will be translated to cache SQL query and executed over the cache instance
+        /// via either <see cref="ICache{TK,TV}.Query" /> or <see cref="ICache{TK,TV}.QueryFields" />,
+        /// depending on requested result.
+        /// <para />
+        /// Result of this method (and subsequent query) can be cast to <see cref="ICacheQueryable" /> for introspection.
+        /// </summary>
+        /// <typeparam name="TKey">The type of the key.</typeparam>
+        /// <typeparam name="TValue">The type of the value.</typeparam>
+        /// <param name="cache">The cache.</param>
+        /// <param name="queryOptions">The query options.</param>
+        /// <returns>
+        ///   <see cref="IQueryable{T}" /> instance over this cache.
+        /// </returns>
+        public static IQueryable<ICacheEntry<TKey, TValue>> AsCacheQueryable<TKey, TValue>(
+            this ICache<TKey, TValue> cache, QueryOptions queryOptions)
+        {
+            IgniteArgumentCheck.NotNull(cache, "cache");
+            IgniteArgumentCheck.NotNull(queryOptions, "queryOptions");
+
+            return new CacheQueryable<TKey, TValue>((ICacheInternal) cache, queryOptions, cache.Ignite);
+        }
+
+        /// <summary>
+        /// Casts this query to <see cref="ICacheQueryable"/>.
+        /// </summary>
+        public static ICacheQueryable ToCacheQueryable<T>(this IQueryable<T> query)
+        {
+            IgniteArgumentCheck.NotNull(query, "query");
+
+            return (ICacheQueryable) query;
+        }
+
+        /// <summary>
+        /// Removes all rows that are matched by the specified query.
+        /// <para />
+        /// This method results in "DELETE FROM" distributed SQL query, performing bulk delete 
+        /// (as opposed to fetching all rows locally).
+        /// </summary>
+        /// <typeparam name="TKey">Key type.</typeparam>
+        /// <typeparam name="TValue">Value type.</typeparam>
+        /// <param name="query">The query.</param>
+        /// <returns>Affected row count.</returns>
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods",
+            Justification = "Validation is present.")]
+        public static int RemoveAll<TKey, TValue>(this IQueryable<ICacheEntry<TKey, TValue>> query)
+        {
+            IgniteArgumentCheck.NotNull(query, "query");
+
+            var method = RemoveAllExpressionNode.RemoveAllMethodInfo.MakeGenericMethod(typeof(TKey), typeof(TValue));
+
+            return query.Provider.Execute<int>(Expression.Call(null, method, query.Expression));
+        }
+
+        /// <summary>
+        /// Deletes all rows that are matched by the specified query.
+        /// <para />
+        /// This method results in "DELETE FROM" distributed SQL query, performing bulk delete
+        /// (as opposed to fetching all rows locally).
+        /// </summary>
+        /// <typeparam name="TKey">Key type.</typeparam>
+        /// <typeparam name="TValue">Value type.</typeparam>
+        /// <param name="query">The query.</param>
+        /// <param name="predicate">The predicate.</param>
+        /// <returns>
+        /// Affected row count.
+        /// </returns>
+        [SuppressMessage("Microsoft.Design", "CA1011:ConsiderPassingBaseTypesAsParameters",
+            Justification = "Only specified type of predicate is valid.")]
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods",
+            Justification = "Validation is present.")]
+        public static int RemoveAll<TKey, TValue>(this IQueryable<ICacheEntry<TKey, TValue>> query, 
+            Expression<Func<ICacheEntry<TKey, TValue>, bool>> predicate)
+        {
+            IgniteArgumentCheck.NotNull(query, "query");
+            IgniteArgumentCheck.NotNull(predicate, "predicate");
+
+            var method = RemoveAllExpressionNode.RemoveAllPredicateMethodInfo
+                .MakeGenericMethod(typeof(TKey), typeof(TValue));
+
+            return query.Provider.Execute<int>(Expression.Call(null, method, query.Expression,
+                Expression.Quote(predicate)));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Linq/ICacheQueryable.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/ICacheQueryable.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/ICacheQueryable.cs
index ef641e2..426d52c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/ICacheQueryable.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/ICacheQueryable.cs
@@ -37,6 +37,7 @@ namespace Apache.Ignite.Linq
         /// <summary>
         /// Gets the Ignite instance associated with this query.
         /// </summary>
+        [Obsolete("Deprecated, null for thin client.")]
         IIgnite Ignite { get; }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryProvider.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryProvider.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryProvider.cs
index cce89fd..4f35a42 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryProvider.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryProvider.cs
@@ -63,7 +63,6 @@ namespace Apache.Ignite.Linq.Impl
         {
             Debug.Assert(queryParser != null);
             Debug.Assert(executor != null);
-            Debug.Assert(ignite != null);
             Debug.Assert(cacheConfiguration != null);
             Debug.Assert(cacheValueType != null);
 
@@ -85,6 +84,7 @@ namespace Apache.Ignite.Linq.Impl
         /// <summary>
         /// Gets the ignite.
         /// </summary>
+        [Obsolete("Deprecated, null for thin client, only used for ICacheQueryable.")]
         public IIgnite Ignite
         {
             get { return _ignite; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryable.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryable.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryable.cs
index e271363..5148020 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryable.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryable.cs
@@ -18,6 +18,7 @@
 namespace Apache.Ignite.Linq.Impl
 {
     using System.Linq;
+    using Apache.Ignite.Core;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Impl.Cache;
 
@@ -31,10 +32,11 @@ namespace Apache.Ignite.Linq.Impl
         /// </summary>
         /// <param name="cache">The cache.</param>
         /// <param name="queryOptions">The query options.</param>
-        public CacheQueryable(ICache<TKey, TValue> cache, QueryOptions queryOptions)
+        /// <param name="ignite">The ignite.</param>
+        public CacheQueryable(ICacheInternal cache, QueryOptions queryOptions, IIgnite ignite = null)
             : base(new CacheFieldsQueryProvider(CacheQueryParser.Instance,
-                new CacheFieldsQueryExecutor((ICacheInternal) cache, queryOptions), 
-                cache.Ignite, cache.GetConfiguration(), queryOptions.TableName, typeof(TValue)))
+                new CacheFieldsQueryExecutor(cache, queryOptions), 
+                ignite, cache.GetConfiguration(), queryOptions.TableName, typeof(TValue)))
         {
             // No-op.
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs
index 5702f4f..c585119 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs
@@ -55,6 +55,7 @@ namespace Apache.Ignite.Linq.Impl
         }
 
         /** <inheritdoc /> */
+        [Obsolete("Deprecated, null for thin client.")]
         public IIgnite Ignite
         {
             get { return CacheQueryProvider.Ignite; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f433752/modules/platforms/dotnet/Apache.Ignite/IgniteRunner.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite/IgniteRunner.cs b/modules/platforms/dotnet/Apache.Ignite/IgniteRunner.cs
index 6d8aa6b..c6660c3 100644
--- a/modules/platforms/dotnet/Apache.Ignite/IgniteRunner.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/IgniteRunner.cs
@@ -25,7 +25,6 @@ namespace Apache.Ignite
     using System.Threading;
     using Apache.Ignite.Config;
     using Apache.Ignite.Core;
-    using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Service;
 
     /// <summary>


[04/11] ignite git commit: IGNITE-6702: SQL: now COUNT(*) use BPlusTree.size instead of cursor iteration. This closes #3037.

Posted by sb...@apache.org.
IGNITE-6702: SQL: now COUNT(*) use BPlusTree.size instead of cursor iteration. This closes #3037.


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

Branch: refs/heads/ignite-zk
Commit: 08ab9af8a94c53c1cd58d5763f2b9d594f3ca58e
Parents: 8292335
Author: gg-shq <ki...@gmail.com>
Authored: Mon Dec 4 15:25:00 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 4 15:25:00 2017 +0300

----------------------------------------------------------------------
 .../internal/jdbc2/JdbcLocalCachesSelfTest.java |  28 +
 .../cache/persistence/tree/BPlusTree.java       | 141 +++-
 .../processors/database/BPlusTreeSelfTest.java  | 819 ++++++++++++++++++-
 .../query/h2/database/H2TreeIndex.java          |  92 ++-
 ...lexClientAtomicPartitionedNoBackupsTest.java |  34 +
 ...exingComplexClientAtomicPartitionedTest.java |   2 +-
 ...dexingComplexClientAtomicReplicatedTest.java |   2 +-
 ...ntTransactionalPartitionedNoBackupsTest.java |  34 +
 ...mplexClientTransactionalPartitionedTest.java |   2 +-
 ...omplexClientTransactionalReplicatedTest.java |   2 +-
 ...lexServerAtomicPartitionedNoBackupsTest.java |  34 +
 ...exingComplexServerAtomicPartitionedTest.java |   2 +-
 ...dexingComplexServerAtomicReplicatedTest.java |   2 +-
 ...erTransactionalPartitionedNoBackupsTest.java |  34 +
 ...mplexServerTransactionalPartitionedTest.java |   2 +-
 ...omplexServerTransactionalReplicatedTest.java |   2 +-
 .../index/H2DynamicIndexingComplexTest.java     |  22 +-
 .../query/IgniteSqlSegmentedIndexSelfTest.java  | 124 ++-
 18 files changed, 1288 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java
index f096e69..09ccbc9 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java
@@ -129,6 +129,34 @@ public class JdbcLocalCachesSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Verifies that <code>select count(*)</code> behaves correctly in
+     * {@link org.apache.ignite.cache.CacheMode#LOCAL} mode.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCountAll() throws Exception {
+        Properties cfg = new Properties();
+
+        cfg.setProperty(PROP_NODE_ID, grid(0).localNode().id().toString());
+
+        Connection conn = null;
+
+        try {
+            conn = DriverManager.getConnection(BASE_URL, cfg);
+
+            ResultSet rs = conn.createStatement().executeQuery("select count(*) from Integer");
+
+            assertTrue(rs.next());
+
+            assertEquals(2L, rs.getLong(1));
+        }
+        finally {
+            if (conn != null)
+                conn.close();
+        }
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testCache2() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
index 8e6e099..436a69d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
@@ -62,6 +62,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.Bool.DONE;
 import static org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.Bool.FALSE;
@@ -1906,57 +1907,122 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * !!! For debug only! May produce wrong results on concurrent access.
+     * Returns number of elements in the tree by scanning pages of the bottom (leaf) level.
+     * Since a concurrent access is permitted, there is no guarantee about
+     * momentary consistency: the method may miss updates made in already scanned pages.
      *
-     * @return Size.
+     * @return Number of elements in the tree.
      * @throws IgniteCheckedException If failed.
      */
     @Override public final long size() throws IgniteCheckedException {
+        return size(null);
+    }
+
+    /**
+     * Returns number of elements in the tree that match the filter by scanning through the pages of the leaf level.
+     * Since a concurrent access to the tree is permitted, there is no guarantee about
+     * momentary consistency: the method may not see updates made in already scanned pages.
+     *
+     * @param filter The filter to use or null to count all elements.
+     * @return Number of either all elements in the tree or the elements that match the filter.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long size(@Nullable TreeRowClosure<L, T> filter) throws IgniteCheckedException {
         checkDestroyed();
 
-        long pageId;
+        for (;;) {
+            long curPageId;
 
-        long metaPage = acquirePage(metaPageId);
-        try {
-            pageId = getFirstPageId(metaPageId, metaPage, 0); // Level 0 is always at the bottom.
-        }
-        finally {
-            releasePage(metaPageId, metaPage);
-        }
+            long metaPage = acquirePage(metaPageId);
 
-        BPlusIO<L> io = null;
+            try {
+                curPageId = getFirstPageId(metaPageId, metaPage, 0); // Level 0 is always at the bottom.
+            }
+            finally {
+                releasePage(metaPageId, metaPage);
+            }
 
-        long cnt = 0;
+            long cnt = 0;
 
-        while (pageId != 0) {
-            long curId = pageId;
-            long curPage = acquirePage(curId);
+            long curPage = acquirePage(curPageId);
             try {
-                long curAddr = readLock(curId, curPage); // No correctness guaranties.
+                long curPageAddr = readLock(curPageId, curPage);
+
+                if (curPageAddr == 0)
+                    continue; // The first page has gone: restart scan.
 
                 try {
-                    if (io == null) {
-                        io = io(curAddr);
+                    BPlusIO<L> io = io(curPageAddr);
 
-                        assert io.isLeaf();
-                    }
+                    assert io.isLeaf();
+
+                    for (;;) {
+                        int curPageSize = io.getCount(curPageAddr);
+
+                        if (filter == null)
+                            cnt += curPageSize;
+                        else {
+                            for (int i = 0; i < curPageSize; ++i) {
+                                if (filter.apply(this, io, curPageAddr, i))
+                                    cnt++;
+                            }
+                        }
+
+                        long nextPageId = io.getForward(curPageAddr);
+
+                        if (nextPageId == 0) {
+                            checkDestroyed();
+
+                            return cnt;
+                        }
+
+                        long nextPage = acquirePage(nextPageId);
+
+                        try {
+                            long nextPageAddr = readLock(nextPageId, nextPage);
+
+                            // In the current implementation the next page can't change when the current page is locked.
+                            assert nextPageAddr != 0 : nextPageAddr;
+
+                            try {
+                                long pa = curPageAddr;
+                                curPageAddr = 0; // Set to zero to avoid double unlocking in finalizer.
+
+                                readUnlock(curPageId, curPage, pa);
 
-                    cnt += io.getCount(curAddr);
+                                long p = curPage;
+                                curPage = 0; // Set to zero to avoid double release in finalizer.
 
-                    pageId = io.getForward(curAddr);
+                                releasePage(curPageId, p);
+
+                                curPageId = nextPageId;
+                                curPage = nextPage;
+                                curPageAddr = nextPageAddr;
+
+                                nextPage = 0;
+                                nextPageAddr = 0;
+                            }
+                            finally {
+                                if (nextPageAddr != 0)
+                                    readUnlock(nextPageId, nextPage, nextPageAddr);
+                            }
+                        }
+                        finally {
+                            if (nextPage != 0)
+                                releasePage(nextPageId, nextPage);
+                        }
+                    }
                 }
                 finally {
-                    readUnlock(curId, curPage, curAddr);
+                    if (curPageAddr != 0)
+                        readUnlock(curPageId, curPage, curPageAddr);
                 }
             }
             finally {
-                releasePage(curId, curPage);
+                if (curPage != 0)
+                    releasePage(curPageId, curPage);
             }
         }
-
-        checkDestroyed();
-
-        return cnt;
     }
 
     /**
@@ -4803,4 +4869,23 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         DONE
     }
+
+    /**
+     * A generic visitor-style interface for performing filtering/modifications/miscellaneous operations on the tree.
+     */
+    public interface TreeRowClosure<L, T extends L> {
+        /**
+         * Performs inspection or operation on a specified row and returns true if this row is
+         * required or matches or /operation successful (depending on the context).
+         *
+         * @param tree The tree.
+         * @param io Th tree IO object.
+         * @param pageAddr The page address.
+         * @param idx The item index.
+         * @return {@code True} if the item passes the predicate.
+         * @throws IgniteCheckedException If failed.
+         */
+        public boolean apply(BPlusTree<L, T> tree, BPlusIO<L> io, long pageAddr, int idx)
+            throws IgniteCheckedException;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index 7b4ca13..85d269f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -17,14 +17,22 @@
 
 package org.apache.ignite.internal.processors.database;
 
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -32,6 +40,8 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLongArray;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
+
+import com.google.common.base.Predicate;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -104,6 +114,9 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     /** */
     private static int RMV_INC = 1;
 
+    /** Forces printing lock/unlock events on the test tree */
+    private static boolean PRINT_LOCKS = false;
+
     /** */
     protected PageMemory pageMem;
 
@@ -1077,10 +1090,6 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         for (long i = 15; i >= 0; i--)
             tree.put(i);
-
-
-
-
     }
 
     /**
@@ -1156,6 +1165,790 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Verifies that {@link BPlusTree#size} and {@link BPlusTree#size} methods behave correctly
+     * on single-threaded addition and removal of elements in random order.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testSizeForPutRmvSequential() throws IgniteCheckedException {
+        MAX_PER_PAGE = 5;
+
+        boolean DEBUG_PRINT = false;
+
+        int itemCnt = (int) Math.pow(MAX_PER_PAGE, 5) + rnd.nextInt(MAX_PER_PAGE * MAX_PER_PAGE);
+
+        Long[] items = new Long[itemCnt];
+        for (int i = 0; i < itemCnt; ++i)
+            items[i] = (long) i;
+
+        TestTree testTree = createTestTree(true);
+        TreeMap<Long,Long> goldenMap = new TreeMap<>();
+
+        assertEquals(0, testTree.size());
+        assertEquals(0, goldenMap.size());
+
+        final Predicate<Long> rowMatcher = new Predicate<Long>() {
+            @Override public boolean apply(Long row) {
+                return row % 7 == 0;
+            }
+        };
+
+        final BPlusTree.TreeRowClosure<Long, Long> rowClosure = new BPlusTree.TreeRowClosure<Long, Long>() {
+            @Override public boolean apply(BPlusTree<Long, Long> tree, BPlusIO<Long> io, long pageAddr, int idx)
+                throws IgniteCheckedException {
+                return rowMatcher.apply(io.getLookupRow(tree, pageAddr, idx));
+            }
+        };
+
+        int correctMatchingRows = 0;
+
+        Collections.shuffle(Arrays.asList(items), rnd);
+
+        for (Long row : items) {
+            if (DEBUG_PRINT) {
+                X.println(" --> put(" + row + ")");
+                X.print(testTree.printTree());
+            }
+
+            assertEquals(goldenMap.put(row, row), testTree.put(row));
+            assertEquals(row, testTree.findOne(row));
+
+            if (rowMatcher.apply(row))
+                ++correctMatchingRows;
+
+            assertEquals(correctMatchingRows, testTree.size(rowClosure));
+
+            long correctSize = goldenMap.size();
+
+            assertEquals(correctSize, testTree.size());
+            assertEquals(correctSize, size(testTree.find(null, null)));
+
+            assertNoLocks();
+        }
+
+        Collections.shuffle(Arrays.asList(items), rnd);
+
+        for (Long row : items) {
+            if (DEBUG_PRINT) {
+                X.println(" --> rmv(" + row + ")");
+                X.print(testTree.printTree());
+            }
+
+            assertEquals(row, goldenMap.remove(row));
+            assertEquals(row, testTree.remove(row));
+            assertNull(testTree.findOne(row));
+
+            if (rowMatcher.apply(row))
+                --correctMatchingRows;
+
+            assertEquals(correctMatchingRows, testTree.size(rowClosure));
+
+            long correctSize = goldenMap.size();
+
+            assertEquals(correctSize, testTree.size());
+            assertEquals(correctSize, size(testTree.find(null, null)));
+
+            assertNoLocks();
+        }
+    }
+
+    /**
+     * Verifies that {@link BPlusTree#size()} method behaves correctly when run concurrently with
+     * {@link BPlusTree#put}, {@link BPlusTree#remove} methods. Please see details in
+     * {@link #doTestSizeForRandomPutRmvMultithreaded}.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSizeForRandomPutRmvMultithreaded_5_4() throws Exception {
+        MAX_PER_PAGE = 5;
+        CNT = 10_000;
+
+        doTestSizeForRandomPutRmvMultithreaded(4);
+    }
+
+    public void testSizeForRandomPutRmvMultithreaded_3_256() throws Exception {
+        MAX_PER_PAGE = 3;
+        CNT = 10_000;
+
+        doTestSizeForRandomPutRmvMultithreaded(256);
+    }
+
+    /**
+     * Verifies that {@link BPlusTree#size()} method behaves correctly when run between series of
+     * concurrent {@link BPlusTree#put}, {@link BPlusTree#remove} methods.
+     *
+     * @param rmvPutSlidingWindowSize Sliding window size (distance between items being deleted and added).
+     * @throws Exception If failed.
+     */
+    private void doTestSizeForRandomPutRmvMultithreaded(final int rmvPutSlidingWindowSize) throws Exception {
+        final TestTree tree = createTestTree(false);
+
+        final boolean DEBUG_PRINT = false;
+
+        final AtomicLong curRmvKey = new AtomicLong(0);
+        final AtomicLong curPutKey = new AtomicLong(rmvPutSlidingWindowSize);
+
+        for (long i = curRmvKey.get(); i < curPutKey.get(); ++i)
+            assertNull(tree.put(i));
+
+        final int putRmvThreadCnt = Math.min(Runtime.getRuntime().availableProcessors(), rmvPutSlidingWindowSize);
+
+        final int loopCnt = CNT / putRmvThreadCnt;
+
+        final CyclicBarrier putRmvOpBarrier = new CyclicBarrier(putRmvThreadCnt);
+        final CyclicBarrier sizeOpBarrier = new CyclicBarrier(putRmvThreadCnt);
+
+        IgniteInternalFuture<?> putRmvFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+
+                for (int i = 0; i < loopCnt && !stop.get(); ++i) {
+                    putRmvOpBarrier.await();
+
+                    Long putVal = curPutKey.getAndIncrement();
+
+                    if (DEBUG_PRINT || (i & 0x7ff) == 0)
+                        X.println(" --> put(" + putVal + ")");
+
+                    assertNull(tree.put(putVal));
+
+                    assertNoLocks();
+
+                    Long rmvVal = curRmvKey.getAndIncrement();
+
+                    if (DEBUG_PRINT || (i & 0x7ff) == 0)
+                        X.println(" --> rmv(" + rmvVal + ")");
+
+                    assertEquals(rmvVal, tree.remove(rmvVal));
+                    assertNull(tree.remove(rmvVal));
+
+                    assertNoLocks();
+
+                    if (stop.get())
+                        break;
+
+                    sizeOpBarrier.await();
+
+                    long correctSize = curPutKey.get() - curRmvKey.get();
+
+                    if (DEBUG_PRINT || (i & 0x7ff) == 0)
+                        X.println("====> correctSize=" + correctSize);
+
+                    assertEquals(correctSize, size(tree.find(null, null)));
+                    assertEquals(correctSize, tree.size());
+                }
+
+                return null;
+            }
+        }, putRmvThreadCnt, "put-remove-size");
+
+        IgniteInternalFuture<?> lockPrintingFut = multithreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                while (!stop.get()) {
+                    Thread.sleep(5000);
+
+                    X.println(TestTree.printLocks());
+                }
+
+                return null;
+            }
+        }, 1, "printLocks");
+
+        asyncRunFut = new GridCompoundFuture<>();
+
+        asyncRunFut.add((IgniteInternalFuture) putRmvFut);
+        asyncRunFut.add((IgniteInternalFuture) lockPrintingFut);
+
+        asyncRunFut.markInitialized();
+
+        try {
+            putRmvFut.get(getTestTimeout(), TimeUnit.MILLISECONDS);
+        }
+        finally {
+            stop.set(true);
+            putRmvOpBarrier.reset();
+            sizeOpBarrier.reset();
+
+            asyncRunFut.get();
+        }
+
+        tree.validateTree();
+
+        assertNoLocks();
+    }
+
+    /**
+     * Verifies that concurrent running of {@link BPlusTree#put} + {@link BPlusTree#remove} sequence
+     * and {@link BPlusTree#size} methods results in correct calculation of tree size.
+     *
+     * @see #doTestSizeForRandomPutRmvMultithreadedAsync doTestSizeForRandomPutRmvMultithreadedAsync() for details.
+     */
+    public void testSizeForRandomPutRmvMultithreadedAsync_16() throws Exception {
+        doTestSizeForRandomPutRmvMultithreadedAsync(16);
+    }
+
+    /**
+     * Verifies that concurrent running of {@link BPlusTree#put} + {@link BPlusTree#remove} sequence
+     * and {@link BPlusTree#size} methods results in correct calculation of tree size.
+     *
+     * @see #doTestSizeForRandomPutRmvMultithreadedAsync doTestSizeForRandomPutRmvMultithreadedAsync() for details.
+     */
+    public void testSizeForRandomPutRmvMultithreadedAsync_3() throws Exception {
+        doTestSizeForRandomPutRmvMultithreadedAsync(3);
+    }
+
+    /**
+     * Verifies that concurrent running of {@link BPlusTree#put} + {@link BPlusTree#remove} sequence
+     * and {@link BPlusTree#size} methods results in correct calculation of tree size.
+     *
+     * Since in the presence of concurrent modifications the size may differ from the actual one, the test maintains
+     * sliding window of records in the tree, uses a barrier between concurrent runs to limit runaway delta in
+     * the calculated size, and checks that the measured size lies within certain bounds.
+     *
+     * NB: This test has to be changed with the integration of IGNITE-3478.
+     *
+     */
+    public void doTestSizeForRandomPutRmvMultithreadedAsync(final int rmvPutSlidingWindowSize) throws Exception {
+        MAX_PER_PAGE = 5;
+
+        final boolean DEBUG_PRINT = false;
+
+        final TestTree tree = createTestTree(false);
+
+        final AtomicLong curRmvKey = new AtomicLong(0);
+        final AtomicLong curPutKey = new AtomicLong(rmvPutSlidingWindowSize);
+
+        for (long i = curRmvKey.get(); i < curPutKey.get(); ++i)
+            assertNull(tree.put(i));
+
+        final int putRmvThreadCnt = Math.min(Runtime.getRuntime().availableProcessors(), rmvPutSlidingWindowSize);
+        final int sizeThreadCnt = putRmvThreadCnt;
+
+        final CyclicBarrier putRmvOpBarrier = new CyclicBarrier(putRmvThreadCnt + sizeThreadCnt, new Runnable() {
+            @Override public void run() {
+                if (DEBUG_PRINT) {
+                    try {
+                        X.println("===BARRIER=== size=" + tree.size()
+                            + "; contents=[" + tree.findFirst() + ".." + tree.findLast() + "]"
+                            + "; rmvVal=" + curRmvKey.get() + "; putVal=" + curPutKey.get());
+
+                        X.println(tree.printTree());
+                    }
+                    catch (IgniteCheckedException e) {
+                        // ignore
+                    }
+                }
+            }
+        });
+
+        final int loopCnt = 500;
+
+        IgniteInternalFuture<?> putRmvFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (int i = 0; i < loopCnt && !stop.get(); ++i) {
+                    int order;
+                    try {
+                        order = putRmvOpBarrier.await();
+                    } catch (BrokenBarrierException e) {
+                        break;
+                    }
+
+                    Long putVal = curPutKey.getAndIncrement();
+
+                    if (DEBUG_PRINT || (i & 0x3ff) == 0)
+                        X.println(order + ": --> put(" + putVal + ")");
+
+                    assertNull(tree.put(putVal));
+
+                    Long rmvVal = curRmvKey.getAndIncrement();
+
+                    if (DEBUG_PRINT || (i & 0x3ff) == 0)
+                        X.println(order + ": --> rmv(" + rmvVal + ")");
+
+                    assertEquals(rmvVal, tree.remove(rmvVal));
+                    assertNull(tree.findOne(rmvVal));
+                }
+
+                return null;
+            }
+        }, putRmvThreadCnt, "put-remove");
+
+        IgniteInternalFuture<?> sizeFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+
+                final List<Long> treeContents = new ArrayList<>(rmvPutSlidingWindowSize * 2);
+
+                final BPlusTree.TreeRowClosure<Long, Long> rowDumper = new BPlusTree.TreeRowClosure<Long, Long>() {
+                    @Override public boolean apply(BPlusTree<Long, Long> tree, BPlusIO<Long> io, long pageAddr, int idx)
+                        throws IgniteCheckedException {
+
+                        treeContents.add(io.getLookupRow(tree, pageAddr, idx));
+                        return true;
+                    }
+                };
+
+                for (long iter = 0; !stop.get(); ++iter) {
+                    int order = 0;
+
+                    try {
+                        order = putRmvOpBarrier.await();
+                    } catch (BrokenBarrierException e) {
+                        break;
+                    }
+
+                    long correctSize = curPutKey.get() - curRmvKey.get();
+
+                    treeContents.clear();
+                    long treeSize = tree.size(rowDumper);
+
+                    long minBound = correctSize - putRmvThreadCnt;
+                    long maxBound = correctSize + putRmvThreadCnt;
+
+                    if (DEBUG_PRINT || (iter & 0x3ff) == 0)
+                      X.println(order + ": size=" + treeSize + "; bounds=[" + minBound + ".." + maxBound
+                            + "]; contents=" + treeContents);
+
+                    if (treeSize < minBound || treeSize > maxBound) {
+                        fail("Tree size is not in bounds ["  + minBound + ".." + maxBound + "]: " + treeSize
+                            + "; Tree contents: " + treeContents);
+                    }
+                }
+
+                return null;
+            }
+        }, sizeThreadCnt, "size");
+
+        IgniteInternalFuture<?> lockPrintingFut = multithreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                while (!stop.get()) {
+                    Thread.sleep(5000);
+
+                    X.println(TestTree.printLocks());
+                }
+
+                return null;
+            }
+        }, 1, "printLocks");
+
+        asyncRunFut = new GridCompoundFuture<>();
+
+        asyncRunFut.add((IgniteInternalFuture) putRmvFut);
+        asyncRunFut.add((IgniteInternalFuture) sizeFut);
+        asyncRunFut.add((IgniteInternalFuture) lockPrintingFut);
+
+        asyncRunFut.markInitialized();
+
+        try {
+            putRmvFut.get(getTestTimeout(), TimeUnit.MILLISECONDS);
+        }
+        finally {
+            stop.set(true);
+            putRmvOpBarrier.reset();
+
+            asyncRunFut.get();
+        }
+
+        tree.validateTree();
+
+        assertNoLocks();
+    }
+
+    /**
+     * The test forces {@link BPlusTree#size} method to run into a livelock: during single run
+     * the method is picking up new pages which are concurrently added to the tree until the new pages are not added
+     * anymore. Test verifies that despite livelock condition a size from a valid range is returned.
+     *
+     * NB: This test has to be changed with the integration of IGNITE-3478.
+     *
+     * @throws Exception if test failed
+     */
+    public void testPutSizeLivelock() throws Exception {
+        MAX_PER_PAGE = 5;
+        CNT = 800;
+
+        final int SLIDING_WINDOW_SIZE = 16;
+        final boolean DEBUG_PRINT = false;
+
+        final TestTree tree = createTestTree(false);
+
+        final AtomicLong curRmvKey = new AtomicLong(0);
+        final AtomicLong curPutKey = new AtomicLong(SLIDING_WINDOW_SIZE);
+
+        for (long i = curRmvKey.get(); i < curPutKey.get(); ++i)
+            assertNull(tree.put(i));
+
+        final int hwThreads = Runtime.getRuntime().availableProcessors();
+        final int putRmvThreadCnt = Math.max(1, hwThreads / 2);
+        final int sizeThreadCnt = hwThreads - putRmvThreadCnt;
+
+        final CyclicBarrier putRmvOpBarrier = new CyclicBarrier(putRmvThreadCnt, new Runnable() {
+            @Override public void run() {
+                if (DEBUG_PRINT) {
+                    try {
+                        X.println("===BARRIER=== size=" + tree.size()
+                            + " [" + tree.findFirst() + ".." + tree.findLast() + "]");
+                    }
+                    catch (IgniteCheckedException e) {
+                        // ignore
+                    }
+                }
+            }
+        });
+
+        final int loopCnt = CNT / hwThreads;
+
+        IgniteInternalFuture<?> putRmvFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (int i = 0; i < loopCnt && !stop.get(); ++i) {
+                    int order;
+                    try {
+                        order = putRmvOpBarrier.await();
+                    } catch (BrokenBarrierException e) {
+                        // barrier reset() has been called: terminate
+                        break;
+                    }
+
+                    Long putVal = curPutKey.getAndIncrement();
+
+                    if ((i & 0xff) == 0)
+                        X.println(order + ": --> put(" + putVal + ")");
+
+                    assertNull(tree.put(putVal));
+
+                    Long rmvVal = curRmvKey.getAndIncrement();
+
+                    if ((i & 0xff) == 0)
+                        X.println(order + ": --> rmv(" + rmvVal + ")");
+
+                    assertEquals(rmvVal, tree.remove(rmvVal));
+                    assertNull(tree.findOne(rmvVal));
+                }
+
+                return null;
+            }
+        }, putRmvThreadCnt, "put-remove");
+
+        IgniteInternalFuture<?> sizeFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+
+                final List<Long> treeContents = new ArrayList<>(SLIDING_WINDOW_SIZE * 2);
+
+                final BPlusTree.TreeRowClosure<Long, Long> rowDumper = new BPlusTree.TreeRowClosure<Long, Long>() {
+                    @Override public boolean apply(BPlusTree<Long, Long> tree, BPlusIO<Long> io, long pageAddr, int idx)
+                        throws IgniteCheckedException {
+
+                        treeContents.add(io.getLookupRow(tree, pageAddr, idx));
+
+                        final long endMs = System.currentTimeMillis() + 10;
+                        final long endPutKey = curPutKey.get() + MAX_PER_PAGE;
+
+                        while (System.currentTimeMillis() < endMs && curPutKey.get() < endPutKey)
+                            Thread.yield();
+
+                        return true;
+                    }
+                };
+
+                while (!stop.get()) {
+                    treeContents.clear();
+
+                    long treeSize = tree.size(rowDumper);
+                    long curPutVal = curPutKey.get();
+
+                    X.println(" ======> size=" + treeSize + "; last-put-value=" + curPutVal);
+
+                    if (treeSize < SLIDING_WINDOW_SIZE || treeSize > curPutVal)
+                        fail("Tree size is not in bounds [" + SLIDING_WINDOW_SIZE + ".." + curPutVal + "]:"
+                            + treeSize + "; contents=" + treeContents);
+                }
+
+                return null;
+            }
+        }, sizeThreadCnt, "size");
+
+        asyncRunFut = new GridCompoundFuture<>();
+
+        asyncRunFut.add((IgniteInternalFuture) putRmvFut);
+        asyncRunFut.add((IgniteInternalFuture) sizeFut);
+
+        asyncRunFut.markInitialized();
+
+        try {
+            putRmvFut.get(getTestTimeout(), TimeUnit.MILLISECONDS);
+        }
+        finally {
+            stop.set(true);
+            putRmvOpBarrier.reset();
+
+            asyncRunFut.get();
+        }
+
+        tree.validateTree();
+
+        assertNoLocks();
+    }
+
+    /**
+     * Verifies that in case for threads concurrently calling put and remove
+     * on a tree with 1-3 pages, the size() method performs correctly.
+     *
+     * @throws Exception If failed.
+     */
+    public void testPutRmvSizeSinglePageContention() throws Exception {
+        MAX_PER_PAGE = 10;
+        CNT = 20_000;
+        final boolean DEBUG_PRINT = false;
+        final int SLIDING_WINDOWS_SIZE = MAX_PER_PAGE * 2;
+
+        final TestTree tree = createTestTree(false);
+
+        final AtomicLong curPutKey = new AtomicLong(0);
+        final BlockingQueue<Long> rowsToRemove = new ArrayBlockingQueue<>(MAX_PER_PAGE / 2);
+
+        final int hwThreadCnt = Runtime.getRuntime().availableProcessors();
+        final int putThreadCnt = Math.max(1, hwThreadCnt / 4);
+        final int rmvThreadCnt = Math.max(1, hwThreadCnt / 2 - putThreadCnt);
+        final int sizeThreadCnt = Math.max(1, hwThreadCnt - putThreadCnt - rmvThreadCnt);
+
+        final AtomicInteger sizeInvokeCnt = new AtomicInteger(0);
+
+        final int loopCnt = CNT;
+
+        IgniteInternalFuture<?> sizeFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int iter = 0;
+                while (!stop.get()) {
+                    long size = tree.size();
+
+                    if (DEBUG_PRINT || (++iter & 0xffff) == 0)
+                        X.println(" --> size() = " + size);
+
+                    sizeInvokeCnt.incrementAndGet();
+                }
+
+                return null;
+            }
+        }, sizeThreadCnt, "size");
+
+        // Let the size threads ignite
+        while (sizeInvokeCnt.get() < sizeThreadCnt * 2)
+            Thread.yield();
+
+        IgniteInternalFuture<?> rmvFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int iter = 0;
+                while(!stop.get()) {
+                    Long rmvVal = rowsToRemove.poll(200, TimeUnit.MILLISECONDS);
+                    if (rmvVal != null)
+                        assertEquals(rmvVal, tree.remove(rmvVal));
+
+                    if (DEBUG_PRINT || (++iter & 0x3ff) == 0)
+                        X.println(" --> rmv(" + rmvVal + ")");
+                }
+
+                return null;
+            }
+        }, rmvThreadCnt, "rmv");
+
+        IgniteInternalFuture<?> putFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (int i = 0; i < loopCnt && !stop.get(); ++i) {
+                    Long putVal = curPutKey.getAndIncrement();
+                    assertNull(tree.put(putVal));
+
+                    while (rowsToRemove.size() > SLIDING_WINDOWS_SIZE && !stop.get())
+                        Thread.yield();
+
+                    rowsToRemove.put(putVal);
+
+                    if (DEBUG_PRINT || (i & 0x3ff) == 0)
+                        X.println(" --> put(" + putVal + ")");
+                }
+
+                return null;
+            }
+        }, putThreadCnt, "put");
+
+        IgniteInternalFuture<?> treePrintFut = multithreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                while (!stop.get()) {
+                    Thread.sleep(1000);
+
+                    X.println(TestTree.printLocks());
+                    X.println(tree.printTree());
+                }
+
+                return null;
+            }
+        }, 1, "printTree");
+
+        asyncRunFut = new GridCompoundFuture<>();
+
+        asyncRunFut.add((IgniteInternalFuture) sizeFut);
+        asyncRunFut.add((IgniteInternalFuture) rmvFut);
+        asyncRunFut.add((IgniteInternalFuture) putFut);
+        asyncRunFut.add((IgniteInternalFuture) treePrintFut);
+
+        asyncRunFut.markInitialized();
+
+        try {
+            putFut.get(getTestTimeout(), TimeUnit.MILLISECONDS);
+        }
+        finally {
+            stop.set(true);
+
+            asyncRunFut.get();
+        }
+
+        tree.validateTree();
+
+        assertNoLocks();
+    }
+
+    /**
+     * The test verifies that {@link BPlusTree#put}, {@link BPlusTree#remove}, {@link BPlusTree#find}, and
+     * {@link BPlusTree#size} run concurrently, perform correctly and report correct values.
+     *
+     * A sliding window of numbers is maintainted in the tests.
+     *
+     * NB: This test has to be changed with the integration of IGNITE-3478.
+     *
+     * @throws Exception If failed.
+     */
+    public void testPutRmvFindSizeMultithreaded() throws Exception {
+        MAX_PER_PAGE = 5;
+        CNT = 60_000;
+
+        final int SLIDING_WINDOW_SIZE = 100;
+
+        final TestTree tree = createTestTree(false);
+
+        final AtomicLong curPutKey = new AtomicLong(0);
+        final BlockingQueue<Long> rowsToRemove = new ArrayBlockingQueue<>(SLIDING_WINDOW_SIZE);
+
+        final int hwThreadCnt = Runtime.getRuntime().availableProcessors();
+        final int putThreadCnt = Math.max(1, hwThreadCnt / 4);
+        final int rmvThreadCnt = Math.max(1, hwThreadCnt / 4);
+        final int findThreadCnt = Math.max(1, hwThreadCnt / 4);
+        final int sizeThreadCnt = Math.max(1, hwThreadCnt - putThreadCnt - rmvThreadCnt - findThreadCnt);
+
+        final AtomicInteger sizeInvokeCnt = new AtomicInteger(0);
+
+        final int loopCnt = CNT;
+
+        IgniteInternalFuture<?> sizeFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int iter = 0;
+                while (!stop.get()) {
+                    long size = tree.size();
+
+                    if ((++iter & 0x3ff) == 0)
+                        X.println(" --> size() = " + size);
+
+                    sizeInvokeCnt.incrementAndGet();
+                }
+
+                return null;
+            }
+        }, sizeThreadCnt, "size");
+
+        // Let the size threads start
+        while (sizeInvokeCnt.get() < sizeThreadCnt * 2)
+            Thread.yield();
+
+        IgniteInternalFuture<?> rmvFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int iter = 0;
+                while(!stop.get()) {
+                    Long rmvVal = rowsToRemove.poll(200, TimeUnit.MILLISECONDS);
+                    if (rmvVal != null)
+                        assertEquals(rmvVal, tree.remove(rmvVal));
+
+                    if ((++iter & 0x3ff) == 0)
+                        X.println(" --> rmv(" + rmvVal + ")");
+                }
+
+                return null;
+            }
+        }, rmvThreadCnt, "rmv");
+
+        IgniteInternalFuture<?> findFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int iter = 0;
+                while(!stop.get()) {
+                    Long findVal = curPutKey.get()
+                        + SLIDING_WINDOW_SIZE / 2
+                        - rnd.nextInt(SLIDING_WINDOW_SIZE * 2);
+
+                    tree.findOne(findVal);
+
+                    if ((++iter & 0x3ff) == 0)
+                        X.println(" --> fnd(" + findVal + ")");
+                }
+
+                return null;
+            }
+        }, findThreadCnt, "find");
+
+        IgniteInternalFuture<?> putFut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (int i = 0; i < loopCnt && !stop.get(); ++i) {
+                    Long putVal = curPutKey.getAndIncrement();
+                    assertNull(tree.put(putVal));
+
+                    while (rowsToRemove.size() > SLIDING_WINDOW_SIZE) {
+                        if (stop.get())
+                            return null;
+
+                        Thread.yield();
+                    }
+
+                    rowsToRemove.put(putVal);
+
+                    if ((i & 0x3ff) == 0)
+                        X.println(" --> put(" + putVal + ")");
+                }
+
+                return null;
+            }
+        }, putThreadCnt, "put");
+
+        IgniteInternalFuture<?> lockPrintingFut = multithreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                while (!stop.get()) {
+                    Thread.sleep(1000);
+
+                    X.println(TestTree.printLocks());
+                }
+
+                return null;
+            }
+        }, 1, "printLocks");
+
+        asyncRunFut = new GridCompoundFuture<>();
+
+        asyncRunFut.add((IgniteInternalFuture) sizeFut);
+        asyncRunFut.add((IgniteInternalFuture) rmvFut);
+        asyncRunFut.add((IgniteInternalFuture) findFut);
+        asyncRunFut.add((IgniteInternalFuture) putFut);
+        asyncRunFut.add((IgniteInternalFuture) lockPrintingFut);
+
+        asyncRunFut.markInitialized();
+
+        try {
+            putFut.get(getTestTimeout(), TimeUnit.MILLISECONDS);
+        }
+        finally {
+            stop.set(true);
+
+            asyncRunFut.get();
+        }
+
+        tree.validateTree();
+
+        assertNoLocks();
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testTestRandomPutRemoveMultithreaded_1_30_0() throws Exception {
@@ -1620,7 +2413,8 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void onBeforeReadLock(int cacheId, long pageId, long page) {
-//            X.println("  onBeforeReadLock: " + U.hexLong(page.id()));
+            if (PRINT_LOCKS)
+                X.println("  onBeforeReadLock: " + U.hexLong(pageId));
 //
 //            U.dumpStack();
 
@@ -1629,7 +2423,8 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void onReadLock(int cacheId, long pageId, long page, long pageAddr) {
-//            X.println("  onReadLock: " + U.hexLong(page.id()));
+            if (PRINT_LOCKS)
+                X.println("  onReadLock: " + U.hexLong(pageId));
 
             if (pageAddr != 0L) {
                 long actual = PageIO.getPageId(pageAddr);
@@ -1644,7 +2439,8 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void onReadUnlock(int cacheId, long pageId, long page, long pageAddr) {
-//            X.println("  onReadUnlock: " + U.hexLong(page.id()));
+            if (PRINT_LOCKS)
+                X.println("  onReadUnlock: " + U.hexLong(pageId));
 
             checkPageId(pageId, pageAddr);
 
@@ -1655,14 +2451,16 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void onBeforeWriteLock(int cacheId, long pageId, long page) {
-//            X.println("  onBeforeWriteLock: " + U.hexLong(page.id()));
+            if (PRINT_LOCKS)
+                X.println("  onBeforeWriteLock: " + U.hexLong(pageId));
 
             assertNull(beforeWriteLock.put(threadId(), pageId));
         }
 
         /** {@inheritDoc} */
         @Override public void onWriteLock(int cacheId, long pageId, long page, long pageAddr) {
-//            X.println("  onWriteLock: " + U.hexLong(page.id()));
+            if (PRINT_LOCKS)
+                X.println("  onWriteLock: " + U.hexLong(pageId));
 //
 //            U.dumpStack();
 
@@ -1682,7 +2480,8 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void onWriteUnlock(int cacheId, long pageId, long page, long pageAddr) {
-//            X.println("  onWriteUnlock: " + U.hexLong(page.id()));
+            if (PRINT_LOCKS)
+                X.println("  onWriteUnlock: " + U.hexLong(pageId));
 
             assertEquals(effectivePageId(pageId), effectivePageId(PageIO.getPageId(pageAddr)));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/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 4ebac88..5a336c5 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
@@ -23,11 +23,14 @@ import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.persistence.RootPage;
 import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.query.h2.H2Cursor;
+import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
 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;
@@ -165,20 +168,13 @@ public class H2TreeIndex extends GridH2IndexBase {
     /** {@inheritDoc} */
     @Override public Cursor find(Session ses, SearchRow lower, SearchRow upper) {
         try {
-            IndexingQueryFilter f = threadLocalFilter();
-            IndexingQueryCacheFilter p = null;
-
-            if (f != null) {
-                String cacheName = getTable().cacheName();
-
-                p = f.forCache(cacheName);
-            }
+            IndexingQueryCacheFilter filter = partitionFilter(threadLocalFilter());
 
             int seg = threadLocalSegment();
 
             H2Tree tree = treeForRead(seg);
 
-            return new H2Cursor(tree.find(lower, upper, p));
+            return new H2Cursor(tree.find(lower, upper, filter));
         }
         catch (IgniteCheckedException e) {
             throw DbException.convert(e);
@@ -274,14 +270,59 @@ public class H2TreeIndex extends GridH2IndexBase {
 
     /** {@inheritDoc} */
     @Override public long getRowCount(Session ses) {
-        Cursor cursor = find(ses, null, null);
+        try {
+            int seg = threadLocalSegment();
 
-        long res = 0;
+            H2Tree tree = treeForRead(seg);
 
-        while (cursor.next())
-            res++;
+            BPlusTree.TreeRowClosure<SearchRow, GridH2Row> filter = filter();
 
-        return res;
+            return tree.size(filter);
+        }
+        catch (IgniteCheckedException e) {
+            throw DbException.convert(e);
+        }
+    }
+
+    /**
+     * An adapter from {@link IndexingQueryCacheFilter} to {@link BPlusTree.TreeRowClosure} to
+     * filter entries that belong to the current partition.
+     */
+    private static class PartitionFilterTreeRowClosure implements BPlusTree.TreeRowClosure<SearchRow, GridH2Row> {
+        private final IndexingQueryCacheFilter filter;
+
+        /**
+         * Creates a {@link BPlusTree.TreeRowClosure} adapter based on the given partition filter.
+         *
+         * @param filter The partition filter.
+         */
+        public PartitionFilterTreeRowClosure(IndexingQueryCacheFilter filter) {
+            this.filter = filter;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(BPlusTree<SearchRow, GridH2Row> tree,
+            BPlusIO<SearchRow> io, long pageAddr, int idx) throws IgniteCheckedException {
+
+            H2RowLinkIO h2io = (H2RowLinkIO)io;
+
+            return filter.applyPartition(
+                PageIdUtils.partId(
+                    PageIdUtils.pageId(
+                        h2io.getLink(pageAddr, idx))));
+        }
+    }
+
+    /**
+     * Returns a filter to apply to rows in the current index to obtain only the
+     * ones owned by the this cache.
+     *
+     * @return The filter, which returns true for rows owned by this cache.
+     */
+    @Nullable private BPlusTree.TreeRowClosure<SearchRow, GridH2Row> filter() {
+        final IndexingQueryCacheFilter filter = partitionFilter(threadLocalFilter());
+
+        return filter != null ? new PartitionFilterTreeRowClosure(filter) : null;
     }
 
     /** {@inheritDoc} */
@@ -344,13 +385,7 @@ public class H2TreeIndex extends GridH2IndexBase {
         @Nullable SearchRow last,
         IndexingQueryFilter filter) {
         try {
-            IndexingQueryCacheFilter p = null;
-
-            if (filter != null) {
-                String cacheName = getTable().cacheName();
-
-                p = filter.forCache(cacheName);
-            }
+            IndexingQueryCacheFilter p = partitionFilter(filter);
 
             GridCursor<GridH2Row> range = t.find(first, last, p);
 
@@ -365,6 +400,21 @@ public class H2TreeIndex extends GridH2IndexBase {
     }
 
     /**
+     * Filter which returns true for entries belonging to a particular partition.
+     *
+     * @param qryFilter Factory that creates a predicate for filtering entries for a particular cache.
+     * @return The filter or null if the filter is not needed (e.g., if the cache is not partitioned).
+     */
+    @Nullable private IndexingQueryCacheFilter partitionFilter(@Nullable IndexingQueryFilter qryFilter) {
+        if (qryFilter == null)
+            return null;
+
+        String cacheName = getTable().cacheName();
+
+        return qryFilter.forCache(cacheName);
+    }
+
+    /**
      * @param inlineIdxs Inline index helpers.
      * @param cfgInlineSize Inline size from cache config.
      * @return Inline size.

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicPartitionedNoBackupsTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicPartitionedNoBackupsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicPartitionedNoBackupsTest.java
new file mode 100644
index 0000000..25be1ed
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicPartitionedNoBackupsTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.index;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ * Test to check work of DML+DDL operations of atomic partitioned cache without backups
+ * with queries initiated from client node.
+ */
+public class H2DynamicIndexingComplexClientAtomicPartitionedNoBackupsTest extends H2DynamicIndexingComplexTest {
+    /**
+     * Constructor.
+     */
+    public H2DynamicIndexingComplexClientAtomicPartitionedNoBackupsTest() {
+        super(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC, 0, CLIENT_IDX);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicPartitionedTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicPartitionedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicPartitionedTest.java
index 78eddbf..a05390f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicPartitionedTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicPartitionedTest.java
@@ -28,6 +28,6 @@ public class H2DynamicIndexingComplexClientAtomicPartitionedTest extends H2Dynam
      * Constructor.
      */
     public H2DynamicIndexingComplexClientAtomicPartitionedTest() {
-        super(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC, CLIENT_IDX);
+        super(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC, 1, CLIENT_IDX);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicReplicatedTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicReplicatedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicReplicatedTest.java
index 0e1004c..6962eff 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicReplicatedTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientAtomicReplicatedTest.java
@@ -28,6 +28,6 @@ public class H2DynamicIndexingComplexClientAtomicReplicatedTest extends H2Dynami
      * Constructor.
      */
     public H2DynamicIndexingComplexClientAtomicReplicatedTest() {
-        super(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC, CLIENT_IDX);
+        super(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC, 1, CLIENT_IDX);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalPartitionedNoBackupsTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalPartitionedNoBackupsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalPartitionedNoBackupsTest.java
new file mode 100644
index 0000000..bccb38e
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalPartitionedNoBackupsTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.index;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ * Test to check work of DML+DDL operations of transactional partitioned cache without backups
+ * with queries initiated from client node.
+ */
+public class H2DynamicIndexingComplexClientTransactionalPartitionedNoBackupsTest extends H2DynamicIndexingComplexTest {
+    /**
+     * Constructor.
+     */
+    public H2DynamicIndexingComplexClientTransactionalPartitionedNoBackupsTest() {
+        super(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL, 0, CLIENT_IDX);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalPartitionedTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalPartitionedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalPartitionedTest.java
index 6dead30..8ec73cf 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalPartitionedTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalPartitionedTest.java
@@ -28,6 +28,6 @@ public class H2DynamicIndexingComplexClientTransactionalPartitionedTest extends
      * Constructor.
      */
     public H2DynamicIndexingComplexClientTransactionalPartitionedTest() {
-        super(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL, CLIENT_IDX);
+        super(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL, 1, CLIENT_IDX);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalReplicatedTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalReplicatedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalReplicatedTest.java
index 3c73d2c..6000277 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalReplicatedTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexClientTransactionalReplicatedTest.java
@@ -28,6 +28,6 @@ public class H2DynamicIndexingComplexClientTransactionalReplicatedTest extends H
      * Constructor.
      */
     public H2DynamicIndexingComplexClientTransactionalReplicatedTest() {
-        super(CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL, CLIENT_IDX);
+        super(CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL, 1, CLIENT_IDX);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicPartitionedNoBackupsTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicPartitionedNoBackupsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicPartitionedNoBackupsTest.java
new file mode 100644
index 0000000..6e806f9
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicPartitionedNoBackupsTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.index;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ * Test to check work of DML+DDL operations of atomic partitioned cache without backups
+ * with queries initiated from server node.
+ */
+public class H2DynamicIndexingComplexServerAtomicPartitionedNoBackupsTest extends H2DynamicIndexingComplexTest {
+    /**
+     * Constructor.
+     */
+    public H2DynamicIndexingComplexServerAtomicPartitionedNoBackupsTest() {
+        super(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC, 0, SRV_IDX);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicPartitionedTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicPartitionedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicPartitionedTest.java
index ff0c1cb..18f4456 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicPartitionedTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicPartitionedTest.java
@@ -28,6 +28,6 @@ public class H2DynamicIndexingComplexServerAtomicPartitionedTest extends H2Dynam
      * Constructor.
      */
     public H2DynamicIndexingComplexServerAtomicPartitionedTest() {
-        super(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC, SRV_IDX);
+        super(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC, 1, SRV_IDX);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicReplicatedTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicReplicatedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicReplicatedTest.java
index 3d7ee18..2bfe678 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicReplicatedTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerAtomicReplicatedTest.java
@@ -28,6 +28,6 @@ public class H2DynamicIndexingComplexServerAtomicReplicatedTest extends H2Dynami
      * Constructor.
      */
     public H2DynamicIndexingComplexServerAtomicReplicatedTest() {
-        super(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC, SRV_IDX);
+        super(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC, 1, SRV_IDX);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest.java
new file mode 100644
index 0000000..37b4489
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.index;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ * Test to check work of DML+DDL operations of transactional partitioned cache without backups
+ * with queries initiated from server node.
+ */
+public class H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest extends H2DynamicIndexingComplexTest {
+    /**
+     * Constructor.
+     */
+    public H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest() {
+        super(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL, 0, SRV_IDX);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalPartitionedTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalPartitionedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalPartitionedTest.java
index aeb3839..85a58c1 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalPartitionedTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalPartitionedTest.java
@@ -28,6 +28,6 @@ public class H2DynamicIndexingComplexServerTransactionalPartitionedTest extends
      * Constructor.
      */
     public H2DynamicIndexingComplexServerTransactionalPartitionedTest() {
-        super(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL, SRV_IDX);
+        super(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL, 1, SRV_IDX);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalReplicatedTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalReplicatedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalReplicatedTest.java
index 4266161..54329b1 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalReplicatedTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexServerTransactionalReplicatedTest.java
@@ -28,6 +28,6 @@ public class H2DynamicIndexingComplexServerTransactionalReplicatedTest extends H
      * Constructor.
      */
     public H2DynamicIndexingComplexServerTransactionalReplicatedTest() {
-        super(CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL, SRV_IDX);
+        super(CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL, 1, SRV_IDX);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexTest.java
index f9d3408..68df58b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexingComplexTest.java
@@ -45,6 +45,9 @@ public abstract class H2DynamicIndexingComplexTest extends DynamicIndexAbstractS
     /** Node index to initiate operations from. */
     private final int nodeIdx;
 
+    /** Backups to configure */
+    private final int backups;
+
     /** Names of companies to use. */
     private final static List<String> COMPANIES = Arrays.asList("ASF", "GNU", "BSD");
 
@@ -61,11 +64,13 @@ public abstract class H2DynamicIndexingComplexTest extends DynamicIndexAbstractS
      * Constructor.
      * @param cacheMode Cache mode.
      * @param atomicityMode Cache atomicity mode.
+     * @param backups Number of backups.
      * @param nodeIdx Node index.
      */
-    H2DynamicIndexingComplexTest(CacheMode cacheMode, CacheAtomicityMode atomicityMode, int nodeIdx) {
+    H2DynamicIndexingComplexTest(CacheMode cacheMode, CacheAtomicityMode atomicityMode, int backups, int nodeIdx) {
         this.cacheMode = cacheMode;
         this.atomicityMode = atomicityMode;
+        this.backups = backups;
         this.nodeIdx = nodeIdx;
     }
 
@@ -94,12 +99,13 @@ public abstract class H2DynamicIndexingComplexTest extends DynamicIndexAbstractS
     public void testOperations() {
         executeSql("CREATE TABLE person (id int, name varchar, age int, company varchar, city varchar, " +
             "primary key (id, name, city)) WITH \"template=" + cacheMode.name() + ",atomicity=" + atomicityMode.name() +
-            ",affinity_key=city\"");
+            ",backups=" + backups + ",affinity_key=city\"");
 
         executeSql("CREATE INDEX idx on person (city asc, name asc)");
 
         executeSql("CREATE TABLE city (name varchar, population int, primary key (name)) WITH " +
-            "\"template=" + cacheMode.name() + ",atomicity=" + atomicityMode.name() + ",affinity_key=name\"");
+            "\"template=" + cacheMode.name() + ",atomicity=" + atomicityMode.name() +
+            ",backups=" + backups + ",affinity_key=name\"");
 
         executeSql("INSERT INTO city (name, population) values(?, ?), (?, ?), (?, ?)",
             "St. Petersburg", 6000000,
@@ -107,7 +113,9 @@ public abstract class H2DynamicIndexingComplexTest extends DynamicIndexAbstractS
             "London", 8000000
         );
 
-        for (int i = 0; i < 100; i++)
+        final long PERSON_COUNT = 100;
+
+        for (int i = 0; i < PERSON_COUNT; i++)
             executeSql("INSERT INTO person (id, name, age, company, city) values (?, ?, ?, ?, ?)",
                 i,
                 "Person " + i,
@@ -121,7 +129,11 @@ public abstract class H2DynamicIndexingComplexTest extends DynamicIndexAbstractS
             }
         });
 
-        long r = (Long)executeSqlSingle("SELECT COUNT(*) from Person p inner join City c on p.city = c.name");
+        long r = (Long)executeSqlSingle("SELECT COUNT(*) from Person");
+
+        assertEquals(PERSON_COUNT, r);
+
+        r = (Long)executeSqlSingle("SELECT COUNT(*) from Person p inner join City c on p.city = c.name");
 
         // Berkeley is not present in City table, although 25 people have it specified as their city.
         assertEquals(75L, r);

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ab9af8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java
index 03c3f1e..dd03274 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java
@@ -132,6 +132,8 @@ public class IgniteSqlSegmentedIndexSelfTest extends GridCommonAbstractTest {
         checkDistributedQueryWithSegmentedIndex();
 
         checkLocalQueryWithSegmentedIndex();
+
+        checkLocalSizeQueryWithSegmentedIndex();
     }
 
     /**
@@ -144,18 +146,43 @@ public class IgniteSqlSegmentedIndexSelfTest extends GridCommonAbstractTest {
         IgniteCache<Object, Object> cache = ignite(0).cache(ORG_CACHE_NAME);
 
         // Unequal entries distribution among partitions.
-        int expectedSize = nodesCount() * QRY_PARALLELISM_LVL *  3 / 2;
+        int expSize = nodesCount() * QRY_PARALLELISM_LVL *  3 / 2;
 
-        for (int i = 0; i < expectedSize; i++)
+        for (int i = 0; i < expSize; i++)
             cache.put(i, new Organization("org-" + i));
 
         String select0 = "select * from \"org\".Organization o";
 
         // Check for stable results.
         for(int i = 0; i < 10; i++) {
-            List<List<?>> result = cache.query(new SqlFieldsQuery(select0)).getAll();
+            List<List<?>> res = cache.query(new SqlFieldsQuery(select0)).getAll();
 
-            assertEquals(expectedSize, result.size());
+            assertEquals(expSize, res.size());
+        }
+    }
+
+    /**
+     * Checks correct <code>select count(*)</code> result with segmented indices.
+     * @throws Exception If failed.
+     */
+    public void testSegmentedIndexSizeReproducableResults() throws Exception {
+        ignite(0).createCache(cacheConfig(ORG_CACHE_NAME, true, Integer.class, Organization.class));
+
+        IgniteCache<Object, Object> cache = ignite(0).cache(ORG_CACHE_NAME);
+
+        // Unequal entries distribution among partitions.
+        long expSize = nodesCount() * QRY_PARALLELISM_LVL *  3 / 2;
+
+        for (int i = 0; i < expSize; i++)
+            cache.put(i, new Organization("org-" + i));
+
+        String select0 = "select count(*) from \"org\".Organization o";
+
+        // Check for stable results.
+        for(int i = 0; i < 10; i++) {
+            List<List<?>> res = cache.query(new SqlFieldsQuery(select0)).getAll();
+
+            assertEquals(expSize, res.get(0).get(0));
         }
     }
 
@@ -170,14 +197,39 @@ public class IgniteSqlSegmentedIndexSelfTest extends GridCommonAbstractTest {
                 .setEvictionPolicy(new FifoEvictionPolicy(10))
                 .setOnheapCacheEnabled(true));
 
-        for (int i = 0; i < 20; i++)
+        final long SIZE = 20;
+
+        for (int i = 0; i < SIZE; i++)
             cache.put(i, new Organization("org-" + i));
 
         String select0 = "select name from \"org\".Organization";
 
-        List<List<?>> result = cache.query(new SqlFieldsQuery(select0)).getAll();
+        List<List<?>> res = cache.query(new SqlFieldsQuery(select0)).getAll();
 
-        assertEquals(20, result.size());
+        assertEquals(SIZE, res.size());
+    }
+
+    /**
+     * Verifies that <code>select count(*)</code> return valid result on a single-node grid.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSizeOnSegmentedIndexWithEvictionPolicy() throws Exception {
+        final IgniteCache<Object, Object> cache = ignite(0).createCache(
+            cacheConfig(ORG_CACHE_NAME, true, Integer.class, Organization.class)
+                .setEvictionPolicy(new FifoEvictionPolicy(10))
+                .setOnheapCacheEnabled(true));
+
+        final long SIZE = 20;
+
+        for (int i = 0; i < SIZE; i++)
+            cache.put(i, new Organization("org-" + i));
+
+        String select0 = "select count(*) from \"org\".Organization";
+
+        List<List<?>> res = cache.query(new SqlFieldsQuery(select0)).getAll();
+
+        assertEquals(SIZE, res.get(0).get(0));
     }
 
     /**
@@ -194,6 +246,8 @@ public class IgniteSqlSegmentedIndexSelfTest extends GridCommonAbstractTest {
         checkDistributedQueryWithSegmentedIndex();
 
         checkLocalQueryWithSegmentedIndex();
+
+        checkLocalSizeQueryWithSegmentedIndex();
     }
 
     /**
@@ -205,21 +259,21 @@ public class IgniteSqlSegmentedIndexSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < nodesCount(); i++) {
             IgniteCache<Integer, Person> c1 = ignite(i).cache(PERSON_CAHE_NAME);
 
-            int expectedPersons = 0;
+            long expPersons = 0;
 
             for (Cache.Entry<Integer, Person> e : c1) {
                 final Integer orgId = e.getValue().orgId;
 
                 // We have as orphan ORG rows as orphan PERSON rows.
                 if (ORPHAN_ROWS <= orgId && orgId < 500)
-                    expectedPersons++;
+                    expPersons++;
             }
 
             String select0 = "select o.name n1, p.name n2 from \"pers\".Person p, \"org\".Organization o where p.orgId = o._key";
 
-            List<List<?>> result = c1.query(new SqlFieldsQuery(select0).setDistributedJoins(true)).getAll();
+            List<List<?>> res = c1.query(new SqlFieldsQuery(select0).setDistributedJoins(true)).getAll();
 
-            assertEquals(expectedPersons, result.size());
+            assertEquals(expPersons, res.size());
         }
     }
 
@@ -235,25 +289,59 @@ public class IgniteSqlSegmentedIndexSelfTest extends GridCommonAbstractTest {
             IgniteCache<Integer, Person> c1 = node.cache(PERSON_CAHE_NAME);
             IgniteCache<Integer, Organization> c2 = node.cache(ORG_CACHE_NAME);
 
-            Set<Integer> localOrgIds = new HashSet<>();
+            Set<Integer> locOrgIds = new HashSet<>();
 
             for (Cache.Entry<Integer, Organization> e : c2.localEntries())
-                localOrgIds.add(e.getKey());
+                locOrgIds.add(e.getKey());
 
-            int expectedPersons = 0;
+            long expPersons = 0;
 
             for (Cache.Entry<Integer, Person> e : c1.localEntries()) {
                 final Integer orgId = e.getValue().orgId;
 
-                if (localOrgIds.contains(orgId))
-                    expectedPersons++;
+                if (locOrgIds.contains(orgId))
+                    expPersons++;
             }
 
             String select0 = "select o.name n1, p.name n2 from \"pers\".Person p, \"org\".Organization o where p.orgId = o._key";
 
-            List<List<?>> result = c1.query(new SqlFieldsQuery(select0).setLocal(true)).getAll();
+            List<List<?>> res = c1.query(new SqlFieldsQuery(select0).setLocal(true)).getAll();
+
+            assertEquals(expPersons, res.size());
+        }
+    }
+
+    /**
+     * Verifies that local <code>select count(*)</code> query returns a correct result.
+     *
+     * @throws Exception If failed.
+     */
+    public void checkLocalSizeQueryWithSegmentedIndex() throws Exception {
+        for (int i = 0; i < nodesCount(); i++) {
+            final Ignite node = ignite(i);
+
+            IgniteCache<Integer, Person> c1 = node.cache(PERSON_CAHE_NAME);
+            IgniteCache<Integer, Organization> c2 = node.cache(ORG_CACHE_NAME);
+
+            Set<Integer> locOrgIds = new HashSet<>();
+
+            for (Cache.Entry<Integer, Organization> e : c2.localEntries())
+                locOrgIds.add(e.getKey());
+
+            int expPersons = 0;
+
+            for (Cache.Entry<Integer, Person> e : c1.localEntries()) {
+                final Integer orgId = e.getValue().orgId;
+
+                if (locOrgIds.contains(orgId))
+                    expPersons++;
+            }
+
+            String select0 = "select count(*) from \"pers\".Person p, \"org\".Organization o where p.orgId = o._key";
+
+            List<List<?>> res = c1.query(new SqlFieldsQuery(select0).setLocal(true)).getAll();
 
-            assertEquals(expectedPersons, result.size());
+            assertEquals((long) expPersons, res.get(0).get(0));
         }
     }
 


[11/11] ignite git commit: zk

Posted by sb...@apache.org.
zk


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

Branch: refs/heads/ignite-zk
Commit: be7ae489bb2cb9b1f45fecc0d54a5a6d6029c34b
Parents: 017f8cb
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 5 10:28:48 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 5 10:28:48 2017 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java      | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be7ae489/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
index 2d2ff01..f62706e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
@@ -203,10 +203,13 @@ public class ZookeeperDiscoverySpi extends IgniteSpiAdapter implements Discovery
                     Integer commPort = (Integer)locNodeAttrs.get(
                         TcpCommunicationSpi.class.getSimpleName() + "." + TcpCommunicationSpi.ATTR_PORT);
 
-                    if (commPort == null)
-                        throw new IgniteSpiException("Can not initialized consistent ID.");
+                    if (commPort == null) {
+                        U.warn(log, "Can not initialize default consistentId, TcpCommunicationSpi port is not initialized.");
 
-                    consistentId = U.consistentId(sortedAddrs, (Integer)commPort);
+                        consistentId = ignite.configuration().getNodeId();
+                    }
+                    else
+                        consistentId = U.consistentId(sortedAddrs, commPort);
                 }
             }
         }