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 2018/11/20 14:54:44 UTC

[01/50] [abbrv] ignite git commit: IGNITE-10253: SQL: Merged SqlQuery execution logic with SqlFieldsQuery. Now the first one is simply an additional converter on top of the second. This closes #5395.

Repository: ignite
Updated Branches:
  refs/heads/ignite-10044 9a0351994 -> 763ea3253


IGNITE-10253: SQL: Merged SqlQuery execution logic with SqlFieldsQuery. Now the first one is simply an additional converter on top of the second. This closes #5395.


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

Branch: refs/heads/ignite-10044
Commit: c4ec543a8b0da9c5d9a2e25739ac31fd68d13a5d
Parents: a802074
Author: devozerov <pp...@gmail.com>
Authored: Thu Nov 15 22:40:22 2018 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Thu Nov 15 22:40:22 2018 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryIndexing.java     |  23 +-
 .../processors/query/GridQueryProcessor.java    | 275 ++++----
 .../processors/query/QueryKeyValueIterable.java |  53 ++
 .../processors/query/QueryKeyValueIterator.java |  68 ++
 ...IgniteClientCacheInitializationFailTest.java |  19 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 285 ++-------
 ...CacheAbstractQueryDetailMetricsSelfTest.java |  15 +-
 .../CacheAbstractQueryMetricsSelfTest.java      |  13 -
 ...niteCacheLockPartitionOnAffinityRunTest.java |   2 +
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 639 +------------------
 ...H2ResultSetIteratorNullifyOnEndSelfTest.java | 186 ------
 11 files changed, 337 insertions(+), 1241 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 60b0221..3eb732c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -21,11 +21,9 @@ import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.List;
-import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
-import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.internal.GridKernalContext;
@@ -70,16 +68,13 @@ public interface GridQueryIndexing {
     public void onClientDisconnect() throws IgniteCheckedException;
 
     /**
-     * Parses SQL query into two step query and executes it.
+     * Generate SqlFieldsQuery from SqlQuery.
      *
-     * @param schemaName Schema name.
      * @param cacheName Cache name.
      * @param qry Query.
-     * @param keepBinary Keep binary flag.
-     * @throws IgniteCheckedException If failed.
+     * @return Fields query.
      */
-    public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(String schemaName, String cacheName, SqlQuery qry,
-        boolean keepBinary) throws IgniteCheckedException;
+    public SqlFieldsQuery generateFieldsQuery(String cacheName, SqlQuery qry);
 
     /**
      * Detect whether SQL query should be executed in distributed or local manner and execute it.
@@ -121,18 +116,6 @@ public interface GridQueryIndexing {
         SqlClientContext cliCtx) throws IgniteCheckedException;
 
     /**
-     * Executes regular query.
-     *
-     * @param schemaName Schema name.
-     * @param cacheName Cache name.
-     * @param qry Query.
-     * @param filter Cache name and key filter.
-     * @param keepBinary Keep binary flag.    @return Cursor.
-     */
-    public <K, V> QueryCursor<Cache.Entry<K,V>> queryLocalSql(String schemaName, String cacheName, SqlQuery qry,
-        IndexingQueryFilter filter, boolean keepBinary) throws IgniteCheckedException;
-
-    /**
      * Queries individual fields (generally used by JDBC drivers).
      *
      * @param schemaName Schema name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 957d3ae..30c28b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
@@ -2078,7 +2079,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     public List<FieldsQueryCursor<List<?>>> querySqlFields(final SqlFieldsQuery qry, final boolean keepBinary,
         final boolean failOnMultipleStmts) {
-        return querySqlFields(null, qry, null, keepBinary, failOnMultipleStmts);
+        return querySqlFields(
+            null,
+            qry,
+            null,
+            keepBinary,
+            failOnMultipleStmts
+        );
     }
 
     /**
@@ -2089,7 +2096,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Cursor.
      */
     public FieldsQueryCursor<List<?>> querySqlFields(final SqlFieldsQuery qry, final boolean keepBinary) {
-        return querySqlFields(null, qry, null, keepBinary, true).get(0);
+        return querySqlFields(
+            null,
+            qry,
+            null,
+            keepBinary,
+            true
+        ).get(0);
     }
 
     /**
@@ -2103,40 +2116,92 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      *      more then one SQL statement.
      * @return Cursor.
      */
-    public List<FieldsQueryCursor<List<?>>> querySqlFields(@Nullable final GridCacheContext<?, ?> cctx,
-        final SqlFieldsQuery qry, final SqlClientContext cliCtx, final boolean keepBinary,
-        final boolean failOnMultipleStmts) {
-        checkxEnabled();
-
-        validateSqlFieldsQuery(qry, ctx, cctx);
+    public List<FieldsQueryCursor<List<?>>> querySqlFields(
+        @Nullable final GridCacheContext<?, ?> cctx,
+        final SqlFieldsQuery qry,
+        final SqlClientContext cliCtx,
+        final boolean keepBinary,
+        final boolean failOnMultipleStmts
+    ) {
+        return querySqlFields(
+            cctx,
+            qry,
+            cliCtx,
+            keepBinary,
+            failOnMultipleStmts,
+            GridCacheQueryType.SQL_FIELDS
+        );
+    }
 
-        if (!busyLock.enterBusy())
-            throw new IllegalStateException("Failed to execute query (grid is stopping).");
+    /**
+     * Query SQL fields.
+     *
+     * @param cctx Cache context.
+     * @param qry Query.
+     * @param cliCtx Client context.
+     * @param keepBinary Keep binary flag.
+     * @param failOnMultipleStmts If {@code true} the method must throws exception when query contains
+     *      more then one SQL statement.
+     * @param qryType Real query type.
+     * @return Cursor.
+     */
+    public List<FieldsQueryCursor<List<?>>> querySqlFields(
+        @Nullable final GridCacheContext<?, ?> cctx,
+        final SqlFieldsQuery qry,
+        final SqlClientContext cliCtx,
+        final boolean keepBinary,
+        final boolean failOnMultipleStmts,
+        GridCacheQueryType qryType
+    ) {
+        // Validate.
+        checkxEnabled();
 
-        GridCacheContext oldCctx = curCache.get();
+        if (qry.isDistributedJoins() && qry.getPartitions() != null)
+            throw new CacheException("Using both partitions and distributed JOINs is not supported for the same query");
 
-        curCache.set(cctx);
+        if (qry.isLocal() && ctx.clientNode() && (cctx == null || cctx.config().getCacheMode() != CacheMode.LOCAL))
+            throw new CacheException("Execution of local SqlFieldsQuery on client node disallowed.");
 
-        final String schemaName = qry.getSchema() != null ? qry.getSchema()
-            : (cctx != null ? idx.schema(cctx.name()) : QueryUtils.DFLT_SCHEMA);
+        return executeQuerySafe(cctx, () -> {
+            final String schemaName = qry.getSchema() != null ? qry.getSchema()
+                : (cctx != null ? idx.schema(cctx.name()) : QueryUtils.DFLT_SCHEMA);
 
-        try {
             IgniteOutClosureX<List<FieldsQueryCursor<List<?>>>> clo =
                 new IgniteOutClosureX<List<FieldsQueryCursor<List<?>>>>() {
-                @Override public List<FieldsQueryCursor<List<?>>> applyx() throws IgniteCheckedException {
-                    GridQueryCancel cancel = new GridQueryCancel();
+                    @Override public List<FieldsQueryCursor<List<?>>> applyx() {
+                        GridQueryCancel cancel = new GridQueryCancel();
 
-                    List<FieldsQueryCursor<List<?>>> res =
-                        idx.querySqlFields(schemaName, qry, cliCtx, keepBinary, failOnMultipleStmts, null, cancel);
+                        List<FieldsQueryCursor<List<?>>> res =
+                            idx.querySqlFields(schemaName, qry, cliCtx, keepBinary, failOnMultipleStmts, null, cancel);
 
-                    if (cctx != null)
-                        sendQueryExecutedEvent(qry.getSql(), qry.getArgs(), cctx);
+                        if (cctx != null)
+                            sendQueryExecutedEvent(qry.getSql(), qry.getArgs(), cctx);
 
-                    return res;
-                }
-            };
+                        return res;
+                    }
+                };
+
+            return executeQuery(qryType, qry.getSql(), cctx, clo, true);
+        });
+    }
+
+    /**
+     * Execute query setting busy lock, preserving current cache context and properly handling checked exceptions.
+     *
+     * @param cctx Cache context.
+     * @param supplier Code to be executed.
+     * @return Result.
+     */
+    private <T> T executeQuerySafe(@Nullable final GridCacheContext<?, ?> cctx, SupplierX<T> supplier) {
+        GridCacheContext oldCctx = curCache.get();
+
+        curCache.set(cctx);
+
+        if (!busyLock.enterBusy())
+            throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
-            return executeQuery(GridCacheQueryType.SQL_FIELDS, qry.getSql(), cctx, clo, true);
+        try {
+            return supplier.get();
         }
         catch (IgniteCheckedException e) {
             throw new CacheException(e);
@@ -2149,34 +2214,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Validate SQL fields query.
-     *
-     * @param qry Query.
-     * @param ctx Kernal context.
-     * @param cctx Cache context.
-     */
-    private static void validateSqlFieldsQuery(SqlFieldsQuery qry, GridKernalContext ctx,
-        @Nullable GridCacheContext<?, ?> cctx) {
-        if (qry.isDistributedJoins() && qry.getPartitions() != null)
-            throw new CacheException("Using both partitions and distributed JOINs is not supported for the same query");
-
-        if (qry.isLocal() && ctx.clientNode() && (cctx == null || cctx.config().getCacheMode() != CacheMode.LOCAL))
-            throw new CacheException("Execution of local SqlFieldsQuery on client node disallowed.");
-    }
-
-    /**
-     * Validate SQL query.
-     *
-     * @param qry Query.
-     * @param ctx Kernal context.
-     * @param cctx Cache context.
-     */
-    private static void validateSqlQuery(SqlQuery qry, GridKernalContext ctx, GridCacheContext<?, ?> cctx) {
-        if (qry.isLocal() && ctx.clientNode() && cctx.config().getCacheMode() != CacheMode.LOCAL)
-            throw new CacheException("Execution of local SqlQuery on client node disallowed.");
-    }
-
-    /**
      * @param cacheName Cache name.
      * @param schemaName Schema name.
      * @param streamer Data streamer.
@@ -2242,99 +2279,38 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
-    public <K, V> QueryCursor<Cache.Entry<K,V>> querySql(final GridCacheContext<?,?> cctx, final SqlQuery qry,
-        boolean keepBinary) {
-        validateSqlQuery(qry, ctx, cctx);
+    public <K, V> QueryCursor<Cache.Entry<K,V>> querySql(
+        final GridCacheContext<?,?> cctx,
+        final SqlQuery qry,
+        boolean keepBinary
+    ) {
+        // Generate.
+        String type = qry.getType();
 
-        if (qry.isReplicatedOnly() && qry.getPartitions() != null)
-            throw new CacheException("Partitions are not supported in replicated only mode.");
+        String typeName = typeName(cctx.name(), type);
 
-        if (qry.isDistributedJoins() && qry.getPartitions() != null)
-            throw new CacheException(
-                "Using both partitions and distributed JOINs is not supported for the same query");
-
-        if ((qry.isReplicatedOnly() && cctx.isReplicatedAffinityNode()) || cctx.isLocal() || qry.isLocal())
-            return queryLocalSql(cctx, qry, keepBinary);
-
-        return queryDistributedSql(cctx, qry, keepBinary);
-    }
+        qry.setType(typeName);
 
-    /**
-     * @param cctx Cache context.
-     * @param qry Query.
-     * @param keepBinary Keep binary flag.
-     * @return Cursor.
-     */
-    private <K,V> QueryCursor<Cache.Entry<K,V>> queryDistributedSql(final GridCacheContext<?,?> cctx,
-        final SqlQuery qry, final boolean keepBinary) {
-        checkxEnabled();
+        SqlFieldsQuery fieldsQry = idx.generateFieldsQuery(cctx.name(), qry);
 
-        if (!busyLock.enterBusy())
-            throw new IllegalStateException("Failed to execute query (grid is stopping).");
+        // Execute.
+        FieldsQueryCursor<List<?>> res = querySqlFields(
+            cctx,
+            fieldsQry,
+            null,
+            keepBinary,
+            true,
+            GridCacheQueryType.SQL
+        ).get(0);
 
-        try {
-            final String schemaName = idx.schema(cctx.name());
+        // Convert.
+        QueryKeyValueIterable<K, V>converted = new QueryKeyValueIterable<>(res);
 
-            return executeQuery(GridCacheQueryType.SQL, qry.getSql(), cctx,
-                new IgniteOutClosureX<QueryCursor<Cache.Entry<K, V>>>() {
-                    @Override public QueryCursor<Cache.Entry<K, V>> applyx() throws IgniteCheckedException {
-                        return idx.queryDistributedSql(schemaName, cctx.name(), qry, keepBinary);
-                    }
-                }, true);
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteException(e);
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
-    }
-
-    /**
-     * @param cctx Cache context.
-     * @param qry Query.
-     * @param keepBinary Keep binary flag.
-     * @return Cursor.
-     */
-    private <K, V> QueryCursor<Cache.Entry<K, V>> queryLocalSql(final GridCacheContext<?, ?> cctx, final SqlQuery qry,
-        final boolean keepBinary) {
-        if (!busyLock.enterBusy())
-            throw new IllegalStateException("Failed to execute query (grid is stopping).");
-
-        final String schemaName = idx.schema(cctx.name());
-
-        try {
-            return executeQuery(GridCacheQueryType.SQL, qry.getSql(), cctx,
-                new IgniteOutClosureX<QueryCursor<Cache.Entry<K, V>>>() {
-                    @Override public QueryCursor<Cache.Entry<K, V>> applyx() throws IgniteCheckedException {
-                        String type = qry.getType();
-
-                        String typeName = typeName(cctx.name(), type);
-
-                        qry.setType(typeName);
-
-                        sendQueryExecutedEvent(
-                            qry.getSql(),
-                            qry.getArgs(),
-                            cctx);
-
-                        if (cctx.config().getQueryParallelism() > 1) {
-                            qry.setDistributedJoins(true);
-
-                            return idx.queryDistributedSql(schemaName, cctx.name(), qry, keepBinary);
-                        }
-                        else
-                            return idx.queryLocalSql(schemaName, cctx.name(), qry, idx.backupFilter(requestTopVer.get(),
-                                qry.getPartitions()), keepBinary);
-                    }
-                }, true);
-        }
-        catch (IgniteCheckedException e) {
-            throw new CacheException(e);
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
+        return new QueryCursorImpl<Cache.Entry<K, V>>(converted) {
+            @Override public void close() {
+                converted.cursor().close();
+            }
+        };
     }
 
     /**
@@ -2667,13 +2643,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param cacheName Cache name.
      * @param typeName Type name.
      * @return Type descriptor.
-     * @throws IgniteCheckedException If failed.
      */
-    private String typeName(@Nullable String cacheName, String typeName) throws IgniteCheckedException {
+    private String typeName(@Nullable String cacheName, String typeName) throws IgniteException {
         QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(cacheName, typeName));
 
         if (type == null)
-            throw new IgniteCheckedException("Failed to find SQL table for type: " + typeName);
+            throw new IgniteException("Failed to find SQL table for type: " + typeName);
 
         return type.name();
     }
@@ -3144,4 +3119,18 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return S.toString(TableCacheFilter.class, this);
         }
     }
+
+    /**
+     * Function which can throw exception.
+     */
+    @FunctionalInterface
+    private interface SupplierX<T> {
+        /**
+         * Get value.
+         *
+         * @return Value.
+         * @throws IgniteCheckedException If failed.
+         */
+        T get() throws IgniteCheckedException;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryKeyValueIterable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryKeyValueIterable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryKeyValueIterable.java
new file mode 100644
index 0000000..41d5145
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryKeyValueIterable.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query;
+
+import org.apache.ignite.cache.query.QueryCursor;
+
+import javax.cache.Cache;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * SqlQuery key-value iterable.
+ */
+public class QueryKeyValueIterable<K, V> implements Iterable<Cache.Entry<K, V>> {
+    /** Underlying fields query cursor. */
+    private final QueryCursor<List<?>> cur;
+
+    /**
+     * Constructor.
+     *
+     * @param cur Underlying fields query cursor.
+     */
+    public QueryKeyValueIterable(QueryCursor<List<?>> cur) {
+        this.cur = cur;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Iterator<Cache.Entry<K, V>> iterator() {
+        return new QueryKeyValueIterator<>(cur.iterator());
+    }
+
+    /**
+     * @return Underlying fields query cursor.
+     */
+    QueryCursor<List<?>> cursor() {
+        return cur;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryKeyValueIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryKeyValueIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryKeyValueIterator.java
new file mode 100644
index 0000000..02dde9d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryKeyValueIterator.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.query;
+
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * SqlQuery key-value iterator.
+ */
+public class QueryKeyValueIterator<K, V> implements Iterator<Cache.Entry<K, V>> {
+    /** Target iterator. */
+    private final Iterator<List<?>> iter;
+
+    /**
+     * Constructor.
+     *
+     * @param iter Target iterator.
+     */
+    public QueryKeyValueIterator(Iterator<List<?>> iter) {
+        this.iter = iter;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() {
+        return iter.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public Cache.Entry<K, V> next() {
+        try {
+            List<?> row = iter.next();
+
+            return new CacheEntryImpl<>((K)row.get(0), (V)row.get(1));
+        }
+        catch (CacheException e) {
+            throw e;
+        }
+        catch (Exception e) {
+            throw new CacheException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove() {
+        throw new UnsupportedOperationException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
index 69b1655..ba2fec6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
@@ -26,7 +26,6 @@ import java.util.List;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.Callable;
-import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -34,7 +33,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
-import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -231,19 +229,18 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
-        @Override public void start(GridKernalContext ctx, GridSpinBusyLock busyLock) throws IgniteCheckedException {
-            // No-op
+        @Override public SqlFieldsQuery generateFieldsQuery(String cacheName, SqlQuery qry) {
+            return null;
         }
 
         /** {@inheritDoc} */
-        @Override public void stop() throws IgniteCheckedException {
+        @Override public void start(GridKernalContext ctx, GridSpinBusyLock busyLock) throws IgniteCheckedException {
             // No-op
         }
 
         /** {@inheritDoc} */
-        @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(String schemaName, String cacheName,
-            SqlQuery qry, boolean keepBinary) throws IgniteCheckedException {
-            return null;
+        @Override public void stop() throws IgniteCheckedException {
+            // No-op
         }
 
         /** {@inheritDoc} */
@@ -265,12 +262,6 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
-        @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryLocalSql(String schemaName, String cacheName,
-            SqlQuery qry, IndexingQueryFilter filter, boolean keepBinary) throws IgniteCheckedException {
-            return null;
-        }
-
-        /** {@inheritDoc} */
         @Override public FieldsQueryCursor<List<?>> queryLocalSqlFields(String schemaName, SqlFieldsQuery qry,
             boolean keepBinary, IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException {
             return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index e2dba50..58e09cb 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -44,7 +44,6 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
-import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
@@ -53,7 +52,6 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.QueryCancelledException;
-import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlFunction;
@@ -62,8 +60,6 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridTopic;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
-import org.apache.ignite.internal.processors.cache.CacheObjectUtils;
 import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -209,7 +205,6 @@ import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.checkAc
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccEnabled;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.tx;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.txStart;
-import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
 import static org.apache.ignite.internal.processors.query.QueryUtils.KEY_FIELD_NAME;
@@ -1568,111 +1563,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         return cursor;
     }
 
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public <K, V> QueryCursor<Cache.Entry<K,V>> queryLocalSql(String schemaName, String cacheName,
-        final SqlQuery qry, final IndexingQueryFilter filter, final boolean keepBinary) throws IgniteCheckedException {
-        String type = qry.getType();
-        String sqlQry = qry.getSql();
-        String alias = qry.getAlias();
-        Object[] params = qry.getArgs();
-
-        GridQueryCancel cancel = new GridQueryCancel();
-
-        final GridCloseableIterator<IgniteBiTuple<K, V>> i = queryLocalSql(schemaName, cacheName, sqlQry, alias,
-            F.asList(params), type, filter, cancel);
-
-        return new QueryCursorImpl<>(new Iterable<Cache.Entry<K, V>>() {
-            @SuppressWarnings("NullableProblems")
-            @Override public Iterator<Cache.Entry<K, V>> iterator() {
-                return new ClIter<Cache.Entry<K, V>>() {
-                    @Override public void close() throws Exception {
-                        i.close();
-                    }
-
-                    @Override public boolean hasNext() {
-                        return i.hasNext();
-                    }
-
-                    @Override public Cache.Entry<K, V> next() {
-                        IgniteBiTuple<K, V> t = i.next();
-
-                        K key = (K)CacheObjectUtils.unwrapBinaryIfNeeded(objectContext(), t.get1(), keepBinary, false);
-                        V val = (V)CacheObjectUtils.unwrapBinaryIfNeeded(objectContext(), t.get2(), keepBinary, false);
-
-                        return new CacheEntryImpl<>(key, val);
-                    }
-
-                    @Override public void remove() {
-                        throw new UnsupportedOperationException();
-                    }
-                };
-            }
-        }, cancel);
-    }
-
-    /**
-     * Executes regular query.
-     *
-     * @param schemaName Schema name.
-     * @param cacheName Cache name.
-     * @param qry Query.
-     * @param alias Table alias.
-     * @param params Query parameters.
-     * @param type Query return type.
-     * @param filter Cache name and key filter.
-     * @param cancel Cancel object.
-     * @return Queried rows.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("unchecked")
-    <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalSql(String schemaName, String cacheName,
-        final String qry, String alias, @Nullable final Collection<Object> params, String type,
-        final IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException {
-        final H2TableDescriptor tbl = tableDescriptor(schemaName, cacheName, type);
-
-        if (tbl == null)
-            throw new IgniteSQLException("Failed to find SQL table for type: " + type,
-                IgniteQueryErrorCode.TABLE_NOT_FOUND);
-
-        String sql = generateQuery(qry, alias, tbl);
-
-        Connection conn = connectionForThread(tbl.schemaName());
-
-        H2Utils.setupConnection(conn, false, false);
-
-        GridH2QueryContext qctx = new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter)
-            .distributedJoinMode(OFF);
-
-        PreparedStatement stmt = preparedStatementWithParams(conn, sql, params, true);
-
-        MvccQueryTracker mvccTracker = mvccTracker(stmt, false);
-
-        if (mvccTracker != null)
-            qctx.mvccSnapshot(mvccTracker.snapshot());
-
-        GridH2QueryContext.set(qctx);
-
-        GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL, schemaName,
-            U.currentTimeMillis(), null, true);
-
-        runs.put(run.id(), run);
-
-        try {
-            ResultSet rs = executeSqlQueryWithTimer(stmt, conn, sql, params, 0, cancel);
-
-            return new H2KeyValueIterator(rs);
-        }
-        finally {
-            GridH2QueryContext.clearThreadLocal();
-
-            if (mvccTracker != null)
-                mvccTracker.onDone();
-
-            runs.remove(run.id());
-        }
-    }
-
     /**
      * Initialises MVCC filter and returns MVCC query tracker if needed.
      * @param stmt Prepared statement.
@@ -1754,8 +1644,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             PreparedStatementEx stmtEx = stmt.unwrap(PreparedStatementEx.class);
 
             if (mvccEnabled) {
-                assert mvccCacheId != null;
-
                 stmtEx.putMeta(MVCC_CACHE_ID, mvccCacheId);
                 stmtEx.putMeta(MVCC_STATE, Boolean.TRUE);
             }
@@ -1840,9 +1728,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(String schemaName, String cacheName,
-        SqlQuery qry, boolean keepBinary) {
+    @Override public SqlFieldsQuery generateFieldsQuery(String cacheName, SqlQuery qry) {
+        String schemaName = schema(cacheName);
+
         String type = qry.getType();
 
         H2TableDescriptor tblDesc = tableDescriptor(schemaName, cacheName, type);
@@ -1860,58 +1748,80 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             throw new IgniteException(e);
         }
 
-        SqlFieldsQuery fqry = new SqlFieldsQuery(sql);
+        SqlFieldsQuery res = new SqlFieldsQuery(sql);
 
-        fqry.setArgs(qry.getArgs());
-        fqry.setPageSize(qry.getPageSize());
-        fqry.setDistributedJoins(qry.isDistributedJoins());
-        fqry.setPartitions(qry.getPartitions());
-        fqry.setLocal(qry.isLocal());
+        res.setArgs(qry.getArgs());
+        res.setDistributedJoins(qry.isDistributedJoins());
+        res.setLocal(qry.isLocal());
+        res.setPageSize(qry.getPageSize());
+        res.setPartitions(qry.getPartitions());
+        res.setReplicatedOnly(qry.isReplicatedOnly());
+        res.setSchema(schemaName);
+        res.setSql(sql);
 
         if (qry.getTimeout() > 0)
-            fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS);
+            res.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS);
+
+        return res;
+    }
 
-        final QueryCursor<List<?>> res =
-            querySqlFields(schemaName, fqry, null, keepBinary, true, null, null).get(0);
+    /**
+     * Prepares statement for query.
+     *
+     * @param qry Query string.
+     * @param tableAlias table alias.
+     * @param tbl Table to use.
+     * @return Prepared statement.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private static String generateQuery(String qry, String tableAlias, H2TableDescriptor tbl)
+        throws IgniteCheckedException {
+        assert tbl != null;
 
-        final Iterable<Cache.Entry<K, V>> converted = new Iterable<Cache.Entry<K, V>>() {
-            @Override public Iterator<Cache.Entry<K, V>> iterator() {
-                final Iterator<List<?>> iter0 = res.iterator();
+        final String qry0 = qry;
 
-                return new Iterator<Cache.Entry<K, V>>() {
-                    @Override public boolean hasNext() {
-                        return iter0.hasNext();
-                    }
+        String t = tbl.fullTableName();
 
-                    @Override public Cache.Entry<K, V> next() {
-                        List<?> l;
+        String from = " ";
 
-                        try {
-                            l = iter0.next();
-                        }
-                        catch (CacheException e) {
-                            throw e;
-                        }
-                        catch (Exception e) {
-                            throw new CacheException(e);
-                        }
+        qry = qry.trim();
 
-                        return new CacheEntryImpl<>((K)l.get(0), (V)l.get(1));
-                    }
+        String upper = qry.toUpperCase();
 
-                    @Override public void remove() {
-                        throw new UnsupportedOperationException();
-                    }
-                };
-            }
-        };
+        if (upper.startsWith("SELECT")) {
+            qry = qry.substring(6).trim();
 
-        // No metadata for SQL queries.
-        return new QueryCursorImpl<Cache.Entry<K, V>>(converted) {
-            @Override public void close() {
-                res.close();
+            final int star = qry.indexOf('*');
+
+            if (star == 0)
+                qry = qry.substring(1).trim();
+            else if (star > 0) {
+                if (F.eq('.', qry.charAt(star - 1))) {
+                    t = qry.substring(0, star - 1);
+
+                    qry = qry.substring(star + 1).trim();
+                }
+                else
+                    throw new IgniteCheckedException("Invalid query (missing alias before asterisk): " + qry0);
             }
-        };
+            else
+                throw new IgniteCheckedException("Only queries starting with 'SELECT *' and 'SELECT alias.*' " +
+                    "are supported (rewrite your query or use SqlFieldsQuery instead): " + qry0);
+
+            upper = qry.toUpperCase();
+        }
+
+        if (!upper.startsWith("FROM"))
+            from = " FROM " + t + (tableAlias != null ? " as " + tableAlias : "") +
+                (upper.startsWith("WHERE") || upper.startsWith("ORDER") || upper.startsWith("LIMIT") ?
+                    " " : " WHERE ");
+
+        if(tableAlias != null)
+            t = tableAlias;
+
+        qry = "SELECT " + t + "." + KEY_FIELD_NAME + ", " + t + "." + VAL_FIELD_NAME + from + qry;
+
+        return qry;
     }
 
     /**
@@ -2774,64 +2684,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Prepares statement for query.
-     *
-     * @param qry Query string.
-     * @param tableAlias table alias.
-     * @param tbl Table to use.
-     * @return Prepared statement.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private String generateQuery(String qry, String tableAlias, H2TableDescriptor tbl) throws IgniteCheckedException {
-        assert tbl != null;
-
-        final String qry0 = qry;
-
-        String t = tbl.fullTableName();
-
-        String from = " ";
-
-        qry = qry.trim();
-
-        String upper = qry.toUpperCase();
-
-        if (upper.startsWith("SELECT")) {
-            qry = qry.substring(6).trim();
-
-            final int star = qry.indexOf('*');
-
-            if (star == 0)
-                qry = qry.substring(1).trim();
-            else if (star > 0) {
-                if (F.eq('.', qry.charAt(star - 1))) {
-                    t = qry.substring(0, star - 1);
-
-                    qry = qry.substring(star + 1).trim();
-                }
-                else
-                    throw new IgniteCheckedException("Invalid query (missing alias before asterisk): " + qry0);
-            }
-            else
-                throw new IgniteCheckedException("Only queries starting with 'SELECT *' and 'SELECT alias.*' " +
-                    "are supported (rewrite your query or use SqlFieldsQuery instead): " + qry0);
-
-            upper = qry.toUpperCase();
-        }
-
-        if (!upper.startsWith("FROM"))
-            from = " FROM " + t + (tableAlias != null ? " as " + tableAlias : "") +
-                (upper.startsWith("WHERE") || upper.startsWith("ORDER") || upper.startsWith("LIMIT") ?
-                    " " : " WHERE ");
-
-        if(tableAlias != null)
-            t = tableAlias;
-
-        qry = "SELECT " + t + "." + KEY_FIELD_NAME + ", " + t + "." + VAL_FIELD_NAME + from + qry;
-
-        return qry;
-    }
-
-    /**
      * Registers new class description.
      *
      * This implementation doesn't support type reregistration.
@@ -3842,11 +3694,4 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         return false;
     }
-
-    /**
-     * Closeable iterator.
-     */
-    private interface ClIter<X> extends AutoCloseable, Iterator<X> {
-        // No-op.
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryDetailMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryDetailMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryDetailMetricsSelfTest.java
index 837de65..e1d7154 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryDetailMetricsSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryDetailMetricsSelfTest.java
@@ -186,7 +186,7 @@ public abstract class CacheAbstractQueryDetailMetricsSelfTest extends GridCommon
 
         assertTrue(lastMetrics.contains("SQL_FIELDS select * from String limit 2;"));
         assertTrue(lastMetrics.contains("SCAN A;"));
-        assertTrue(lastMetrics.contains("SQL from String;"));
+        assertTrue(lastMetrics.contains("SELECT \"A\".\"STRING\"._KEY, \"A\".\"STRING\"._VAL from String;"));
 
         cache = grid(0).context().cache().jcache("B");
 
@@ -349,19 +349,6 @@ public abstract class CacheAbstractQueryDetailMetricsSelfTest extends GridCommon
     }
 
     /**
-     * Test metrics for failed Scan queries.
-     *
-     * @throws Exception In case of error.
-     */
-    public void testSqlQueryFailedMetrics() throws Exception {
-        IgniteCache<Integer, String> cache = grid(0).context().cache().jcache("A");
-
-        SqlQuery<Integer, String> qry = new SqlQuery<>("Long", "from Long");
-
-        checkQueryFailedMetrics(cache, qry);
-    }
-
-    /**
      * Test metrics for Sql queries.
      *
      * @throws Exception In case of error.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
index eb3c8d6..e1c796b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
@@ -209,19 +209,6 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
     }
 
     /**
-     * Test metrics for failed Scan queries.
-     *
-     * @throws Exception In case of error.
-     */
-    public void testSqlQueryFailedMetrics() throws Exception {
-        IgniteCache<Integer, String> cache = grid(0).context().cache().jcache("A");
-
-        SqlQuery qry = new SqlQuery<>("Long", "from Long");
-
-        checkQueryFailedMetrics(cache, qry);
-    }
-
-    /**
      * Test metrics for Sql queries.
      *
      * @throws Exception In case of error.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunTest.java
index 89ef607..d0cfbd9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunTest.java
@@ -291,6 +291,8 @@ public class IgniteCacheLockPartitionOnAffinityRunTest extends IgniteCacheLockPa
      * @throws Exception If failed.
      */
     public void testSingleCache() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7692");
+
         final PersonsCountGetter personsCntGetter = new PersonsCountGetter() {
             @Override public int getPersonsCount(IgniteEx ignite, IgniteLogger log, int orgId) throws Exception {
                 return getPersonsCountSingleCache(ignite, log, orgId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 6b76230..7e198f8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -17,20 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.QueryIndexType;
@@ -38,48 +25,33 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.internal.binary.BinaryObjectImpl;
-import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.CacheObjectContext;
-import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
-import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryProperty;
-import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.plugin.extensions.communication.MessageReader;
-import org.apache.ignite.plugin.extensions.communication.MessageWriter;
-import org.apache.ignite.spi.IgniteSpiCloseableIterator;
-import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.testframework.GridStringLogger;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.h2.util.JdbcUtils;
-import org.jetbrains.annotations.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
 
 /**
  * Tests for all SQL based indexing SPI implementations.
  */
 public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstractTest {
     /** */
-    private static final TextIndex textIdx = new TextIndex(F.asList("txt"));
-
-    /** */
     private static final LinkedHashMap<String, String> fieldsAA = new LinkedHashMap<>();
 
     /** */
     private static final LinkedHashMap<String, String> fieldsAB = new LinkedHashMap<>();
 
     /** */
-    private static final LinkedHashMap<String, String> fieldsBA = new LinkedHashMap<>();
-
-    /** */
     private IgniteEx ignite0;
 
     /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
@@ -98,20 +70,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
 
         fieldsAB.putAll(fieldsAA);
         fieldsAB.put("txt", String.class.getName());
-
-        fieldsBA.putAll(fieldsAA);
-        fieldsBA.put("sex", Boolean.class.getName());
     }
 
-    /** */
-    private static TypeDesc typeAA = new TypeDesc("A", "A", "A", Collections.<String, Class<?>>emptyMap(), null);
-
-    /** */
-    private static TypeDesc typeAB = new TypeDesc("A", "A", "B", Collections.<String, Class<?>>emptyMap(), textIdx);
-
-    /** */
-    private static TypeDesc typeBA = new TypeDesc("B", "B", "A", Collections.<String, Class<?>>emptyMap(), null);
-
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         ignite0 = startGrid(0);
@@ -144,82 +104,12 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         return cfg;
     }
 
-    /**
-     *
-     */
-    private CacheConfiguration cacheBCfg() {
-        CacheConfiguration cfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
-
-        cfg.setName("B");
-
-        QueryEntity eA = new QueryEntity(Integer.class.getName(), "A");
-        eA.setFields(fieldsBA);
-
-        cfg.setQueryEntities(Collections.singleton(eA));
-
-        return cfg;
-    }
-
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
     }
 
     /**
-     * @param id Id.
-     * @param name Name.
-     * @param age Age.
-     * @return AA.
-     */
-    private BinaryObjectBuilder aa(String typeName, long id, String name, int age) {
-        BinaryObjectBuilder aBuilder = ignite0.binary().builder(typeName)
-                .setField("id", id)
-                .setField("name", name)
-                .setField("age", age);
-
-        return aBuilder;
-    }
-
-    /**
-     * @param id Id.
-     * @param name Name.
-     * @param age Age.
-     * @param txt Text.
-     * @return AB.
-     */
-    private BinaryObjectBuilder ab(long id, String name, int age, String txt) {
-        BinaryObjectBuilder aBuilder = aa("B", id, name, age);
-
-        aBuilder.setField("txt", txt);
-
-        return aBuilder;
-    }
-
-    /**
-     * @param id Id.
-     * @param name Name.
-     * @param age Age.
-     * @param sex Sex.
-     * @return BA.
-     */
-    private BinaryObjectBuilder ba(long id, String name, int age, boolean sex) {
-        BinaryObjectBuilder builder = aa("A", id, name, age);
-
-        builder.setField("sex", sex);
-
-        return builder;
-    }
-
-    /**
-     * @param row Row
-     * @return Value.
-     * @throws IgniteSpiException If failed.
-     */
-    private BinaryObjectImpl value(IgniteBiTuple<Integer, BinaryObjectImpl> row) throws IgniteSpiException {
-        return row.get2();
-    }
-
-    /**
      * @return Indexing.
      */
     private IgniteH2Indexing getIndexing() {
@@ -234,147 +124,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
     }
 
     /**
-     * @param key Key.
-     * @return Cache object.
-     */
-    private KeyCacheObject key(int key) {
-        return new TestCacheObject(key);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSpi() throws Exception {
-        IgniteH2Indexing spi = getIndexing();
-
-        IgniteCache<Integer, BinaryObject> cacheA = ignite0.createCache(cacheACfg());
-
-        IgniteCache<Integer, BinaryObject> cacheB = ignite0.createCache(cacheBCfg());
-
-        assertFalse(spi.queryLocalSql(spi.schema(typeAA.cacheName()), typeAA.cacheName(), "select * from A.A", null,
-            Collections.emptySet(), typeAA.name(), null, null).hasNext());
-
-        assertFalse(spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(), "select * from A.B", null,
-            Collections.emptySet(), typeAB.name(), null, null).hasNext());
-
-        assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(), "select * from B.A", null,
-            Collections.emptySet(), typeBA.name(), null, null).hasNext());
-
-        assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(),
-            "select * from B.A, A.B, A.A", null, Collections.emptySet(), typeBA.name(), null, null).hasNext());
-
-        try {
-            spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(),
-                "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba",
-                null, Collections.emptySet(), typeBA.name(), null, null).hasNext();
-
-            fail("Enumerations of aliases in select block must be prohibited");
-        }
-        catch (IgniteCheckedException ignored) {
-            // all fine
-        }
-
-        assertFalse(spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(), "select ab.* from A.B ab",
-            null, Collections.emptySet(), typeAB.name(), null, null).hasNext());
-
-        assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(),
-            "select   ba.*   from B.A  as ba", null, Collections.emptySet(), typeBA.name(), null, null).hasNext());
-
-        cacheA.put(1, aa("A", 1, "Vasya", 10).build());
-        cacheA.put(1, ab(1, "Vasya", 20, "Some text about Vasya goes here.").build());
-        cacheB.put(1, ba(2, "Petya", 25, true).build());
-        cacheB.put(1, ba(2, "Kolya", 25, true).build());
-        cacheA.put(2, aa("A", 2, "Valera", 19).build());
-        cacheA.put(3, aa("A", 3, "Borya", 18).build());
-        cacheA.put(4, ab(4, "Vitalya", 20, "Very Good guy").build());
-
-        // Query data.
-        Iterator<IgniteBiTuple<Integer, BinaryObjectImpl>> res = spi.queryLocalSql(spi.schema(typeAA.cacheName()),
-            typeAA.cacheName(), "from a order by age", null, Collections.emptySet(), typeAA.name(), null, null);
-
-        assertTrue(res.hasNext());
-        assertEquals(aa("A", 3, "Borya", 18).build(), value(res.next()));
-        assertTrue(res.hasNext());
-        assertEquals(aa("A", 2, "Valera", 19).build(), value(res.next()));
-        assertFalse(res.hasNext());
-
-        res = spi.queryLocalSql(spi.schema(typeAA.cacheName()), typeAA.cacheName(),
-            "select aa.* from a aa order by aa.age", null, Collections.emptySet(), typeAA.name(), null, null);
-
-        assertTrue(res.hasNext());
-        assertEquals(aa("A", 3, "Borya", 18).build(), value(res.next()));
-        assertTrue(res.hasNext());
-        assertEquals(aa("A", 2, "Valera", 19).build(), value(res.next()));
-        assertFalse(res.hasNext());
-
-        res = spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(), "from b order by name", null,
-            Collections.emptySet(), typeAB.name(), null, null);
-
-        assertTrue(res.hasNext());
-        assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").build(), value(res.next()));
-        assertTrue(res.hasNext());
-        assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(res.next()));
-        assertFalse(res.hasNext());
-
-        res = spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(),
-            "select bb.* from b as bb order by bb.name", null, Collections.emptySet(), typeAB.name(), null, null);
-
-        assertTrue(res.hasNext());
-        assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").build(), value(res.next()));
-        assertTrue(res.hasNext());
-        assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(res.next()));
-        assertFalse(res.hasNext());
-
-        res = spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(), "from a", null,
-            Collections.emptySet(), typeBA.name(), null, null);
-
-        assertTrue(res.hasNext());
-        assertEquals(ba(2, "Kolya", 25, true).build(), value(res.next()));
-        assertFalse(res.hasNext());
-
-        // Text queries
-        Iterator<IgniteBiTuple<Integer, BinaryObjectImpl>> txtRes = spi.queryLocalText(spi.schema(typeAB.cacheName()),
-            typeAB.cacheName(), "good", typeAB.name(), null);
-
-        assertTrue(txtRes.hasNext());
-        assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(txtRes.next()));
-        assertFalse(txtRes.hasNext());
-
-        // Fields query
-        GridQueryFieldsResult fieldsRes =
-            spi.queryLocalSqlFields(spi.schema("A"), "select a.a.name n1, a.a.age a1, b.a.name n2, " +
-            "b.a.age a2 from a.a, b.a where a.a.id = b.a.id ", Collections.emptySet(), null, false, false, 0, null);
-
-        String[] aliases = {"N1", "A1", "N2", "A2"};
-        Object[] vals = { "Valera", 19, "Kolya", 25};
-
-        IgniteSpiCloseableIterator<List<?>> it = fieldsRes.iterator();
-
-        assertTrue(it.hasNext());
-
-        List<?> fields = it.next();
-
-        assertEquals(4, fields.size());
-
-        int i = 0;
-
-        for (Object f : fields) {
-            assertEquals(aliases[i], fieldsRes.metaData().get(i).fieldName());
-            assertEquals(vals[i++], f);
-        }
-
-        assertFalse(it.hasNext());
-
-        // Remove
-        cacheA.remove(2);
-        cacheB.remove(1);
-    }
-
-    /**
      * Test long queries write explain warnings into log.
      *
      * @throws Exception If failed.
      */
+    @SuppressWarnings({"unchecked", "deprecation"})
     public void testLongQueries() throws Exception {
         IgniteH2Indexing spi = getIndexing();
 
@@ -416,381 +170,4 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
             GridTestUtils.setFieldValue(spi, "log", oldLog);
         }
     }
-
-    /**
-     * Index descriptor.
-     */
-    private static class TextIndex implements GridQueryIndexDescriptor {
-        /** */
-        private final Collection<String> fields;
-
-        /**
-         * @param fields Fields.
-         */
-        private TextIndex(Collection<String> fields) {
-            this.fields = Collections.unmodifiableCollection(fields);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String name() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<String> fields() {
-            return fields;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean descending(String field) {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public QueryIndexType type() {
-            return QueryIndexType.FULLTEXT;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int inlineSize() {
-            return 0;
-        }
-    }
-
-    /**
-     * Type descriptor.
-     */
-    private static class TypeDesc implements GridQueryTypeDescriptor {
-        /** */
-        private final String name;
-
-        /** */
-        private final String cacheName;
-
-        /** */
-        private final String schemaName;
-
-        /** */
-        private final Map<String, Class<?>> valFields;
-
-        /** */
-        private final GridQueryIndexDescriptor textIdx;
-
-        /**
-         * @param cacheName Cache name.
-         * @param schemaName Schema name.
-         * @param name Type name.
-         * @param valFields Fields.
-         * @param textIdx Fulltext index.
-         */
-        private TypeDesc(String cacheName, String schemaName, String name, Map<String, Class<?>> valFields, GridQueryIndexDescriptor textIdx) {
-            this.name = name;
-            this.cacheName = cacheName;
-            this.schemaName = schemaName;
-            this.valFields = Collections.unmodifiableMap(valFields);
-            this.textIdx = textIdx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String affinityKey() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String name() {
-            return name;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String schemaName() {
-            return schemaName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String tableName() {
-            return null;
-        }
-
-        /**
-         * @return Cache name.
-         */
-        String cacheName() {
-            return cacheName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Map<String, Class<?>> fields() {
-            return valFields;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridQueryProperty property(final String name) {
-            return new GridQueryProperty() {
-                /** */
-                @Override public Object value(Object key, Object val) throws IgniteCheckedException {
-                    return TypeDesc.this.value(name, key, val);
-                }
-
-                /** */
-                @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
-                    throw new UnsupportedOperationException();
-                }
-
-                /** */
-                @Override public String name() {
-                    return name;
-                }
-
-                /** */
-                @Override public Class<?> type() {
-                    return Object.class;
-                }
-
-                /** */
-                @Override public boolean key() {
-                    return false;
-                }
-
-                /** */
-                @Override public GridQueryProperty parent() {
-                    return null;
-                }
-
-                /** */
-                @Override public boolean notNull() {
-                    return false;
-                }
-
-                /** */
-                @Override public Object defaultValue() {
-                    return null;
-                }
-
-                /** */
-                @Override public int precision() {
-                    return -1;
-                }
-
-                /** */
-                @Override public int scale() {
-                    return -1;
-                }
-            };
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override public <T> T value(String field, Object key, Object val) throws IgniteSpiException {
-            assert !F.isEmpty(field);
-
-            assert key instanceof Integer;
-
-            Map<String, T> m = (Map<String, T>)val;
-
-            if (m.containsKey(field))
-                return m.get(field);
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override public void setValue(String field, Object key, Object val, Object propVal) throws IgniteCheckedException {
-            assert !F.isEmpty(field);
-
-            assert key instanceof Integer;
-
-            Map<String, Object> m = (Map<String, Object>)val;
-
-            m.put(field, propVal);
-        }
-
-        /** */
-        @Override public Map<String, GridQueryIndexDescriptor> indexes() {
-            return Collections.emptyMap();
-        }
-
-        /** */
-        @Override public GridQueryIndexDescriptor textIndex() {
-            return textIdx;
-        }
-
-        /** */
-        @Override public Class<?> valueClass() {
-            return Object.class;
-        }
-
-        /** */
-        @Override public Class<?> keyClass() {
-            return Integer.class;
-        }
-
-        /** */
-        @Override public String keyTypeName() {
-            return null;
-        }
-
-        /** */
-        @Override public String valueTypeName() {
-            return null;
-        }
-
-        /** */
-        @Override public boolean valueTextIndex() {
-            return textIdx == null;
-        }
-
-        /** */
-        @Override public int typeId() {
-            return 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String keyFieldName() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String valueFieldName() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public String keyFieldAlias() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public String valueFieldAlias() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void validateKeyAndValue(Object key, Object value) throws IgniteCheckedException {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setDefaults(Object key, Object val) throws IgniteCheckedException {
-            // No-op.
-        }
-    }
-
-    /**
-     */
-    private static class TestCacheObject implements KeyCacheObject {
-        /** */
-        private Object val;
-
-        /** */
-        private int part;
-
-        /**
-         * @param val Value.
-         */
-        private TestCacheObject(Object val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onAckReceived() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public <T> T value(CacheObjectValueContext ctx, boolean cpy) {
-            return (T)val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int partition() {
-            return part;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void partition(int part) {
-            this.part = part;
-        }
-
-        /** {@inheritDoc} */
-        @Override public byte[] valueBytes(CacheObjectValueContext ctx) throws IgniteCheckedException {
-            return JdbcUtils.serialize(val, null);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean putValue(ByteBuffer buf) throws IgniteCheckedException {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int putValue(long addr) throws IgniteCheckedException {
-            return 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
-            throws IgniteCheckedException {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int valueBytesLength(CacheObjectContext ctx) throws IgniteCheckedException {
-            return 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public byte cacheObjectType() {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean isPlatformType() {
-            return true;
-        }
-
-        /** {@inheritDoc} */
-        @Override public KeyCacheObject copy(int part) {
-            return this;
-        }
-
-        /** {@inheritDoc} */
-        @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void finishUnmarshal(CacheObjectValueContext ctx, ClassLoader ldr) throws IgniteCheckedException {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void prepareMarshal(CacheObjectValueContext ctx) throws IgniteCheckedException {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public short directType() {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public byte fieldsCount() {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean internal() {
-            return false;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ec543a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIteratorNullifyOnEndSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIteratorNullifyOnEndSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIteratorNullifyOnEndSelfTest.java
index 31b0b97..66f9dd2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIteratorNullifyOnEndSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIteratorNullifyOnEndSelfTest.java
@@ -17,22 +17,16 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
-import javax.cache.Cache;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
-import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
-import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -47,80 +41,9 @@ public class H2ResultSetIteratorNullifyOnEndSelfTest extends GridCommonAbstractT
     private static final int PERSON_COUNT = 20;
 
     /** */
-    private static final String SELECT_ALL_SQL = "SELECT p.* FROM Person p ORDER BY p.salary";
-
-    /** */
     private static final String SELECT_MAX_SAL_SQLF = "select max(salary) from Person";
 
     /**
-     * Non local SQL check nullification after close
-     */
-    public void testSqlQueryClose() {
-        SqlQuery<String, Person> qry = new SqlQuery<>(Person.class, SELECT_ALL_SQL);
-
-        QueryCursor<Cache.Entry<String, Person>> qryCurs = cache().query(qry);
-
-        qryCurs.iterator();
-
-        qryCurs.close();
-
-        H2ResultSetIterator h2It = extractIteratorInnerGridIteratorInnerH2ResultSetIterator(qryCurs);
-
-        checkIterator(h2It);
-    }
-
-    /**
-     * Non local SQL check nullification after complete
-     */
-    public void testSqlQueryComplete() {
-        SqlQuery<String, Person> qry = new SqlQuery<>(Person.class, SELECT_ALL_SQL);
-
-        QueryCursor<Cache.Entry<String, Person>> qryCurs = cache().query(qry);
-
-        qryCurs.getAll();
-
-        H2ResultSetIterator h2It = extractIteratorInnerGridIteratorInnerH2ResultSetIterator(qryCurs);
-
-        checkIterator(h2It);
-    }
-
-    /**
-     * Local SQL check nullification after close
-     */
-    public void testSqlQueryLocalClose() {
-        SqlQuery<String, Person> qry = new SqlQuery<>(Person.class, SELECT_ALL_SQL);
-
-        qry.setLocal(true);
-
-        QueryCursor<Cache.Entry<String, Person>> qryCurs = cache().query(qry);
-
-        qryCurs.iterator();
-
-        qryCurs.close();
-
-        H2ResultSetIterator h2It = extractIterableInnerH2ResultSetIterator(qryCurs);
-
-        checkIterator(h2It);
-    }
-
-    /**
-     * Local SQL check nullification after complete
-     */
-    public void testSqlQueryLocalComplete() {
-        SqlQuery<String, Person> qry = new SqlQuery<>(Person.class, SELECT_ALL_SQL);
-
-        qry.setLocal(true);
-
-        QueryCursor<Cache.Entry<String, Person>> qryCurs = cache().query(qry);
-
-        qryCurs.getAll();
-
-        H2ResultSetIterator h2It = extractIterableInnerH2ResultSetIterator(qryCurs);
-
-        checkIterator(h2It);
-    }
-
-    /**
      * Non local SQL Fields check nullification after close
      */
     public void testSqlFieldsQueryClose() {
@@ -200,45 +123,6 @@ public class H2ResultSetIteratorNullifyOnEndSelfTest extends GridCommonAbstractT
     }
 
     /**
-     * Extract H2ResultSetIterator by reflection for non local SQL cases
-     * @param qryCurs source cursor
-     * @return target iterator or null of not extracted
-     */
-    private H2ResultSetIterator extractIteratorInnerGridIteratorInnerH2ResultSetIterator(
-        QueryCursor<Cache.Entry<String, Person>> qryCurs) {
-        if (QueryCursorImpl.class.isAssignableFrom(qryCurs.getClass())) {
-            Iterator inner = GridTestUtils.getFieldValue(qryCurs, QueryCursorImpl.class, "iter");
-
-            GridQueryCacheObjectsIterator it = GridTestUtils.getFieldValue(inner, inner.getClass(), "val$iter0");
-
-            Iterator<List<?>> h2RsIt = GridTestUtils.getFieldValue(it, GridQueryCacheObjectsIterator.class, "iter");
-
-            if (H2ResultSetIterator.class.isAssignableFrom(h2RsIt.getClass()))
-                return (H2ResultSetIterator)h2RsIt;
-        }
-        return null;
-    }
-
-    /**
-     * Extract H2ResultSetIterator by reflection for local SQL cases.
-     *
-     * @param qryCurs source cursor
-     * @return target iterator or null of not extracted
-     */
-    private H2ResultSetIterator extractIterableInnerH2ResultSetIterator(
-        QueryCursor<Cache.Entry<String, Person>> qryCurs) {
-        if (QueryCursorImpl.class.isAssignableFrom(qryCurs.getClass())) {
-            Iterable iterable = GridTestUtils.getFieldValue(qryCurs, QueryCursorImpl.class, "iterExec");
-
-            Iterator h2RsIt = GridTestUtils.getFieldValue(iterable, iterable.getClass(), "val$i");
-
-            if (H2ResultSetIterator.class.isAssignableFrom(h2RsIt.getClass()))
-                return (H2ResultSetIterator)h2RsIt;
-        }
-        return null;
-    }
-
-    /**
      * Extract H2ResultSetIterator by reflection for SQL Fields cases.
      *
      * @param qryCurs source cursor
@@ -256,67 +140,6 @@ public class H2ResultSetIteratorNullifyOnEndSelfTest extends GridCommonAbstractT
         return null;
     }
 
-    /**
-     * "onClose" should remove links to data.
-     */
-    public void testOnClose() {
-        try {
-            GridCloseableIterator it = indexing().queryLocalSql(
-                indexing().schema(cache().getName()),
-                cache().getName(),
-                SELECT_ALL_SQL,
-                null,
-                Collections.emptySet(),
-                "Person",
-                null,
-                null);
-
-            if (H2ResultSetIterator.class.isAssignableFrom(it.getClass())) {
-                H2ResultSetIterator h2it = (H2ResultSetIterator)it;
-
-                h2it.onClose();
-
-                assertNull(GridTestUtils.getFieldValue(h2it, H2ResultSetIterator.class, "data"));
-            }
-            else
-                fail();
-        }
-        catch (IgniteCheckedException e) {
-            fail(e.getMessage());
-        }
-    }
-
-    /**
-     * Complete iterate should remove links to data.
-     */
-    public void testOnComplete() {
-        try {
-            GridCloseableIterator it = indexing().queryLocalSql(
-                indexing().schema(cache().getName()),
-                cache().getName(),
-                SELECT_ALL_SQL,
-                null,
-                Collections.emptySet(),
-                "Person",
-                null,
-                null);
-
-            if (H2ResultSetIterator.class.isAssignableFrom(it.getClass())) {
-                H2ResultSetIterator h2it = (H2ResultSetIterator)it;
-
-                while (h2it.onHasNext())
-                    h2it.onNext();
-
-                assertNull(GridTestUtils.getFieldValue(h2it, H2ResultSetIterator.class, "data"));
-            }
-            else
-                fail();
-        }
-        catch (IgniteCheckedException e) {
-            fail(e.getMessage());
-        }
-    }
-
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         startGrids(NODES_COUNT);
@@ -336,15 +159,6 @@ public class H2ResultSetIteratorNullifyOnEndSelfTest extends GridCommonAbstractT
     }
 
     /**
-     * @return H2 indexing instance.
-     */
-    private IgniteH2Indexing indexing() {
-        GridQueryProcessor qryProcessor = grid(0).context().query();
-
-        return GridTestUtils.getFieldValue(qryProcessor, GridQueryProcessor.class, "idx");
-    }
-
-    /**
      * @return Cache.
      */
     private IgniteCache<String, Person> cache() {


[11/50] [abbrv] ignite git commit: IGNITE-9996: Performance drop fix base on instanceof NoopEncryptionSpi

Posted by sb...@apache.org.
IGNITE-9996: Performance drop fix base on instanceof NoopEncryptionSpi


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

Branch: refs/heads/ignite-10044
Commit: 7076f42763dcfdb28f626d2f8072e7f34b1ff8d7
Parents: 6923bd8
Author: Nikolay Izhikov <ni...@apache.org>
Authored: Sat Nov 17 11:37:24 2018 +0300
Committer: Nikolay Izhikov <ni...@apache.org>
Committed: Sat Nov 17 11:37:24 2018 +0300

----------------------------------------------------------------------
 .../encryption/GridEncryptionManager.java         |  2 +-
 .../cache/persistence/pagemem/PageMemoryImpl.java | 10 ++++++++--
 .../wal/serializer/RecordDataV1Serializer.java    | 18 +++++++++++++++---
 .../IgnitePageMemReplaceDelayedWriteUnitTest.java |  1 +
 4 files changed, 25 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7076f427/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
index 5182040..5667676 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
@@ -134,7 +134,7 @@ public class GridEncryptionManager extends GridManagerAdapter<EncryptionSpi> imp
         (IgnitePredicate<String>)key -> key.startsWith(ENCRYPTION_KEY_PREFIX);
 
     /** Group encryption keys. */
-    private Map<Integer, Serializable> grpEncKeys = new HashMap<>();
+    private final ConcurrentHashMap<Integer, Serializable> grpEncKeys = new ConcurrentHashMap<>();
 
     /** Pending generate encryption key futures. */
     private ConcurrentMap<IgniteUuid, GenerateEncryptionKeyFuture> genEncKeyFuts = new ConcurrentHashMap<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7076f427/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index 8d54498..6f7d2c5 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -85,6 +85,8 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapOutOfMemoryException;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -223,7 +225,10 @@ public class PageMemoryImpl implements PageMemoryEx {
     private IgniteWriteAheadLogManager walMgr;
 
     /** */
-    private GridEncryptionManager encMgr;
+    private final GridEncryptionManager encMgr;
+
+    /** */
+    private final EncryptionSpi encSpi;
 
     /** */
     private final IgniteLogger log;
@@ -319,6 +324,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         storeMgr = ctx.pageStore();
         walMgr = ctx.wal();
         encMgr = ctx.kernalContext().encryption();
+        encSpi = ctx.gridConfig().getEncryptionSpi();
 
         assert storeMgr != null;
         assert walMgr != null;
@@ -967,7 +973,7 @@ public class PageMemoryImpl implements PageMemoryEx {
 
     /** {@inheritDoc} */
     @Override public int realPageSize(int grpId) {
-        if (encMgr.groupKey(grpId) == null)
+        if ((encSpi instanceof NoopEncryptionSpi) || encMgr.groupKey(grpId) == null)
             return pageSize();
 
         return encPageSize;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7076f427/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
index a097361..183e147 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.wal.record.CacheState;
 import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord;
@@ -106,6 +107,7 @@ import org.apache.ignite.internal.util.typedef.T3;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD;
@@ -140,6 +142,9 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
     /** Encryption SPI instance. */
     private final EncryptionSpi encSpi;
 
+    /** Encryption manager. */
+    private final GridEncryptionManager encMgr;
+
     /** */
     private static final byte ENCRYPTED = 1;
 
@@ -155,6 +160,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
         this.co = cctx.kernalContext().cacheObjects();
         this.pageSize = cctx.database().pageSize();
         this.encSpi = cctx.gridConfig().getEncryptionSpi();
+        this.encMgr = cctx.kernalContext().encryption();
 
         //This happen on offline WAL iteration(we don't have encryption keys available).
         if (encSpi != null)
@@ -221,6 +227,9 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
      * @return {@code True} if this record should be encrypted.
      */
     private boolean needEncryption(WALRecord rec) {
+        if (encSpi instanceof NoopEncryptionSpi)
+            return false;
+
         if (!(rec instanceof WalRecordCacheGroupAware) || rec instanceof MetastoreDataRecord)
             return false;
 
@@ -232,7 +241,10 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
      * @return {@code True} if this record should be encrypted.
      */
     private boolean needEncryption(int grpId) {
-        return cctx.kernalContext().encryption().groupKey(grpId) != null;
+        if (encSpi instanceof NoopEncryptionSpi)
+            return false;
+
+        return encMgr.groupKey(grpId) != null;
     }
 
     /**
@@ -258,7 +270,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
 
         in.readFully(encData);
 
-        Serializable key = cctx.kernalContext().encryption().groupKey(grpId);
+        Serializable key = encMgr.groupKey(grpId);
 
         if (key == null)
             return new T3<>(null, grpId, plainRecType);
@@ -309,7 +321,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
         if (plainRecType != null)
             putRecordType(dst, plainRecType);
 
-        Serializable key = cctx.kernalContext().encryption().groupKey(grpId);
+        Serializable key = encMgr.groupKey(grpId);
 
         assert key != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7076f427/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java
index 5304c45..af09f9a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java
@@ -218,6 +218,7 @@ public class IgnitePageMemReplaceDelayedWriteUnitTest {
 
         GridCacheSharedContext sctx = Mockito.mock(GridCacheSharedContext.class);
 
+        when(sctx.gridConfig()).thenReturn(cfg);
         when(sctx.pageStore()).thenReturn(new NoOpPageStoreManager());
         when(sctx.wal()).thenReturn(new NoOpWALManager());
         when(sctx.database()).thenReturn(db);


[25/50] [abbrv] ignite git commit: IGNITE-9558 Avoid blocking transactions on client connect when possible - Fixes #4933.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
index 61896b5..04f411d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
@@ -393,44 +393,38 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
         }
 
         switch (writer.state()) {
-            case 21:
-                if (!writer.writeByte("isolation", isolation != null ? (byte)isolation.ordinal() : -1))
-                    return false;
-
-                writer.incrementState();
-
             case 22:
-                if (!writer.writeInt("miniId", miniId))
+                if (!writer.writeByte("isolation", isolation != null ? (byte)isolation.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
             case 23:
-                if (!writer.writeUuid("nearNodeId", nearNodeId))
+                if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
             case 24:
-                if (!writer.writeMessage("partUpdateCnt", partUpdateCnt))
+                if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
                     return false;
 
                 writer.incrementState();
 
             case 25:
-                if (!writer.writeCollection("pendingVers", pendingVers, MessageCollectionItemType.MSG))
+                if (!writer.writeUuid("nearNodeId", nearNodeId))
                     return false;
 
                 writer.incrementState();
 
             case 26:
-                if (!writer.writeMessage("writeVer", writeVer))
+                if (!writer.writeMessage("partUpdateCnt", partUpdateCnt))
                     return false;
 
                 writer.incrementState();
 
             case 27:
-                if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
+                if (!writer.writeCollection("pendingVers", pendingVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
@@ -441,6 +435,12 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 writer.incrementState();
 
+            case 29:
+                if (!writer.writeMessage("writeVer", writeVer))
+                    return false;
+
+                writer.incrementState();
+
         }
 
         return true;
@@ -457,7 +457,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
             return false;
 
         switch (reader.state()) {
-            case 21:
+            case 22:
                 byte isolationOrd;
 
                 isolationOrd = reader.readByte("isolation");
@@ -469,16 +469,8 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 22:
-                miniId = reader.readInt("miniId");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
             case 23:
-                nearNodeId = reader.readUuid("nearNodeId");
+                miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -486,7 +478,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
                 reader.incrementState();
 
             case 24:
-                partUpdateCnt = reader.readMessage("partUpdateCnt");
+                mvccSnapshot = reader.readMessage("mvccSnapshot");
 
                 if (!reader.isLastRead())
                     return false;
@@ -494,7 +486,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
                 reader.incrementState();
 
             case 25:
-                pendingVers = reader.readCollection("pendingVers", MessageCollectionItemType.MSG);
+                nearNodeId = reader.readUuid("nearNodeId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -502,7 +494,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
                 reader.incrementState();
 
             case 26:
-                writeVer = reader.readMessage("writeVer");
+                partUpdateCnt = reader.readMessage("partUpdateCnt");
 
                 if (!reader.isLastRead())
                     return false;
@@ -510,7 +502,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
                 reader.incrementState();
 
             case 27:
-                mvccSnapshot = reader.readMessage("mvccSnapshot");
+                pendingVers = reader.readCollection("pendingVers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -525,6 +517,14 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
+            case 29:
+                writeVer = reader.readMessage("writeVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
         }
 
         return reader.afterMessageRead(GridDhtTxFinishRequest.class);
@@ -537,7 +537,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 29;
+        return 30;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java
index 6d717eb..d777a22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java
@@ -173,19 +173,19 @@ public class GridDhtTxFinishResponse extends GridDistributedTxFinishResponse {
         }
 
         switch (writer.state()) {
-            case 6:
+            case 7:
                 if (!writer.writeByteArray("checkCommittedErrBytes", checkCommittedErrBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeMessage("retVal", retVal))
                     return false;
 
@@ -207,7 +207,7 @@ public class GridDhtTxFinishResponse extends GridDistributedTxFinishResponse {
             return false;
 
         switch (reader.state()) {
-            case 6:
+            case 7:
                 checkCommittedErrBytes = reader.readByteArray("checkCommittedErrBytes");
 
                 if (!reader.isLastRead())
@@ -215,7 +215,7 @@ public class GridDhtTxFinishResponse extends GridDistributedTxFinishResponse {
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -223,7 +223,7 @@ public class GridDhtTxFinishResponse extends GridDistributedTxFinishResponse {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 retVal = reader.readMessage("retVal");
 
                 if (!reader.isLastRead())
@@ -243,7 +243,7 @@ public class GridDhtTxFinishResponse extends GridDistributedTxFinishResponse {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 9;
+        return 10;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxOnePhaseCommitAckRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxOnePhaseCommitAckRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxOnePhaseCommitAckRequest.java
index 67eacd3..50f2e79 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxOnePhaseCommitAckRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxOnePhaseCommitAckRequest.java
@@ -97,7 +97,7 @@ public class GridDhtTxOnePhaseCommitAckRequest extends GridCacheMessage {
         }
 
         switch (writer.state()) {
-            case 2:
+            case 3:
                 if (!writer.writeCollection("vers", vers, MessageCollectionItemType.MSG))
                     return false;
 
@@ -119,7 +119,7 @@ public class GridDhtTxOnePhaseCommitAckRequest extends GridCacheMessage {
             return false;
 
         switch (reader.state()) {
-            case 2:
+            case 3:
                 vers = reader.readCollection("vers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -139,6 +139,6 @@ public class GridDhtTxOnePhaseCommitAckRequest extends GridCacheMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 3;
+        return 4;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
index c86265a..5b8292f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
@@ -90,7 +90,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
     /** */
     @GridDirectCollection(PartitionUpdateCountersMessage.class)
-    private Collection<PartitionUpdateCountersMessage> counters;
+    private Collection<PartitionUpdateCountersMessage> updCntrs;
 
     /** Near transaction ID. */
     private GridCacheVersion nearXidVer;
@@ -140,7 +140,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
      * @param storeWriteThrough Cache store write through flag.
      * @param retVal Need return value flag
      * @param mvccSnapshot Mvcc snapshot.
-     * @param counters Update counters for mvcc Tx.
+     * @param updCntrs Update counters for mvcc Tx.
      */
     public GridDhtTxPrepareRequest(
         IgniteUuid futId,
@@ -160,7 +160,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
         boolean storeWriteThrough,
         boolean retVal,
         MvccSnapshot mvccSnapshot,
-        Collection<PartitionUpdateCountersMessage> counters) {
+        Collection<PartitionUpdateCountersMessage> updCntrs) {
         super(tx,
             timeout,
             null,
@@ -182,7 +182,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
         this.subjId = subjId;
         this.taskNameHash = taskNameHash;
         this.mvccSnapshot = mvccSnapshot;
-        this.counters = counters;
+        this.updCntrs = updCntrs;
 
         storeWriteThrough(storeWriteThrough);
         needReturnValue(retVal);
@@ -207,7 +207,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
      * @return Update counters list.
      */
     public Collection<PartitionUpdateCountersMessage> updateCounters() {
-        return counters;
+        return updCntrs;
     }
 
     /**
@@ -436,12 +436,6 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
         }
 
         switch (writer.state()) {
-            case 20:
-                if (!writer.writeCollection("counters", counters, MessageCollectionItemType.MSG))
-                    return false;
-
-                writer.incrementState();
-
             case 21:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
@@ -521,7 +515,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 writer.incrementState();
 
             case 34:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -532,6 +526,12 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 writer.incrementState();
 
+            case 36:
+                if (!writer.writeCollection("updCntrs", updCntrs, MessageCollectionItemType.MSG))
+                    return false;
+
+                writer.incrementState();
+
         }
 
         return true;
@@ -548,14 +548,6 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
             return false;
 
         switch (reader.state()) {
-            case 20:
-                counters = reader.readCollection("counters", MessageCollectionItemType.MSG);
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
             case 21:
                 futId = reader.readIgniteUuid("futId");
 
@@ -661,7 +653,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 34:
-                topVer = reader.readMessage("topVer");
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -676,6 +668,14 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
+            case 36:
+                updCntrs = reader.readCollection("updCntrs", MessageCollectionItemType.MSG);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
         }
 
         return reader.afterMessageRead(GridDhtTxPrepareRequest.class);
@@ -688,7 +688,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 36;
+        return 37;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
index 0c2bf81..fcb14a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
@@ -245,31 +245,31 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
         }
 
         switch (writer.state()) {
-            case 10:
+            case 11:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeMap("invalidParts", invalidParts, MessageCollectionItemType.INT, MessageCollectionItemType.INT_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeCollection("nearEvicted", nearEvicted, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeCollection("preloadEntries", preloadEntries, MessageCollectionItemType.MSG))
                     return false;
 
@@ -291,7 +291,7 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
             return false;
 
         switch (reader.state()) {
-            case 10:
+            case 11:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -299,7 +299,7 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 invalidParts = reader.readMap("invalidParts", MessageCollectionItemType.INT, MessageCollectionItemType.INT_ARR, false);
 
                 if (!reader.isLastRead())
@@ -307,7 +307,7 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -315,7 +315,7 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 nearEvicted = reader.readCollection("nearEvicted", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -323,7 +323,7 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 preloadEntries = reader.readCollection("preloadEntries", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -343,7 +343,7 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 15;
+        return 16;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistRequest.java
index b3aa56d..27b7c81 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistRequest.java
@@ -247,43 +247,43 @@ public class GridDhtTxQueryEnlistRequest extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeInt("batchId", batchId))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeIgniteUuid("dhtFutId", dhtFutId))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeMessage("lockVer", lockVer))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeInt("mvccOpCnt", mvccOpCnt))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG))
                     return false;
 
@@ -305,7 +305,7 @@ public class GridDhtTxQueryEnlistRequest extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 batchId = reader.readInt("batchId");
 
                 if (!reader.isLastRead())
@@ -313,7 +313,7 @@ public class GridDhtTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 dhtFutId = reader.readIgniteUuid("dhtFutId");
 
                 if (!reader.isLastRead())
@@ -321,7 +321,7 @@ public class GridDhtTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -329,7 +329,7 @@ public class GridDhtTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 lockVer = reader.readMessage("lockVer");
 
                 if (!reader.isLastRead())
@@ -337,7 +337,7 @@ public class GridDhtTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 mvccOpCnt = reader.readInt("mvccOpCnt");
 
                 if (!reader.isLastRead())
@@ -345,7 +345,7 @@ public class GridDhtTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 byte opOrd;
 
                 opOrd = reader.readByte("op");
@@ -357,7 +357,7 @@ public class GridDhtTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 vals = reader.readCollection("vals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -372,7 +372,7 @@ public class GridDhtTxQueryEnlistRequest extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 10;
+        return 11;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistResponse.java
index f3b4aa7..18f4baa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistResponse.java
@@ -117,7 +117,7 @@ public class GridDhtTxQueryEnlistResponse extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 6;
+        return 7;
     }
 
     /** {@inheritDoc} */
@@ -135,19 +135,19 @@ public class GridDhtTxQueryEnlistResponse extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeInt("batchId", batchId))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
@@ -169,7 +169,7 @@ public class GridDhtTxQueryEnlistResponse extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 batchId = reader.readInt("batchId");
 
                 if (!reader.isLastRead())
@@ -177,7 +177,7 @@ public class GridDhtTxQueryEnlistResponse extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -185,7 +185,7 @@ public class GridDhtTxQueryEnlistResponse extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryFirstEnlistRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryFirstEnlistRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryFirstEnlistRequest.java
index 5c1bf6c..2220b29 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryFirstEnlistRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryFirstEnlistRequest.java
@@ -208,56 +208,56 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
         }
 
         switch (writer.state()) {
-            case 10:
+            case 11:
                 if (!writer.writeLong("cleanupVer", cleanupVer))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeLong("cntr", cntr))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeLong("crdVer", crdVer))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeUuid("nearNodeId", nearNodeId))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeMessage("nearXidVer", nearXidVer))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 17:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 18:
                 if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
-                if (!writer.writeMessage("topVer", topVer))
+            case 19:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -278,7 +278,7 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
             return false;
 
         switch (reader.state()) {
-            case 10:
+            case 11:
                 cleanupVer = reader.readLong("cleanupVer");
 
                 if (!reader.isLastRead())
@@ -286,7 +286,7 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 cntr = reader.readLong("cntr");
 
                 if (!reader.isLastRead())
@@ -294,7 +294,7 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 crdVer = reader.readLong("crdVer");
 
                 if (!reader.isLastRead())
@@ -302,7 +302,7 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 nearNodeId = reader.readUuid("nearNodeId");
 
                 if (!reader.isLastRead())
@@ -310,7 +310,7 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 nearXidVer = reader.readMessage("nearXidVer");
 
                 if (!reader.isLastRead())
@@ -318,7 +318,7 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -326,7 +326,7 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
 
                 reader.incrementState();
 
-            case 16:
+            case 17:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -334,7 +334,7 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
 
                 reader.incrementState();
 
-            case 17:
+            case 18:
                 timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
@@ -342,8 +342,8 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
 
                 reader.incrementState();
 
-            case 18:
-                topVer = reader.readMessage("topVer");
+            case 19:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -357,7 +357,7 @@ public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistReques
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 19;
+        return 20;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtUnlockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtUnlockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtUnlockRequest.java
index 5671d7f..3bc4de0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtUnlockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtUnlockRequest.java
@@ -113,7 +113,7 @@ public class GridDhtUnlockRequest extends GridDistributedUnlockRequest {
         }
 
         switch (writer.state()) {
-            case 8:
+            case 9:
                 if (!writer.writeCollection("nearKeys", nearKeys, MessageCollectionItemType.MSG))
                     return false;
 
@@ -135,7 +135,7 @@ public class GridDhtUnlockRequest extends GridDistributedUnlockRequest {
             return false;
 
         switch (reader.state()) {
-            case 8:
+            case 9:
                 nearKeys = reader.readCollection("nearKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -155,6 +155,6 @@ public class GridDhtUnlockRequest extends GridDistributedUnlockRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 9;
+        return 10;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCountersMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCountersMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCountersMessage.java
index 3e5953f..f30d51d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCountersMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCountersMessage.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -28,6 +29,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 /**
  * Partition update counters message.
  */
+@IgniteCodeGeneratingFail
 public class PartitionUpdateCountersMessage implements Message {
     /** */
     private static final int ITEM_SIZE = 4 /* partition */ + 8 /* initial counter */ + 8 /* updates count */;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java
index a5e9feb..0096f01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java
@@ -484,7 +484,7 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheIdMess
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 12;
+        return 13;
     }
 
     /** {@inheritDoc} */
@@ -502,55 +502,55 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheIdMess
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeLong("nearFutId", nearFutId))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeUuid("nearNodeId", nearNodeId))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
-                if (!writer.writeMessage("topVer", topVer))
+            case 11:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeMessage("writeVer", writeVer))
                     return false;
 
@@ -572,7 +572,7 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheIdMess
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -580,7 +580,7 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheIdMess
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -588,7 +588,7 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheIdMess
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 nearFutId = reader.readLong("nearFutId");
 
                 if (!reader.isLastRead())
@@ -596,7 +596,7 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheIdMess
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 nearNodeId = reader.readUuid("nearNodeId");
 
                 if (!reader.isLastRead())
@@ -604,7 +604,7 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheIdMess
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -612,7 +612,7 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheIdMess
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 byte syncModeOrd;
 
                 syncModeOrd = reader.readByte("syncMode");
@@ -624,7 +624,7 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheIdMess
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -632,15 +632,15 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheIdMess
 
                 reader.incrementState();
 
-            case 10:
-                topVer = reader.readMessage("topVer");
+            case 11:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 writeVer = reader.readMessage("writeVer");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index cbb217c..86d7b3c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -1773,7 +1773,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                 // Can not wait for topology future since it will break
                                 // GridNearAtomicCheckUpdateRequest processing.
                                 remap = !top.topologyVersionFuture().exchangeDone() ||
-                                    needRemap(req.topologyVersion(), top.readyTopologyVersion());
+                                    needRemap(req.topologyVersion(), top.readyTopologyVersion(), req.keys());
                             }
 
                             if (!remap) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java
index 0c069da..ee5eac1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java
@@ -119,7 +119,7 @@ public class GridDhtAtomicDeferredUpdateResponse extends GridCacheIdMessage impl
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeMessage("futIds", futIds))
                     return false;
 
@@ -141,7 +141,7 @@ public class GridDhtAtomicDeferredUpdateResponse extends GridCacheIdMessage impl
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 futIds = reader.readMessage("futIds");
 
                 if (!reader.isLastRead())
@@ -161,7 +161,7 @@ public class GridDhtAtomicDeferredUpdateResponse extends GridCacheIdMessage impl
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 4;
+        return 5;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicNearResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicNearResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicNearResponse.java
index 71d2321..8f11ead 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicNearResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicNearResponse.java
@@ -171,7 +171,7 @@ public class GridDhtAtomicNearResponse extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 9;
     }
 
     /** {@inheritDoc} */
@@ -210,31 +210,31 @@ public class GridDhtAtomicNearResponse extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeMessage("errs", errs))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeInt("partId", partId))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeUuid("primaryId", primaryId))
                     return false;
 
@@ -256,7 +256,7 @@ public class GridDhtAtomicNearResponse extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 errs = reader.readMessage("errs");
 
                 if (!reader.isLastRead())
@@ -264,7 +264,7 @@ public class GridDhtAtomicNearResponse extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -272,7 +272,7 @@ public class GridDhtAtomicNearResponse extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -280,7 +280,7 @@ public class GridDhtAtomicNearResponse extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 partId = reader.readInt("partId");
 
                 if (!reader.isLastRead())
@@ -288,7 +288,7 @@ public class GridDhtAtomicNearResponse extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 primaryId = reader.readUuid("primaryId");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
index 19b24b0..16be80e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
@@ -373,25 +373,25 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
         }
 
         switch (writer.state()) {
-            case 12:
+            case 13:
                 if (!writer.writeMessage("key", key))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeMessage("prevVal", prevVal))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeLong("updateCntr", updateCntr))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeMessage("val", val))
                     return false;
 
@@ -413,7 +413,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
             return false;
 
         switch (reader.state()) {
-            case 12:
+            case 13:
                 key = reader.readMessage("key");
 
                 if (!reader.isLastRead())
@@ -421,7 +421,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 prevVal = reader.readMessage("prevVal");
 
                 if (!reader.isLastRead())
@@ -429,7 +429,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 updateCntr = reader.readLong("updateCntr");
 
                 if (!reader.isLastRead())
@@ -437,7 +437,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 val = reader.readMessage("val");
 
                 if (!reader.isLastRead())
@@ -487,7 +487,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 16;
+        return 17;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
index 30be9dc..67281f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
@@ -558,97 +558,97 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
         }
 
         switch (writer.state()) {
-            case 12:
+            case 13:
                 if (!writer.writeMessage("conflictExpireTimes", conflictExpireTimes))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeCollection("conflictVers", conflictVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeCollection("entryProcessorsBytes", entryProcessorsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeBoolean("forceTransformBackups", forceTransformBackups))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 17:
                 if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 18:
                 if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
+            case 19:
                 if (!writer.writeCollection("nearEntryProcessorsBytes", nearEntryProcessorsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 19:
+            case 20:
                 if (!writer.writeMessage("nearExpireTimes", nearExpireTimes))
                     return false;
 
                 writer.incrementState();
 
-            case 20:
+            case 21:
                 if (!writer.writeCollection("nearKeys", nearKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 21:
+            case 22:
                 if (!writer.writeMessage("nearTtls", nearTtls))
                     return false;
 
                 writer.incrementState();
 
-            case 22:
+            case 23:
                 if (!writer.writeCollection("nearVals", nearVals, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 23:
+            case 24:
                 if (!writer.writeMessage("obsoleteIndexes", obsoleteIndexes))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 25:
                 if (!writer.writeCollection("prevVals", prevVals, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 25:
+            case 26:
                 if (!writer.writeMessage("ttls", ttls))
                     return false;
 
                 writer.incrementState();
 
-            case 26:
+            case 27:
                 if (!writer.writeMessage("updateCntrs", updateCntrs))
                     return false;
 
                 writer.incrementState();
 
-            case 27:
+            case 28:
                 if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG))
                     return false;
 
@@ -670,7 +670,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
             return false;
 
         switch (reader.state()) {
-            case 12:
+            case 13:
                 conflictExpireTimes = reader.readMessage("conflictExpireTimes");
 
                 if (!reader.isLastRead())
@@ -678,7 +678,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 conflictVers = reader.readCollection("conflictVers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -686,7 +686,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 entryProcessorsBytes = reader.readCollection("entryProcessorsBytes", MessageCollectionItemType.BYTE_ARR);
 
                 if (!reader.isLastRead())
@@ -694,7 +694,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 forceTransformBackups = reader.readBoolean("forceTransformBackups");
 
                 if (!reader.isLastRead())
@@ -702,7 +702,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 16:
+            case 17:
                 invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR, byte[].class);
 
                 if (!reader.isLastRead())
@@ -710,7 +710,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 17:
+            case 18:
                 keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -718,7 +718,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 18:
+            case 19:
                 nearEntryProcessorsBytes = reader.readCollection("nearEntryProcessorsBytes", MessageCollectionItemType.BYTE_ARR);
 
                 if (!reader.isLastRead())
@@ -726,7 +726,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 19:
+            case 20:
                 nearExpireTimes = reader.readMessage("nearExpireTimes");
 
                 if (!reader.isLastRead())
@@ -734,7 +734,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 20:
+            case 21:
                 nearKeys = reader.readCollection("nearKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -742,7 +742,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 21:
+            case 22:
                 nearTtls = reader.readMessage("nearTtls");
 
                 if (!reader.isLastRead())
@@ -750,7 +750,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 22:
+            case 23:
                 nearVals = reader.readCollection("nearVals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -758,7 +758,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 23:
+            case 24:
                 obsoleteIndexes = reader.readMessage("obsoleteIndexes");
 
                 if (!reader.isLastRead())
@@ -766,7 +766,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 24:
+            case 25:
                 prevVals = reader.readCollection("prevVals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -774,7 +774,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 25:
+            case 26:
                 ttls = reader.readMessage("ttls");
 
                 if (!reader.isLastRead())
@@ -782,7 +782,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 26:
+            case 27:
                 updateCntrs = reader.readMessage("updateCntrs");
 
                 if (!reader.isLastRead())
@@ -790,7 +790,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 27:
+            case 28:
                 vals = reader.readCollection("vals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -816,7 +816,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 28;
+        return 29;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
index 70bf6f5..21efbb1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
@@ -179,25 +179,25 @@ public class GridDhtAtomicUpdateResponse extends GridCacheIdMessage implements G
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeMessage("errs", errs))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeCollection("nearEvicted", nearEvicted, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeInt("partId", partId))
                     return false;
 
@@ -219,7 +219,7 @@ public class GridDhtAtomicUpdateResponse extends GridCacheIdMessage implements G
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 errs = reader.readMessage("errs");
 
                 if (!reader.isLastRead())
@@ -227,7 +227,7 @@ public class GridDhtAtomicUpdateResponse extends GridCacheIdMessage implements G
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -235,7 +235,7 @@ public class GridDhtAtomicUpdateResponse extends GridCacheIdMessage implements G
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 nearEvicted = reader.readCollection("nearEvicted", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -243,7 +243,7 @@ public class GridDhtAtomicUpdateResponse extends GridCacheIdMessage implements G
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 partId = reader.readInt("partId");
 
                 if (!reader.isLastRead())
@@ -263,7 +263,7 @@ public class GridDhtAtomicUpdateResponse extends GridCacheIdMessage implements G
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 7;
+        return 8;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java
index 62618f8..64fe1ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java
@@ -528,7 +528,7 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheIdMes
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 10;
+        return 11;
     }
 
     /** {@inheritDoc} */
@@ -546,44 +546,44 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheIdMes
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
-                if (!writer.writeMessage("topVer", topVer))
+            case 10:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -604,7 +604,7 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheIdMes
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -612,7 +612,7 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheIdMes
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -620,7 +620,7 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheIdMes
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 byte opOrd;
 
                 opOrd = reader.readByte("op");
@@ -632,7 +632,7 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheIdMes
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -640,7 +640,7 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheIdMes
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 byte syncModeOrd;
 
                 syncModeOrd = reader.readByte("syncMode");
@@ -652,7 +652,7 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheIdMes
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -660,8 +660,8 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheIdMes
 
                 reader.incrementState();
 
-            case 9:
-                topVer = reader.readMessage("topVer");
+            case 10:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicCheckUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicCheckUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicCheckUpdateRequest.java
index 96be023..a19e280 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicCheckUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicCheckUpdateRequest.java
@@ -101,7 +101,7 @@ public class GridNearAtomicCheckUpdateRequest extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 5;
+        return 6;
     }
 
     /** {@inheritDoc} */
@@ -119,13 +119,13 @@ public class GridNearAtomicCheckUpdateRequest extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeInt("partId", partId))
                     return false;
 
@@ -147,7 +147,7 @@ public class GridNearAtomicCheckUpdateRequest extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -155,7 +155,7 @@ public class GridNearAtomicCheckUpdateRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 partId = reader.readInt("partId");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
index d6956a6..170586b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
@@ -435,55 +435,55 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
         }
 
         switch (writer.state()) {
-            case 10:
+            case 11:
                 if (!writer.writeMessage("conflictExpireTimes", conflictExpireTimes))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeMessage("conflictTtls", conflictTtls))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeCollection("conflictVers", conflictVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeCollection("entryProcessorsBytes", entryProcessorsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeByteArray("expiryPlcBytes", expiryPlcBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 17:
                 if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 18:
                 if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
+            case 19:
                 if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG))
                     return false;
 
@@ -505,7 +505,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
             return false;
 
         switch (reader.state()) {
-            case 10:
+            case 11:
                 conflictExpireTimes = reader.readMessage("conflictExpireTimes");
 
                 if (!reader.isLastRead())
@@ -513,7 +513,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 conflictTtls = reader.readMessage("conflictTtls");
 
                 if (!reader.isLastRead())
@@ -521,7 +521,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 conflictVers = reader.readCollection("conflictVers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -529,7 +529,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 entryProcessorsBytes = reader.readCollection("entryProcessorsBytes", MessageCollectionItemType.BYTE_ARR);
 
                 if (!reader.isLastRead())
@@ -537,7 +537,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 expiryPlcBytes = reader.readByteArray("expiryPlcBytes");
 
                 if (!reader.isLastRead())
@@ -545,7 +545,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class);
 
                 if (!reader.isLastRead())
@@ -553,7 +553,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 16:
+            case 17:
                 invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR, byte[].class);
 
                 if (!reader.isLastRead())
@@ -561,7 +561,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 17:
+            case 18:
                 keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -569,7 +569,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 18:
+            case 19:
                 vals = reader.readCollection("vals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -601,7 +601,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 19;
+        return 20;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java
index 1ebfe80..4c20344 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java
@@ -154,7 +154,7 @@ public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingl
         }
 
         switch (writer.state()) {
-            case 12:
+            case 13:
                 if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG))
                     return false;
 
@@ -176,7 +176,7 @@ public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingl
             return false;
 
         switch (reader.state()) {
-            case 12:
+            case 13:
                 filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class);
 
                 if (!reader.isLastRead())
@@ -196,7 +196,7 @@ public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingl
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 13;
+        return 14;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
index 865d6f8..ee3d2a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
@@ -225,13 +225,13 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
         }
 
         switch (writer.state()) {
-            case 12:
+            case 13:
                 if (!writer.writeByteArray("entryProcessorBytes", entryProcessorBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
@@ -253,7 +253,7 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
             return false;
 
         switch (reader.state()) {
-            case 12:
+            case 13:
                 entryProcessorBytes = reader.readByteArray("entryProcessorBytes");
 
                 if (!reader.isLastRead())
@@ -261,7 +261,7 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR, byte[].class);
 
                 if (!reader.isLastRead())
@@ -276,7 +276,7 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 14;
+        return 15;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
index dd3a7be..83ec456 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
@@ -247,13 +247,13 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
         }
 
         switch (writer.state()) {
-            case 10:
+            case 11:
                 if (!writer.writeMessage("key", key))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeMessage("val", val))
                     return false;
 
@@ -275,7 +275,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
             return false;
 
         switch (reader.state()) {
-            case 10:
+            case 11:
                 key = reader.readMessage("key");
 
                 if (!reader.isLastRead())
@@ -283,7 +283,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 val = reader.readMessage("val");
 
                 if (!reader.isLastRead())
@@ -311,7 +311,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 12;
+        return 13;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
index 0e40788..e3f4258 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
@@ -403,43 +403,43 @@ public class GridNearAtomicUpdateResponse extends GridCacheIdMessage implements
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeMessage("errs", errs))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeCollection("mapping", mapping, MessageCollectionItemType.UUID))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeMessage("nearUpdates", nearUpdates))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeInt("partId", partId))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
-                if (!writer.writeMessage("remapTopVer", remapTopVer))
+            case 9:
+                if (!writer.writeAffinityTopologyVersion("remapTopVer", remapTopVer))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeMessage("ret", ret))
                     return false;
 
@@ -461,7 +461,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheIdMessage implements
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 errs = reader.readMessage("errs");
 
                 if (!reader.isLastRead())
@@ -469,7 +469,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheIdMessage implements
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -477,7 +477,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheIdMessage implements
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 mapping = reader.readCollection("mapping", MessageCollectionItemType.UUID);
 
                 if (!reader.isLastRead())
@@ -485,7 +485,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheIdMessage implements
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 nearUpdates = reader.readMessage("nearUpdates");
 
                 if (!reader.isLastRead())
@@ -493,7 +493,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheIdMessage implements
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 partId = reader.readInt("partId");
 
                 if (!reader.isLastRead())
@@ -501,15 +501,15 @@ public class GridNearAtomicUpdateResponse extends GridCacheIdMessage implements
 
                 reader.incrementState();
 
-            case 8:
-                remapTopVer = reader.readMessage("remapTopVer");
+            case 9:
+                remapTopVer = reader.readAffinityTopologyVersion("remapTopVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 ret = reader.readMessage("ret");
 
                 if (!reader.isLastRead())
@@ -529,7 +529,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheIdMessage implements
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 10;
+        return 11;
     }
 
     /** {@inheritDoc} */


[28/50] [abbrv] ignite git commit: IGNITE-10285 Fixed U.doInParallel may lead to deadlock - Fixes #5404.

Posted by sb...@apache.org.
IGNITE-10285 Fixed U.doInParallel may lead to deadlock - Fixes #5404.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


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

Branch: refs/heads/ignite-10044
Commit: acfdcdaa0f667b1ebebc26cea90df030804056a7
Parents: a63a81a
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon Nov 19 16:41:28 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Nov 19 16:41:28 2018 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       | 130 ++++++++++++++--
 .../internal/util/IgniteUtilsSelfTest.java      | 147 ++++++++++++++++++-
 2 files changed, 257 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/acfdcdaa/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index fc6be6d..f1f4253 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -10782,40 +10782,92 @@ public abstract class IgniteUtils {
     ) throws IgniteCheckedException, IgniteInterruptedCheckedException {
         if(srcDatas.isEmpty())
             return Collections.emptyList();
+
         int[] batchSizes = calculateOptimalBatchSizes(parallelismLvl, srcDatas.size());
 
-        List<List<T>> batches = new ArrayList<>(batchSizes.length);
+        List<Batch<T, R>> batches = new ArrayList<>(batchSizes.length);
+
+        // Set for sharing batches between executor and current thread.
+        // If executor cannot perform immediately, we will execute task in the current thread.
+        Set<Batch<T, R>> sharedBatchesSet = new GridConcurrentHashSet<>(batchSizes.length);
 
         Iterator<T> iterator = srcDatas.iterator();
 
-        for (int batchSize : batchSizes) {
-            List<T> batch = new ArrayList<>(batchSize);
+        for (int idx = 0; idx < batchSizes.length; idx++) {
+            int batchSize = batchSizes[idx];
+
+            Batch<T, R> batch = new Batch<>(batchSize);
 
             for (int i = 0; i < batchSize; i++)
-                batch.add(iterator.next());
+                batch.addTask(iterator.next());
 
             batches.add(batch);
         }
 
-        List<Future<Collection<R>>> consumerFutures = batches.stream()
-            .filter(batch -> !batch.isEmpty())
-            .map(batch -> executorSvc.submit(() -> {
-                Collection<R> results = new ArrayList<>(batch.size());
+        batches = batches.stream()
+            .filter(batch -> !batch.tasks.isEmpty())
+            // Add to set only after check that batch is not empty.
+            .peek(sharedBatchesSet::add)
+            // Setup future in batch for waiting result.
+            .peek(batch -> batch.future = executorSvc.submit(() -> {
+                // Batch was stolen by the main stream.
+                if (!sharedBatchesSet.remove(batch)) {
+                    return null;
+                }
+
+                Collection<R> results = new ArrayList<>(batch.tasks.size());
 
-                for (T item : batch)
+                for (T item : batch.tasks)
                     results.add(operation.accept(item));
 
                 return results;
             }))
             .collect(Collectors.toList());
 
-        Throwable error =null;
+        Throwable error = null;
+
+        // Stealing jobs if executor is busy and cannot process task immediately.
+        // Perform batches in a current thread.
+        for (Batch<T, R> batch : sharedBatchesSet) {
+            // Executor steal task.
+            if (!sharedBatchesSet.remove(batch))
+                continue;
+
+            Collection<R> res = new ArrayList<>(batch.tasks.size());
+
+            try {
+                for (T item : batch.tasks)
+                    res.add(operation.accept(item));
+
+                batch.result(res);
+            }
+            catch (IgniteCheckedException e) {
+                batch.result(e);
+            }
+        }
 
+        // Final result collection.
         Collection<R> results = new ArrayList<>(srcDatas.size());
 
-        for (Future<Collection<R>> future : consumerFutures) {
+        for (Batch<T, R> batch: batches) {
             try {
-                results.addAll(future.get());
+                Throwable err = batch.error;
+
+                if (err != null) {
+                    if (error == null)
+                        error = err;
+                    else
+                        error.addSuppressed(err);
+
+                    continue;
+                }
+
+                Collection<R> res = batch.result();
+
+                if (res != null)
+                    results.addAll(res);
+                else
+                    assert error != null;
             }
             catch (InterruptedException e) {
                 Thread.currentThread().interrupt();
@@ -10853,6 +10905,60 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * The batch of tasks with a batch index in global array.
+     */
+    private static class Batch<T,R> {
+        /** List tasks. */
+        private final List<T> tasks;
+
+        /** */
+        private Collection<R> result;
+
+        /** */
+        private Throwable error;
+
+        /** */
+        private Future<Collection<R>> future;
+
+        /**
+         * @param batchSize Batch size.
+         */
+        private Batch(int batchSize) {
+            this.tasks = new ArrayList<>(batchSize);
+        }
+
+        /**
+         * @param task Add task.
+         */
+        public void addTask(T task){
+            tasks.add(task);
+        }
+
+        /**
+         * @param res Setup results for tasks.
+         */
+        public void result(Collection<R> res) {
+            this.result = res;
+        }
+
+        /**
+         * @param e Throwable if task was completed with error.
+         */
+        public void result(Throwable e) {
+            this.error = e;
+        }
+
+        /**
+         * Get tasks results.
+         */
+        public Collection<R> result() throws ExecutionException, InterruptedException {
+            assert future != null;
+
+            return result != null ? result : future.get();
+        }
+    }
+
+    /**
      * Split number of tasks into optimized batches.
      * @param parallelismLvl Level of parallelism.
      * @param size number of tasks to split.

http://git-wip-us.apache.org/repos/asf/ignite/blob/acfdcdaa/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
index 2f52b21..39bb21ea 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
@@ -44,20 +44,27 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Random;
 import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.lang.GridPeerDeployAware;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -70,7 +77,9 @@ import org.apache.ignite.testframework.http.GridEmbeddedHttpServer;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.common.GridCommonTest;
 import org.jetbrains.annotations.Nullable;
+import org.junit.Assert;
 
+import static java.util.Arrays.asList;
 import static org.junit.Assert.assertArrayEquals;
 
 /**
@@ -334,7 +343,7 @@ public class IgniteUtilsSelfTest extends GridCommonAbstractTest {
 
             arr = new SelfReferencedJob[] {this, this};
 
-            col = Arrays.asList(this, this, this);
+            col = asList(this, this, this);
 
             newContext();
 
@@ -875,10 +884,10 @@ public class IgniteUtilsSelfTest extends GridCommonAbstractTest {
         TcpDiscoveryNode node250ts = new TcpDiscoveryNode();
         node250ts.version(v250ts);
 
-        assertTrue(U.isOldestNodeVersionAtLeast(v240, Arrays.asList(node240, node241, node250, node250ts)));
-        assertFalse(U.isOldestNodeVersionAtLeast(v241, Arrays.asList(node240, node241, node250, node250ts)));
-        assertTrue(U.isOldestNodeVersionAtLeast(v250, Arrays.asList(node250, node250ts)));
-        assertTrue(U.isOldestNodeVersionAtLeast(v250ts, Arrays.asList(node250, node250ts)));
+        assertTrue(U.isOldestNodeVersionAtLeast(v240, asList(node240, node241, node250, node250ts)));
+        assertFalse(U.isOldestNodeVersionAtLeast(v241, asList(node240, node241, node250, node250ts)));
+        assertTrue(U.isOldestNodeVersionAtLeast(v250, asList(node250, node250ts)));
+        assertTrue(U.isOldestNodeVersionAtLeast(v250ts, asList(node250, node250ts)));
     }
 
     /**
@@ -892,7 +901,7 @@ public class IgniteUtilsSelfTest extends GridCommonAbstractTest {
         try {
             IgniteUtils.doInParallel(3,
                 executorService,
-                Arrays.asList(1, 2, 3),
+                asList(1, 2, 3),
                 i -> {
                     try {
                         barrier.await(1, TimeUnit.SECONDS);
@@ -920,7 +929,7 @@ public class IgniteUtilsSelfTest extends GridCommonAbstractTest {
         try {
             IgniteUtils.doInParallel(2,
                 executorService,
-                Arrays.asList(1, 2, 3),
+                asList(1, 2, 3),
                 i -> {
                     try {
                         barrier.await(400, TimeUnit.MILLISECONDS);
@@ -989,6 +998,128 @@ public class IgniteUtilsSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test parallel execution steal job.
+     */
+    public void testDoInParallelWithStealingJob() throws IgniteCheckedException {
+        // Pool size should be less that input data collection.
+        ExecutorService executorService = Executors.newFixedThreadPool(1);
+
+        CountDownLatch mainThreadLatch = new CountDownLatch(1);
+        CountDownLatch poolThreadLatch = new CountDownLatch(1);
+
+        // Busy one thread from the pool.
+        executorService.submit(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    poolThreadLatch.await();
+                }
+                catch (InterruptedException e) {
+                    throw new IgniteInterruptedException(e);
+                }
+            }
+        });
+
+        List<Integer> data = asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
+
+        AtomicInteger taskProcessed = new AtomicInteger();
+
+        long threadId = Thread.currentThread().getId();
+
+        AtomicInteger curThreadCnt = new AtomicInteger();
+        AtomicInteger poolThreadCnt = new AtomicInteger();
+
+        Collection<Integer> res = U.doInParallel(10,
+            executorService,
+            data,
+            new IgniteThrowableConsumer<Integer, Integer>() {
+                @Override public Integer accept(Integer cnt) throws IgniteInterruptedCheckedException {
+                    // Release thread in pool in the middle of range.
+                    if (taskProcessed.getAndIncrement() == (data.size() / 2) - 1) {
+                        poolThreadLatch.countDown();
+
+                        try {
+                            // Await thread in thread pool complete task.
+                            mainThreadLatch.await();
+                        }
+                        catch (InterruptedException e) {
+                            Thread.currentThread().interrupt();
+
+                            throw new IgniteInterruptedCheckedException(e);
+                        }
+                    }
+
+                    // Increment if executed in current thread.
+                    if (Thread.currentThread().getId() == threadId)
+                        curThreadCnt.incrementAndGet();
+                    else {
+                        poolThreadCnt.incrementAndGet();
+
+                        if (taskProcessed.get() == data.size())
+                            mainThreadLatch.countDown();
+                    }
+
+                    return -cnt;
+                }
+            });
+
+        Assert.assertEquals(curThreadCnt.get() + poolThreadCnt.get(), data.size());
+        Assert.assertEquals(5, curThreadCnt.get());
+        Assert.assertEquals(5, poolThreadCnt.get());
+        Assert.assertEquals(asList(0, -1, -2, -3, -4, -5, -6, -7, -8, -9), res);
+    }
+
+    /**
+     * Test parallel execution steal job.
+     */
+    public void testDoInParallelWithStealingJobRunTaskInExecutor() throws Exception {
+        // Pool size should be less that input data collection.
+        ExecutorService executorService = Executors.newFixedThreadPool(2);
+
+        Future<?> f1 = executorService.submit(()-> runTask(executorService));
+        Future<?> f2 = executorService.submit(()-> runTask(executorService));
+        Future<?> f3 = executorService.submit(()-> runTask(executorService));
+
+        f1.get();
+        f2.get();
+        f3.get();
+    }
+
+    /**
+     *
+     * @param executorService Executor service.
+     */
+    private void runTask(ExecutorService executorService) {
+        List<Integer> data = asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
+
+        long threadId = Thread.currentThread().getId();
+
+        AtomicInteger curThreadCnt = new AtomicInteger();
+
+        Collection<Integer> res;
+
+        try {
+            res = U.doInParallel(10,
+                executorService,
+                data,
+                new IgniteThrowableConsumer<Integer, Integer>() {
+                    @Override public Integer accept(Integer cnt) {
+                        if (Thread.currentThread().getId() == threadId)
+                            curThreadCnt.incrementAndGet();
+
+                        return -cnt;
+                    }
+                });
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+
+        Assert.assertTrue(curThreadCnt.get() > 0);
+        Assert.assertEquals(asList(0, -1, -2, -3, -4, -5, -6, -7, -8, -9), res);
+    }
+
+    /**
      * Template method to test parallel execution
      * @param executorService ExecutorService.
      * @param size Size.
@@ -1030,7 +1161,7 @@ public class IgniteUtilsSelfTest extends GridCommonAbstractTest {
             IgniteUtils.doInParallel(
                 1,
                 executorService,
-                Arrays.asList(1, 2, 3),
+                asList(1, 2, 3),
                 i -> {
                     if (Integer.valueOf(1).equals(i))
                         throw new IgniteCheckedException(expectedException);


[02/50] [abbrv] ignite git commit: IGNITE-10237 Fixed inspections

Posted by sb...@apache.org.
IGNITE-10237 Fixed inspections


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

Branch: refs/heads/ignite-10044
Commit: a9c5aedfa13e16a04a9567ce18ecb730cf6691f3
Parents: c4ec543
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Nov 16 11:18:31 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Nov 16 11:18:31 2018 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheContext.java  | 15 ---------------
 .../persistence/metastorage/MetastorageTree.java     |  1 -
 .../metastorage/IgniteMetaStorageBasicTest.java      |  3 +--
 3 files changed, 1 insertion(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a9c5aedf/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 53c0bf0..30cf969 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -1254,21 +1254,6 @@ public class GridCacheContext<K, V> implements Externalizable {
      *
      * @param e Element.
      * @param p Predicates.
-     * @return {@code True} if predicates passed.
-     * @throws IgniteCheckedException If failed.
-     */
-    public <K1, V1> boolean isAll(
-        GridCacheEntryEx e,
-        @Nullable IgnitePredicate<Cache.Entry<K1, V1>>[] p
-    ) throws IgniteCheckedException {
-        return F.isEmpty(p) || isAll(e.<K1, V1>wrapLazyValue(keepBinary()), p);
-    }
-
-    /**
-     * Same as {@link GridFunc#isAll(Object, IgnitePredicate[])}, but safely unwraps exceptions.
-     *
-     * @param e Element.
-     * @param p Predicates.
      * @param <E> Element type.
      * @return {@code True} if predicates passed.
      * @throws IgniteCheckedException If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9c5aedf/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageTree.java
index 27de684..420c51d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageTree.java
@@ -33,7 +33,6 @@ import org.apache.ignite.internal.processors.failure.FailureProcessor;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
-import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9c5aedf/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
index b6dfc12..538b332 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
@@ -14,11 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.ignite.internal.processors.cache.persistence.metastorage;
 
 import java.io.Serializable;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Random;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;


[30/50] [abbrv] ignite git commit: IGNITE-10098 .NET: Add missing TcpCommunicationSpi properties

Posted by sb...@apache.org.
IGNITE-10098 .NET: Add missing TcpCommunicationSpi properties

This closes #5379


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

Branch: refs/heads/ignite-10044
Commit: a198b3ff5799899690419e3c0b6306a1eaebd6e7
Parents: c711d4f
Author: Max-Pudov <pu...@gmail.com>
Authored: Mon Nov 19 17:31:47 2018 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Nov 19 17:31:47 2018 +0300

----------------------------------------------------------------------
 .../utils/PlatformConfigurationUtils.java       | 12 +++
 .../Apache.Ignite.Core.Tests.DotNetCore.csproj  |  1 +
 .../Apache.Ignite.Core.Tests.csproj             |  1 +
 .../ApiParity/TcpCommunicationSpiParityTest.cs  | 84 +++++++++++++++++
 .../IgniteConfigurationTest.cs                  | 16 +++-
 .../Communication/Tcp/TcpCommunicationSpi.cs    | 99 ++++++++++++++++++--
 6 files changed, 201 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a198b3ff/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 4649584..fada9d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -711,9 +711,11 @@ public class PlatformConfigurationUtils {
             TcpCommunicationSpi comm = new TcpCommunicationSpi();
 
             comm.setAckSendThreshold(in.readInt());
+            comm.setConnectionsPerNode(in.readInt());
             comm.setConnectTimeout(in.readLong());
             comm.setDirectBuffer(in.readBoolean());
             comm.setDirectSendBuffer(in.readBoolean());
+            comm.setFilterReachableAddresses(in.readBoolean());
             comm.setIdleConnectionTimeout(in.readLong());
             comm.setLocalAddress(in.readString());
             comm.setLocalPort(in.readInt());
@@ -722,11 +724,15 @@ public class PlatformConfigurationUtils {
             comm.setMessageQueueLimit(in.readInt());
             comm.setReconnectCount(in.readInt());
             comm.setSelectorsCount(in.readInt());
+            comm.setSelectorSpins(in.readLong());
+            comm.setSharedMemoryPort(in.readInt());
             comm.setSlowClientQueueLimit(in.readInt());
             comm.setSocketReceiveBuffer(in.readInt());
             comm.setSocketSendBuffer(in.readInt());
+            comm.setSocketWriteTimeout(in.readLong());
             comm.setTcpNoDelay(in.readBoolean());
             comm.setUnacknowledgedMessagesBufferSize(in.readInt());
+            comm.setUsePairedConnections(in.readBoolean());
 
             cfg.setCommunicationSpi(comm);
         }
@@ -1297,9 +1303,11 @@ public class PlatformConfigurationUtils {
             TcpCommunicationSpi tcp = (TcpCommunicationSpi) comm;
 
             w.writeInt(tcp.getAckSendThreshold());
+            w.writeInt(tcp.getConnectionsPerNode());
             w.writeLong(tcp.getConnectTimeout());
             w.writeBoolean(tcp.isDirectBuffer());
             w.writeBoolean(tcp.isDirectSendBuffer());
+            w.writeBoolean(tcp.isFilterReachableAddresses());
             w.writeLong(tcp.getIdleConnectionTimeout());
             w.writeString(tcp.getLocalAddress());
             w.writeInt(tcp.getLocalPort());
@@ -1308,11 +1316,15 @@ public class PlatformConfigurationUtils {
             w.writeInt(tcp.getMessageQueueLimit());
             w.writeInt(tcp.getReconnectCount());
             w.writeInt(tcp.getSelectorsCount());
+            w.writeLong(tcp.getSelectorSpins());
+            w.writeInt(tcp.getSharedMemoryPort());
             w.writeInt(tcp.getSlowClientQueueLimit());
             w.writeInt(tcp.getSocketReceiveBuffer());
             w.writeInt(tcp.getSocketSendBuffer());
+            w.writeLong(tcp.getSocketWriteTimeout());
             w.writeBoolean(tcp.isTcpNoDelay());
             w.writeInt(tcp.getUnacknowledgedMessagesBufferSize());
+            w.writeBoolean(tcp.isUsePairedConnections());
         }
         else
             w.writeBoolean(false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a198b3ff/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 6550a7f..2ec0bdb 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
@@ -42,6 +42,7 @@
     <Compile Include="..\Apache.Ignite.Core.Tests\ApiParity\QueryEntityConfigurationParityTest.cs" Link="ApiParity\QueryEntityConfigurationParityTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\ApiParity\ServicesParityTest.cs" Link="ApiParity\ServicesParityTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\ApiParity\StreamerParityTest.cs" Link="ApiParity\StreamerParityTest.cs" />
+    <Compile Include="..\Apache.Ignite.Core.Tests\ApiParity\TcpCommunicationSpiParityTest.cs" Link="ApiParity\TcpCommunicationSpiParityTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\ApiParity\TransactionMetricsParityTest.cs" Link="ApiParity\TransactionMetricsParityTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\ApiParity\TransactionsParityTest.cs" Link="ApiParity\TransactionsParityTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\AssertExtensions.cs" Link="Common\AssertExtensions.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/a198b3ff/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 aa58afc..e2d4146 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
@@ -94,6 +94,7 @@
     <Compile Include="ApiParity\QueryEntityConfigurationParityTest.cs" />
     <Compile Include="ApiParity\ServicesParityTest.cs" />
     <Compile Include="ApiParity\StreamerParityTest.cs" />
+    <Compile Include="ApiParity\TcpCommunicationSpiParityTest.cs" />
     <Compile Include="ApiParity\TransactionMetricsParityTest.cs" />
     <Compile Include="ApiParity\TransactionsParityTest.cs" />
     <Compile Include="AssertExtensions.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/a198b3ff/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/TcpCommunicationSpiParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/TcpCommunicationSpiParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/TcpCommunicationSpiParityTest.cs
new file mode 100644
index 0000000..be8bd11
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/TcpCommunicationSpiParityTest.cs
@@ -0,0 +1,84 @@
+/*
+ * 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.Cache.Configuration;
+    using Apache.Ignite.Core.Communication.Tcp;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that .NET <see cref="CacheConfiguration"/> has all properties from Java configuration APIs.
+    /// </summary>
+    public class TcpCommunicationSpiParityTest
+    {
+        /** Known property name mappings. */
+        private static readonly Dictionary<string, string> KnownMappings = new Dictionary<string, string>()
+        {
+            {"SocketReceiveBuffer", "SocketReceiveBufferSize"},
+            {"SocketSendBuffer", "SocketSendBufferSize"}
+        };
+
+        /** Properties that are not needed on .NET side. */
+        private static readonly string[] UnneededProperties =
+        {
+            // Java-specific.
+            "AddressResolver",
+            "Listener",
+            "run",
+            "ReceivedMessagesByType",
+            "ReceivedMessagesByNode",
+            "SentMessagesByType",
+            "SentMessagesByNode",
+            "SentMessagesCount",
+            "SentBytesCount",
+            "ReceivedMessagesCount",
+            "ReceivedBytesCount",
+            "OutboundMessagesQueueSize",
+            "resetMetrics",
+            "dumpStats",
+            "boundPort",
+            "SpiContext",
+            "simulateNodeFailure",
+            "cancel",
+            "order",
+            "onTimeout",
+            "endTime",
+            "id",
+            "connectionIndex",
+            "NodeFilter"
+        };
+
+        /** Properties that are missing on .NET side. */
+        private static readonly string[] MissingProperties = {};
+
+        /// <summary>
+        /// Tests the cache configuration parity.
+        /// </summary>
+        [Test]
+        public void TestTcpCommunicationSpi()
+        {
+            ParityTest.CheckConfigurationParity(
+                @"modules\core\src\main\java\org\apache\ignite\spi\communication\tcp\TcpCommunicationSpi.java", 
+                typeof(TcpCommunicationSpi),
+                UnneededProperties,
+                MissingProperties,
+                KnownMappings);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a198b3ff/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index c26e5a3..f0f3b7c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -186,9 +186,11 @@ namespace Apache.Ignite.Core.Tests
                 var com = (TcpCommunicationSpi) cfg.CommunicationSpi;
                 var resCom = (TcpCommunicationSpi) resCfg.CommunicationSpi;
                 Assert.AreEqual(com.AckSendThreshold, resCom.AckSendThreshold);
+                Assert.AreEqual(com.ConnectionsPerNode, resCom.ConnectionsPerNode);
                 Assert.AreEqual(com.ConnectTimeout, resCom.ConnectTimeout);
                 Assert.AreEqual(com.DirectBuffer, resCom.DirectBuffer);
                 Assert.AreEqual(com.DirectSendBuffer, resCom.DirectSendBuffer);
+                Assert.AreEqual(com.FilterReachableAddresses, resCom.FilterReachableAddresses);
                 Assert.AreEqual(com.IdleConnectionTimeout, resCom.IdleConnectionTimeout);
                 Assert.AreEqual(com.LocalAddress, resCom.LocalAddress);
                 Assert.AreEqual(com.LocalPort, resCom.LocalPort);
@@ -197,12 +199,16 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(com.MessageQueueLimit, resCom.MessageQueueLimit);
                 Assert.AreEqual(com.ReconnectCount, resCom.ReconnectCount);
                 Assert.AreEqual(com.SelectorsCount, resCom.SelectorsCount);
+                Assert.AreEqual(com.SelectorSpins, resCom.SelectorSpins);
+                Assert.AreEqual(com.SharedMemoryPort, resCom.SharedMemoryPort);
                 Assert.AreEqual(com.SlowClientQueueLimit, resCom.SlowClientQueueLimit);
                 Assert.AreEqual(com.SocketReceiveBufferSize, resCom.SocketReceiveBufferSize);
                 Assert.AreEqual(com.SocketSendBufferSize, resCom.SocketSendBufferSize);
+                Assert.AreEqual(com.SocketWriteTimeout, resCom.SocketWriteTimeout);
                 Assert.AreEqual(com.TcpNoDelay, resCom.TcpNoDelay);
                 Assert.AreEqual(com.UnacknowledgedMessagesBufferSize, resCom.UnacknowledgedMessagesBufferSize);
-
+                Assert.AreEqual(com.UsePairedConnections, resCom.UsePairedConnections);
+                
                 Assert.AreEqual(cfg.FailureDetectionTimeout, resCfg.FailureDetectionTimeout);
                 Assert.AreEqual(cfg.SystemWorkerBlockedTimeout, resCfg.SystemWorkerBlockedTimeout);
                 Assert.AreEqual(cfg.ClientFailureDetectionTimeout, resCfg.ClientFailureDetectionTimeout);
@@ -750,7 +756,13 @@ namespace Apache.Ignite.Core.Tests
                     TcpNoDelay = false,
                     SlowClientQueueLimit = 98,
                     SocketSendBufferSize = 2045,
-                    UnacknowledgedMessagesBufferSize = 3450
+                    UnacknowledgedMessagesBufferSize = 3450,
+                    ConnectionsPerNode = 12, 
+                    UsePairedConnections = true,
+                    SharedMemoryPort = 1234,
+                    SocketWriteTimeout = 2222,
+                    SelectorSpins = 12,
+                    FilterReachableAddresses = true
                 },
                 FailureDetectionTimeout = TimeSpan.FromSeconds(3.5),
                 SystemWorkerBlockedTimeout = TimeSpan.FromSeconds(8.5),

http://git-wip-us.apache.org/repos/asf/ignite/blob/a198b3ff/modules/platforms/dotnet/Apache.Ignite.Core/Communication/Tcp/TcpCommunicationSpi.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Communication/Tcp/TcpCommunicationSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Communication/Tcp/TcpCommunicationSpi.cs
index d272906..b070f9a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Communication/Tcp/TcpCommunicationSpi.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Communication/Tcp/TcpCommunicationSpi.cs
@@ -40,6 +40,9 @@ namespace Apache.Ignite.Core.Communication.Tcp
         /// <summary> Default value of <see cref="AckSendThreshold"/> property. </summary>
         public const int DefaultAckSendThreshold = 16;
 
+        /// <summary> Default value of <see cref="ConnectionsPerNode"/> property. </summary>
+        public const int DefaultConnectionsPerNode = 1;
+
         /// <summary> Default value of <see cref="ConnectTimeout"/> property. </summary>
         public static readonly TimeSpan DefaultConnectTimeout = TimeSpan.FromSeconds(5);
 
@@ -49,6 +52,9 @@ namespace Apache.Ignite.Core.Communication.Tcp
         /// <summary> Default value of <see cref="DirectSendBuffer"/> property. </summary>
         public const bool DefaultDirectSendBuffer = false;
 
+        /// <summary> Default value of <see cref="FilterReachableAddresses"/> property. </summary>
+        public const bool DefaultFilterReachableAddresses = false;
+
         /// <summary> Default value of <see cref="IdleConnectionTimeout"/> property. </summary>
         public static readonly TimeSpan DefaultIdleConnectionTimeout = TimeSpan.FromSeconds(30);
 
@@ -70,31 +76,49 @@ namespace Apache.Ignite.Core.Communication.Tcp
         /// <summary> Default value of <see cref="SelectorsCount"/> property. </summary>
         public static readonly int DefaultSelectorsCount = Math.Min(4, Environment.ProcessorCount);
 
+        /// <summary> Default value of <see cref="SelectorSpins"/> property. </summary>
+        public const long DefaultSelectorSpins = 0;
+
+        /// <summary> Default value of <see cref="SharedMemoryPort"/> property. </summary>
+        public const int DefaultSharedMemoryPort = -1;
+
         /// <summary> Default socket buffer size. </summary>
         public const int DefaultSocketBufferSize = 32 * 1024;
 
+        /// <summary> Default value of <see cref="SocketWriteTimeout"/> property. </summary>
+        public const long DefaultSocketWriteTimeout = 2000;
+
         /// <summary> Default value of <see cref="TcpNoDelay"/> property. </summary>
         public const bool DefaultTcpNoDelay = true;
 
+        /// <summary> Default value of <see cref="UsePairedConnections"/> property. </summary>
+        public const bool DefaultUsePairedConnections = false;
+
         /// <summary>
         /// Initializes a new instance of the <see cref="TcpCommunicationSpi"/> class.
         /// </summary>
         public TcpCommunicationSpi()
         {
             AckSendThreshold = DefaultAckSendThreshold;
+            ConnectionsPerNode = DefaultConnectionsPerNode;
             ConnectTimeout = DefaultConnectTimeout;
             DirectBuffer = DefaultDirectBuffer;
             DirectSendBuffer = DefaultDirectSendBuffer;
+            FilterReachableAddresses = DefaultFilterReachableAddresses;
             IdleConnectionTimeout = DefaultIdleConnectionTimeout;
             LocalPort = DefaultLocalPort;
             LocalPortRange = DefaultLocalPortRange;
             MaxConnectTimeout = DefaultMaxConnectTimeout;
             MessageQueueLimit = DefaultMessageQueueLimit;
             ReconnectCount = DefaultReconnectCount;
+            SharedMemoryPort = DefaultSharedMemoryPort;
             SelectorsCount = DefaultSelectorsCount;
+            SelectorSpins = DefaultSelectorSpins;
             SocketReceiveBufferSize = DefaultSocketBufferSize;
             SocketSendBufferSize = DefaultSocketBufferSize;
+            SocketWriteTimeout = DefaultSocketWriteTimeout;
             TcpNoDelay = DefaultTcpNoDelay;
+            UsePairedConnections = DefaultUsePairedConnections;
         }
 
         /// <summary>
@@ -104,9 +128,11 @@ namespace Apache.Ignite.Core.Communication.Tcp
         internal TcpCommunicationSpi(IBinaryRawReader reader)
         {
             AckSendThreshold = reader.ReadInt();
+            ConnectionsPerNode = reader.ReadInt();
             ConnectTimeout = reader.ReadLongAsTimespan();
             DirectBuffer = reader.ReadBoolean();
             DirectSendBuffer = reader.ReadBoolean();
+            FilterReachableAddresses = reader.ReadBoolean();
             IdleConnectionTimeout = reader.ReadLongAsTimespan();
             LocalAddress = reader.ReadString();
             LocalPort = reader.ReadInt();
@@ -115,15 +141,19 @@ namespace Apache.Ignite.Core.Communication.Tcp
             MessageQueueLimit = reader.ReadInt();
             ReconnectCount = reader.ReadInt();
             SelectorsCount = reader.ReadInt();
+            SelectorSpins = reader.ReadLong();
+            SharedMemoryPort = reader.ReadInt();
             SlowClientQueueLimit = reader.ReadInt();
             SocketReceiveBufferSize = reader.ReadInt();
             SocketSendBufferSize = reader.ReadInt();
+            SocketWriteTimeout = reader.ReadLong();
             TcpNoDelay = reader.ReadBoolean();
             UnacknowledgedMessagesBufferSize = reader.ReadInt();
+            UsePairedConnections = reader.ReadBoolean();
         }
 
         /// <summary>
-        /// Gets or sets the number of received messages per connection to node 
+        /// Gets or sets the number of received messages per connection to node
         /// after which acknowledgment message is sent.
         /// </summary>
         [DefaultValue(DefaultAckSendThreshold)]
@@ -136,14 +166,14 @@ namespace Apache.Ignite.Core.Communication.Tcp
         public TimeSpan ConnectTimeout { get; set; }
 
         /// <summary>
-        /// Gets or sets a value indicating whether to allocate direct (ByteBuffer.allocateDirect) 
+        /// Gets or sets a value indicating whether to allocate direct (ByteBuffer.allocateDirect)
         /// or heap (ByteBuffer.allocate) buffer.
         /// </summary>
         [DefaultValue(DefaultDirectBuffer)]
         public bool DirectBuffer { get; set; }
 
         /// <summary>
-        /// Gets or sets a value indicating whether to allocate direct (ByteBuffer.allocateDirect) 
+        /// Gets or sets a value indicating whether to allocate direct (ByteBuffer.allocateDirect)
         /// or heap (ByteBuffer.allocate) send buffer.
         /// </summary>
         [DefaultValue(DefaultDirectSendBuffer)]
@@ -156,7 +186,7 @@ namespace Apache.Ignite.Core.Communication.Tcp
         public TimeSpan IdleConnectionTimeout { get; set; }
 
         /// <summary>
-        /// Gets or sets the local host address for socket binding. Note that one node could have 
+        /// Gets or sets the local host address for socket binding. Note that one node could have
         /// additional addresses beside the loopback one. This configuration parameter is optional.
         /// </summary>
         public string LocalAddress { get; set; }
@@ -193,7 +223,7 @@ namespace Apache.Ignite.Core.Communication.Tcp
         /// <summary>
         /// Gets or sets the message queue limit for incoming and outgoing messages.
         /// <para />
-        /// When set to positive number send queue is limited to the configured value. 
+        /// When set to positive number send queue is limited to the configured value.
         /// <c>0</c> disables the limitation.
         /// </summary>
         [DefaultValue(DefaultMessageQueueLimit)]
@@ -216,11 +246,11 @@ namespace Apache.Ignite.Core.Communication.Tcp
         /// <summary>
         /// Gets or sets slow client queue limit.
         /// <para/>
-        /// When set to a positive number, communication SPI will monitor clients outbound message queue sizes 
+        /// When set to a positive number, communication SPI will monitor clients outbound message queue sizes
         /// and will drop those clients whose queue exceeded this limit.
         /// <para/>
         /// Usually this value should be set to the same value as <see cref="MessageQueueLimit"/> which controls
-        /// message back-pressure for server nodes. The default value for this parameter is <c>0</c> 
+        /// message back-pressure for server nodes. The default value for this parameter is <c>0</c>
         /// which means unlimited.
         /// </summary>
         public int SlowClientQueueLimit { get; set; }
@@ -230,7 +260,7 @@ namespace Apache.Ignite.Core.Communication.Tcp
         /// </summary>
         [DefaultValue(DefaultSocketBufferSize)]
         public int SocketReceiveBufferSize { get; set; }
-        
+
         /// <summary>
         /// Gets or sets the size of the socket send buffer.
         /// </summary>
@@ -250,21 +280,66 @@ namespace Apache.Ignite.Core.Communication.Tcp
         public bool TcpNoDelay { get; set; }
 
         /// <summary>
-        /// Gets or sets the maximum number of stored unacknowledged messages per connection to node. 
-        /// If number of unacknowledged messages exceeds this number 
+        /// Gets or sets the maximum number of stored unacknowledged messages per connection to node.
+        /// If number of unacknowledged messages exceeds this number
         /// then connection to node is closed and reconnect is attempted.
         /// </summary>
         public int UnacknowledgedMessagesBufferSize { get; set; }
 
         /// <summary>
+        /// Gets or sets the number of connections per node.
+        /// </summary>
+        [DefaultValue(DefaultConnectionsPerNode)]
+        public int ConnectionsPerNode { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether separate connections should be used for incoming and outgoing data.
+        /// Set this to <c>true</c> if <see cref="ConnectionsPerNode"/> should maintain connection for outgoing
+        /// and incoming messages separately. In this case total number of connections between local and each remote
+        /// node is equals to <see cref="ConnectionsPerNode"/> * 2.
+        /// </summary>
+        public bool UsePairedConnections { get; set; }
+
+        /// <summary>
+        /// Gets or sets a local port to accept shared memory connections.
+        /// </summary>
+        [DefaultValue(DefaultSharedMemoryPort)]
+        public int SharedMemoryPort { get; set; }
+
+        /// <summary>
+        /// Gets or sets socket write timeout for TCP connection. If message can not be written to
+        /// socket within this time then connection is closed and reconnect is attempted.
+        /// <para />
+        /// Default value is <see cref="DefaultSocketWriteTimeout"/>.
+        /// </summary>
+        [DefaultValue(DefaultSocketWriteTimeout)]
+        public long SocketWriteTimeout { get; set; }
+
+        /// <summary>
+        /// Gets or sets a values that defines how many non-blocking selectors should be made.
+        /// Can be set to <see cref="Int64.MaxValue"/> so selector threads will never block.
+        /// <para />
+        /// Default value is <see cref="DefaultSelectorSpins"/>.
+        /// </summary>
+        public long SelectorSpins { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether filter for reachable addresses
+        /// should be enabled on creating tcp client.
+        /// </summary>
+        public bool FilterReachableAddresses { get; set; }
+
+        /// <summary>
         /// Writes this instance to the specified writer.
         /// </summary>
         internal void Write(IBinaryRawWriter writer)
         {
             writer.WriteInt(AckSendThreshold);
+            writer.WriteInt(ConnectionsPerNode);
             writer.WriteLong((long) ConnectTimeout.TotalMilliseconds);
             writer.WriteBoolean(DirectBuffer);
             writer.WriteBoolean(DirectSendBuffer);
+            writer.WriteBoolean(FilterReachableAddresses);
             writer.WriteLong((long) IdleConnectionTimeout.TotalMilliseconds);
             writer.WriteString(LocalAddress);
             writer.WriteInt(LocalPort);
@@ -273,11 +348,15 @@ namespace Apache.Ignite.Core.Communication.Tcp
             writer.WriteInt(MessageQueueLimit);
             writer.WriteInt(ReconnectCount);
             writer.WriteInt(SelectorsCount);
+            writer.WriteLong(SelectorSpins);
+            writer.WriteInt(SharedMemoryPort);
             writer.WriteInt(SlowClientQueueLimit);
             writer.WriteInt(SocketReceiveBufferSize);
             writer.WriteInt(SocketSendBufferSize);
+            writer.WriteLong(SocketWriteTimeout);
             writer.WriteBoolean(TcpNoDelay);
             writer.WriteInt(UnacknowledgedMessagesBufferSize);
+            writer.WriteBoolean(UsePairedConnections);
         }
     }
 }


[42/50] [abbrv] ignite git commit: IGNITE-10341 Added loss policy tests with persistence - Fixes #5444.

Posted by sb...@apache.org.
IGNITE-10341 Added loss policy tests with persistence - Fixes #5444.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


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

Branch: refs/heads/ignite-10044
Commit: eb8888561561f92343e0620082b81c757153b4be
Parents: a9b5c8c
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Tue Nov 20 15:15:16 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Tue Nov 20 15:15:16 2018 +0300

----------------------------------------------------------------------
 .../IgniteCachePartitionLossPolicySelfTest.java | 592 +++++++++++++------
 1 file changed, 421 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/eb888856/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
index f02563d..cfe578d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
@@ -19,10 +19,11 @@ package org.apache.ignite.internal.processors.cache.distributed;
 
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Semaphore;
@@ -36,6 +37,8 @@ import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.CacheRebalancingEvent;
 import org.apache.ignite.events.Event;
@@ -43,22 +46,25 @@ import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.TestDelayingCommunicationSpi;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsAbstractMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId;
 
 /**
  *
@@ -77,12 +83,14 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     private int backups;
 
     /** */
-    private final AtomicBoolean delayPartExchange = new AtomicBoolean();
+    private final AtomicBoolean delayPartExchange = new AtomicBoolean(false);
 
     /** */
     private final TopologyChanger killSingleNode = new TopologyChanger(
-        false, asList(3), asList(0, 1, 2, 4), 0
-    );
+        false, singletonList(3), asList(0, 1, 2, 4), 0);
+
+    /** */
+    private boolean isPersistenceEnabled;
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
@@ -101,9 +109,16 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
 
         cfg.setClientMode(client);
 
+        cfg.setCacheConfiguration(cacheConfiguration());
+
         cfg.setConsistentId(gridName);
 
-        cfg.setCacheConfiguration(cacheConfiguration());
+        cfg.setDataStorageConfiguration(
+            new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration()
+                        .setPersistenceEnabled(isPersistenceEnabled)
+                ));
 
         return cfg;
     }
@@ -124,19 +139,25 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     }
 
     /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
-        cleanPersistenceDir();
-
         delayPartExchange.set(false);
 
+        partLossPlc = PartitionLossPolicy.IGNORE;
+
         backups = 0;
+
+        isPersistenceEnabled = false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        super.afterTest();
     }
 
     /**
@@ -151,6 +172,17 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception if failed.
      */
+    public void testReadOnlySafeWithPersistence() throws Exception {
+        partLossPlc = PartitionLossPolicy.READ_ONLY_SAFE;
+
+        isPersistenceEnabled = true;
+
+        checkLostPartition(false, true, killSingleNode);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
     public void testReadOnlyAll() throws Exception {
         partLossPlc = PartitionLossPolicy.READ_ONLY_ALL;
 
@@ -160,6 +192,19 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception if failed.
      */
+    public void testReadOnlyAllWithPersistence() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10041");
+
+        partLossPlc = PartitionLossPolicy.READ_ONLY_ALL;
+
+        isPersistenceEnabled = true;
+
+        checkLostPartition(false, false, killSingleNode);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
     public void testReadWriteSafe() throws Exception {
         partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
 
@@ -169,6 +214,17 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception if failed.
      */
+    public void testReadWriteSafeWithPersistence() throws Exception {
+        partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
+
+        isPersistenceEnabled = true;
+
+        checkLostPartition(true, true, killSingleNode);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
     public void testReadWriteAll() throws Exception {
         partLossPlc = PartitionLossPolicy.READ_WRITE_ALL;
 
@@ -178,6 +234,19 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception if failed.
      */
+    public void testReadWriteAllWithPersistence() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10041");
+
+        partLossPlc = PartitionLossPolicy.READ_WRITE_ALL;
+
+        isPersistenceEnabled = true;
+
+        checkLostPartition(true, false, killSingleNode);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
     public void testReadWriteSafeAfterKillTwoNodes() throws Exception {
         partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
 
@@ -187,6 +256,17 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception if failed.
      */
+    public void testReadWriteSafeAfterKillTwoNodesWithPersistence() throws Exception {
+        partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
+
+        isPersistenceEnabled = true;
+
+        checkLostPartition(true, true, new TopologyChanger(false, asList(3, 2), asList(0, 1, 4), 0));
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
     public void testReadWriteSafeAfterKillTwoNodesWithDelay() throws Exception {
         partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
 
@@ -196,6 +276,17 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception if failed.
      */
+    public void testReadWriteSafeAfterKillTwoNodesWithDelayWithPersistence() throws Exception {
+        partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
+
+        isPersistenceEnabled = true;
+
+        checkLostPartition(true, true, new TopologyChanger(false, asList(3, 2), asList(0, 1, 4), 20));
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
     public void testReadWriteSafeWithBackupsAfterKillThreeNodes() throws Exception {
         partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
 
@@ -207,6 +298,21 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception if failed.
      */
+    public void testReadWriteSafeWithBackupsAfterKillThreeNodesWithPersistence() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10043");
+
+        partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
+
+        backups = 1;
+
+        isPersistenceEnabled = true;
+
+        checkLostPartition(true, true, new TopologyChanger(true, asList(3, 2, 1), asList(0, 4), 0));
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
     public void testReadWriteSafeAfterKillCrd() throws Exception {
         partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
 
@@ -216,6 +322,17 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception if failed.
      */
+    public void testReadWriteSafeAfterKillCrdWithPersistence() throws Exception {
+        partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
+
+        isPersistenceEnabled = true;
+
+        checkLostPartition(true, true, new TopologyChanger(true, asList(3, 0), asList(1, 2, 4), 0));
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
     public void testReadWriteSafeWithBackups() throws Exception {
         partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
 
@@ -227,6 +344,19 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception if failed.
      */
+    public void testReadWriteSafeWithBackupsWithPersistence() throws Exception {
+        partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
+
+        backups = 1;
+
+        isPersistenceEnabled = true;
+
+        checkLostPartition(true, true, new TopologyChanger(true, asList(3, 2), asList(0, 1, 4), 0));
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
     public void testReadWriteSafeWithBackupsAfterKillCrd() throws Exception {
         partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
 
@@ -236,12 +366,81 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     }
 
     /**
-     * @param topChanger topology changer.
      * @throws Exception if failed.
      */
-    public void testIgnore(TopologyChanger topChanger) throws Exception {
+    public void testReadWriteSafeWithBackupsAfterKillCrdWithPersistence() throws Exception {
+        partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
+
+        backups = 1;
+
+        isPersistenceEnabled = true;
+
+        checkLostPartition(true, true, new TopologyChanger(true, asList(3, 0), asList(1, 2, 4), 0));
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testIgnore() throws Exception {
         fail("https://issues.apache.org/jira/browse/IGNITE-5078");
 
+        partLossPlc = PartitionLossPolicy.IGNORE;
+
+        checkIgnore(killSingleNode);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testIgnoreWithPersistence() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-5078");
+
+        fail("https://issues.apache.org/jira/browse/IGNITE-10041");
+
+        partLossPlc = PartitionLossPolicy.IGNORE;
+
+        isPersistenceEnabled = true;
+
+        checkIgnore(killSingleNode);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testIgnoreKillThreeNodes() throws Exception {
+        partLossPlc = PartitionLossPolicy.IGNORE;
+
+        // TODO aliveNodes should include node 4, but it fails due to https://issues.apache.org/jira/browse/IGNITE-5078.
+        // TODO need to add 4 to the aliveNodes after IGNITE-5078 is fixed.
+        // TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, Arrays.asList(1, 2, 3), Arrays.asList(0, 4), 0);
+        TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, asList(1, 2, 3), singletonList(0), 0);
+
+        checkIgnore(onlyCrdIsAlive);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testIgnoreKillThreeNodesWithPersistence() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10041");
+
+        partLossPlc = PartitionLossPolicy.IGNORE;
+
+        isPersistenceEnabled = true;
+
+        // TODO aliveNodes should include node 4, but it fails due to https://issues.apache.org/jira/browse/IGNITE-5078.
+        // TODO need to add 4 to the aliveNodes after IGNITE-5078 is fixed.
+        // TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, Arrays.asList(1, 2, 3), Arrays.asList(0, 4), 0);
+        TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, asList(1, 2, 3), singletonList(0), 0);
+
+        checkIgnore(onlyCrdIsAlive);
+    }
+
+    /**
+     * @param topChanger topology changer.
+     * @throws Exception if failed.
+     */
+    private void checkIgnore(TopologyChanger topChanger) throws Exception {
         topChanger.changeTopology();
 
         for (Ignite ig : G.allGrids()) {
@@ -270,42 +469,80 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     private void checkLostPartition(boolean canWrite, boolean safe, TopologyChanger topChanger) throws Exception {
         assert partLossPlc != null;
 
-        int part = topChanger.changeTopology().get(0);
+        List<Integer> lostParts = topChanger.changeTopology();
+
+        // Wait for all grids (servers and client) have same topology version
+        // to make sure that all nodes received map with lost partition.
+        boolean success = GridTestUtils.waitForCondition(() -> {
+            AffinityTopologyVersion last = null;
+            for (Ignite ig : G.allGrids()) {
+                AffinityTopologyVersion ver = ((IgniteEx)ig).context().cache().context().exchange().readyAffinityVersion();
+
+                if (last != null && !last.equals(ver))
+                    return false;
+
+                last = ver;
+            }
+
+            return true;
+        }, 10000);
+
+        assertTrue("Failed to wait for new topology", success);
 
         for (Ignite ig : G.allGrids()) {
             info("Checking node: " + ig.cluster().localNode().id());
 
             IgniteCache<Integer, Integer> cache = ig.cache(DEFAULT_CACHE_NAME);
 
-            verifyCacheOps(canWrite, safe, part, ig);
+            verifyLostPartitions(ig, lostParts);
 
-            // Check we can read and write to lost partition in recovery mode.
-            IgniteCache<Integer, Integer> recoverCache = cache.withPartitionRecover();
+            verifyCacheOps(canWrite, safe, ig);
 
-            for (int lostPart : recoverCache.lostPartitions()) {
-                recoverCache.get(lostPart);
-                recoverCache.put(lostPart, lostPart);
-            }
+            validateQuery(safe, ig);
 
-            // Check that writing in recover mode does not clear partition state.
-            verifyCacheOps(canWrite, safe, part, ig);
+            // TODO withPartitionRecover doesn't work with BLT - https://issues.apache.org/jira/browse/IGNITE-10041.
+            if (!isPersistenceEnabled) {
+                // Check we can read and write to lost partition in recovery mode.
+                IgniteCache<Integer, Integer> recoverCache = cache.withPartitionRecover();
 
-            // Validate queries.
-            validateQuery(safe, ig);
+                for (int lostPart : recoverCache.lostPartitions()) {
+                    recoverCache.get(lostPart);
+                    recoverCache.put(lostPart, lostPart);
+                }
+
+                // Check that writing in recover mode does not clear partition state.
+                verifyLostPartitions(ig, lostParts);
+
+                verifyCacheOps(canWrite, safe, ig);
+
+                validateQuery(safe, ig);
+            }
         }
 
-        checkNewNode(true, canWrite, safe, part);
-        checkNewNode(false, canWrite, safe, part);
+        checkNewNode(true, canWrite, safe);
+        checkNewNode(false, canWrite, safe);
+
+        // Bring all nodes back.
+        for (int i : topChanger.killNodes) {
+            IgniteEx grd = startGrid(i);
 
-        // Check that partition state does not change after we start a new node.
-        IgniteEx grd = startGrid(3);
+            info("Newly started node: " + grd.cluster().localNode().id());
 
-        info("Newly started node: " + grd.cluster().localNode().id());
+            // Check that partition state does not change after we start each node.
+            // TODO With persistence enabled LOST partitions become OWNING after a node joins back - https://issues.apache.org/jira/browse/IGNITE-10044.
+            if (!isPersistenceEnabled) {
+                for (Ignite ig : G.allGrids()) {
+                    verifyCacheOps(canWrite, safe, ig);
 
-        for (Ignite ig : G.allGrids())
-            verifyCacheOps(canWrite, safe, part, ig);
+                    // TODO Query effectively waits for rebalance due to https://issues.apache.org/jira/browse/IGNITE-10057
+                    // TODO and after resetLostPartition there is another OWNING copy in the cluster due to https://issues.apache.org/jira/browse/IGNITE-10058.
+                    // TODO Uncomment after https://issues.apache.org/jira/browse/IGNITE-10058 is fixed.
+//                    validateQuery(safe, ig);
+                }
+            }
+        }
 
-        ignite(4).resetLostPartitions(Collections.singletonList(DEFAULT_CACHE_NAME));
+        ignite(4).resetLostPartitions(singletonList(DEFAULT_CACHE_NAME));
 
         awaitPartitionMapExchange(true, true, null);
 
@@ -321,6 +558,16 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
 
                 cache.put(i, i);
             }
+
+            for (int i = 0; i < parts; i++) {
+                checkQueryPasses(ig, false, i);
+
+                if (shouldExecuteLocalQuery(ig, i))
+                    checkQueryPasses(ig, true, i);
+
+            }
+
+            checkQueryPasses(ig, false);
         }
     }
 
@@ -328,25 +575,23 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
      * @param client Client flag.
      * @param canWrite Can write flag.
      * @param safe Safe flag.
-     * @param part List of lost partitions.
      * @throws Exception If failed to start a new node.
      */
     private void checkNewNode(
         boolean client,
         boolean canWrite,
-        boolean safe,
-        int part
+        boolean safe
     ) throws Exception {
         this.client = client;
 
         try {
-            IgniteEx cl = startGrid("newNode");
+            IgniteEx cl = (IgniteEx)startGrid("newNode");
 
-            CacheGroupContext grpCtx = cl.context().cache().cacheGroup(cacheId(DEFAULT_CACHE_NAME));
+            CacheGroupContext grpCtx = cl.context().cache().cacheGroup(CU.cacheId(DEFAULT_CACHE_NAME));
 
             assertTrue(grpCtx.needsRecovery());
 
-            verifyCacheOps(canWrite, safe, part, cl);
+            verifyCacheOps(canWrite, safe, cl);
 
             validateQuery(safe, cl);
         }
@@ -358,20 +603,26 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
     }
 
     /**
-     *
+     * @param node Node.
+     * @param lostParts Lost partition IDs.
+     */
+    private void verifyLostPartitions(Ignite node, List<Integer> lostParts) {
+        IgniteCache<Integer, Integer> cache = node.cache(DEFAULT_CACHE_NAME);
+
+        Set<Integer> actualSortedLostParts = new TreeSet<>(cache.lostPartitions());
+        Set<Integer> expSortedLostParts = new TreeSet<>(lostParts);
+
+        assertEqualsCollections(expSortedLostParts, actualSortedLostParts);
+    }
+
+    /**
      * @param canWrite {@code True} if writes are allowed.
      * @param safe {@code True} if lost partition should trigger exception.
-     * @param part Lost partition ID.
      * @param ig Ignite instance.
      */
-    private void verifyCacheOps(boolean canWrite, boolean safe, int part, Ignite ig) {
+    private void verifyCacheOps(boolean canWrite, boolean safe, Ignite ig) {
         IgniteCache<Integer, Integer> cache = ig.cache(DEFAULT_CACHE_NAME);
 
-        Collection<Integer> lost = cache.lostPartitions();
-
-        assertTrue("Failed to find expected lost partition [exp=" + part + ", lost=" + lost + ']',
-            lost.contains(part));
-
         int parts = ig.affinity(DEFAULT_CACHE_NAME).partitions();
 
         // Check read.
@@ -419,7 +670,7 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
      * @param nodes List of nodes to find partition.
      * @return List of partitions that aren't primary or backup for specified nodes.
      */
-    protected List<Integer> noPrimaryOrBackupPartition(List<Integer> nodes) {
+    private List<Integer> noPrimaryOrBackupPartition(List<Integer> nodes) {
         Affinity<Object> aff = ignite(4).affinity(DEFAULT_CACHE_NAME);
 
         List<Integer> parts = new ArrayList<>();
@@ -444,127 +695,6 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
         return parts;
     }
 
-    /** */
-    private class TopologyChanger {
-        /** Flag to delay partition exchange */
-        private boolean delayExchange;
-
-        /** List of nodes to kill */
-        private List<Integer> killNodes;
-
-        /** List of nodes to be alive */
-        private List<Integer> aliveNodes;
-
-        /** Delay between node stops */
-        private long stopDelay;
-
-        /**
-         * @param delayExchange Flag for delay partition exchange.
-         * @param killNodes List of nodes to kill.
-         * @param aliveNodes List of nodes to be alive.
-         * @param stopDelay Delay between stopping nodes.
-         */
-        public TopologyChanger(
-            boolean delayExchange,
-            List<Integer> killNodes,
-            List<Integer> aliveNodes,
-            long stopDelay
-        ) {
-            this.delayExchange = delayExchange;
-            this.killNodes = killNodes;
-            this.aliveNodes = aliveNodes;
-            this.stopDelay = stopDelay;
-        }
-
-        /**
-         * @return Lost partition ID.
-         * @throws Exception If failed.
-         */
-        protected List<Integer> changeTopology() throws Exception {
-            startGrids(4);
-
-            Affinity<Object> aff = ignite(0).affinity(DEFAULT_CACHE_NAME);
-
-            for (int i = 0; i < aff.partitions(); i++)
-                ignite(0).cache(DEFAULT_CACHE_NAME).put(i, i);
-
-            client = true;
-
-            startGrid(4);
-
-            client = false;
-
-            for (int i = 0; i < 5; i++)
-                info(">>> Node [idx=" + i + ", nodeId=" + ignite(i).cluster().localNode().id() + ']');
-
-            awaitPartitionMapExchange();
-
-            final List<Integer> parts = noPrimaryOrBackupPartition(aliveNodes);
-
-            if (parts.isEmpty())
-                throw new IllegalStateException("No partition on nodes: " + killNodes);
-
-            final List<Map<Integer, Semaphore>> lostMap = new ArrayList<>();
-
-            for (int i : aliveNodes) {
-                HashMap<Integer, Semaphore> semaphoreMap = new HashMap<>();
-
-                for (Integer part : parts)
-                    semaphoreMap.put(part, new Semaphore(0));
-
-                lostMap.add(semaphoreMap);
-
-                grid(i).events().localListen(new P1<Event>() {
-                    @Override public boolean apply(Event evt) {
-                        assert evt.type() == EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST;
-
-                        CacheRebalancingEvent cacheEvt = (CacheRebalancingEvent)evt;
-
-                        if (F.eq(DEFAULT_CACHE_NAME, cacheEvt.cacheName())) {
-                            if (semaphoreMap.containsKey(cacheEvt.partition()))
-                                semaphoreMap.get(cacheEvt.partition()).release();
-                        }
-
-                        return true;
-                    }
-                }, EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST);
-            }
-
-            if (delayExchange)
-                delayPartExchange.set(true);
-
-            ExecutorService executor = Executors.newFixedThreadPool(killNodes.size());
-
-            for (Integer node : killNodes) {
-                executor.submit(new Runnable() {
-                    @Override public void run() {
-                        grid(node).close();
-                    }
-                });
-
-                Thread.sleep(stopDelay);
-            }
-
-            executor.shutdown();
-
-            delayPartExchange.set(false);
-
-            Thread.sleep(5_000L);
-
-            for (Map<Integer, Semaphore> map : lostMap) {
-                for (Map.Entry<Integer, Semaphore> entry : map.entrySet())
-                    assertTrue("Failed to wait for partition LOST event for partition:" + entry.getKey(), entry.getValue().tryAcquire(1));
-            }
-
-            for (Map<Integer, Semaphore> map : lostMap) {
-                for (Map.Entry<Integer, Semaphore> entry : map.entrySet())
-                    assertFalse("Partition LOST event raised twice for partition:" + entry.getKey(), entry.getValue().tryAcquire(1));
-            }
-
-            return parts;
-        }
-    }
-
     /**
      * Validate query execution on a node.
      *
@@ -685,4 +815,124 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
         // TODO Need to add an actual check after https://issues.apache.org/jira/browse/IGNITE-9902 is fixed.
         // No-op.
     }
+
+    /** */
+    private class TopologyChanger {
+        /** Flag to delay partition exchange */
+        private boolean delayExchange;
+
+        /** List of nodes to kill */
+        private List<Integer> killNodes;
+
+        /** List of nodes to be alive */
+        private List<Integer> aliveNodes;
+
+        /** Delay between node stops */
+        private long stopDelay;
+
+        /**
+         * @param delayExchange Flag for delay partition exchange.
+         * @param killNodes List of nodes to kill.
+         * @param aliveNodes List of nodes to be alive.
+         * @param stopDelay Delay between stopping nodes.
+         */
+        private TopologyChanger(boolean delayExchange, List<Integer> killNodes, List<Integer> aliveNodes,
+            long stopDelay) {
+            this.delayExchange = delayExchange;
+            this.killNodes = killNodes;
+            this.aliveNodes = aliveNodes;
+            this.stopDelay = stopDelay;
+        }
+
+        /**
+         * @return Lost partition ID.
+         * @throws Exception If failed.
+         */
+        private List<Integer> changeTopology() throws Exception {
+            startGrids(4);
+
+            if (isPersistenceEnabled)
+                grid(0).cluster().active(true);
+
+            Affinity<Object> aff = ignite(0).affinity(DEFAULT_CACHE_NAME);
+
+            for (int i = 0; i < aff.partitions(); i++)
+                ignite(0).cache(DEFAULT_CACHE_NAME).put(i, i);
+
+            client = true;
+
+            startGrid(4);
+
+            client = false;
+
+            for (int i = 0; i < 5; i++)
+                info(">>> Node [idx=" + i + ", nodeId=" + ignite(i).cluster().localNode().id() + ']');
+
+            awaitPartitionMapExchange();
+
+            final List<Integer> parts = noPrimaryOrBackupPartition(aliveNodes);
+
+            if (parts.isEmpty())
+                throw new IllegalStateException("No partition on nodes: " + killNodes);
+
+            final List<Map<Integer, Semaphore>> lostMap = new ArrayList<>();
+
+            for (int i : aliveNodes) {
+                HashMap<Integer, Semaphore> semaphoreMap = new HashMap<>();
+
+                for (Integer part : parts)
+                    semaphoreMap.put(part, new Semaphore(0));
+
+                lostMap.add(semaphoreMap);
+
+                grid(i).events().localListen(new P1<Event>() {
+                    @Override public boolean apply(Event evt) {
+                        assert evt.type() == EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST;
+
+                        CacheRebalancingEvent cacheEvt = (CacheRebalancingEvent)evt;
+
+                        if (F.eq(DEFAULT_CACHE_NAME, cacheEvt.cacheName())) {
+                            if (semaphoreMap.containsKey(cacheEvt.partition()))
+                                semaphoreMap.get(cacheEvt.partition()).release();
+                        }
+
+                        return true;
+                    }
+                }, EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST);
+            }
+
+            if (delayExchange)
+                delayPartExchange.set(true);
+
+            ExecutorService executor = Executors.newFixedThreadPool(killNodes.size());
+
+            for (Integer node : killNodes) {
+                executor.submit(new Runnable() {
+                    @Override public void run() {
+                        grid(node).close();
+                    }
+                });
+
+                Thread.sleep(stopDelay);
+            }
+
+            executor.shutdown();
+
+            delayPartExchange.set(false);
+
+            Thread.sleep(5_000L);
+
+            for (Map<Integer, Semaphore> map : lostMap) {
+                for (Map.Entry<Integer, Semaphore> entry : map.entrySet())
+                    assertTrue("Failed to wait for partition LOST event for partition:" + entry.getKey(), entry.getValue().tryAcquire(1));
+            }
+
+            for (Map<Integer, Semaphore> map : lostMap) {
+                for (Map.Entry<Integer, Semaphore> entry : map.entrySet())
+                    assertFalse("Partition LOST event raised twice for partition:" + entry.getKey(), entry.getValue().tryAcquire(1));
+            }
+
+            return parts;
+        }
+    }
 }


[33/50] [abbrv] ignite git commit: IGNITE-9946 Web Console: Fixed fields validation order.

Posted by sb...@apache.org.
IGNITE-9946 Web Console: Fixed fields validation order.


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

Branch: refs/heads/ignite-10044
Commit: 7da186455f6fad9afd62be7f5802a4ee30667ef6
Parents: 7c632e6
Author: Vasiliy Sisko <vs...@gridgain.com>
Authored: Tue Nov 20 14:13:48 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Nov 20 14:13:48 2018 +0700

----------------------------------------------------------------------
 .../web-console/frontend/app/primitives/datepicker/index.pug | 6 +++---
 .../frontend/app/primitives/form-field/dropdown.pug          | 6 +++---
 .../web-console/frontend/app/primitives/form-field/email.pug | 6 +++---
 .../frontend/app/primitives/form-field/number.pug            | 6 +++---
 .../frontend/app/primitives/form-field/password.pug          | 8 ++++----
 .../web-console/frontend/app/primitives/form-field/phone.pug | 5 +++--
 .../web-console/frontend/app/primitives/form-field/text.pug  | 6 +++---
 .../frontend/app/primitives/form-field/typeahead.pug         | 6 +++---
 .../web-console/frontend/app/primitives/timepicker/index.pug | 6 +++---
 9 files changed, 28 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7da18645/modules/web-console/frontend/app/primitives/datepicker/index.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/datepicker/index.pug b/modules/web-console/frontend/app/primitives/datepicker/index.pug
index 28cd1a0..dd586dc 100644
--- a/modules/web-console/frontend/app/primitives/datepicker/index.pug
+++ b/modules/web-console/frontend/app/primitives/datepicker/index.pug
@@ -58,8 +58,8 @@ mixin form-field__datepicker({ label, model, name, mindate, maxdate, minview = 1
             ng-messages=`$input.$error`
             ng-show=`($input.$dirty || $input.$touched || $input.$submitted) && $input.$invalid`
         )
-            if block
-                block
-
             if required
                 +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` })
+
+            if block
+                block

http://git-wip-us.apache.org/repos/asf/ignite/blob/7da18645/modules/web-console/frontend/app/primitives/form-field/dropdown.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/form-field/dropdown.pug b/modules/web-console/frontend/app/primitives/form-field/dropdown.pug
index 96d8482..b13a177 100644
--- a/modules/web-console/frontend/app/primitives/form-field/dropdown.pug
+++ b/modules/web-console/frontend/app/primitives/form-field/dropdown.pug
@@ -50,8 +50,8 @@ mixin form-field__dropdown({ label, model, name, disabled, required, multiple, p
             ng-messages=`$input.$error`
             ng-show=`($input.$dirty || $input.$touched || $input.$submitted) && $input.$invalid`
         )
-            if block
-                block
-
             if required
                 +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` })
+
+            if block
+                block

http://git-wip-us.apache.org/repos/asf/ignite/blob/7da18645/modules/web-console/frontend/app/primitives/form-field/email.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/form-field/email.pug b/modules/web-console/frontend/app/primitives/form-field/email.pug
index 5fb0c30..b1e1202 100644
--- a/modules/web-console/frontend/app/primitives/form-field/email.pug
+++ b/modules/web-console/frontend/app/primitives/form-field/email.pug
@@ -29,10 +29,10 @@ mixin form-field__email({ label, model, name, disabled, required, placeholder, t
             ng-messages=`$input.$error`
             ng-show=`($input.$dirty || $input.$touched || $input.$submitted) && $input.$invalid`
         )
-            if block
-                block
-
             if required
                 +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` })
 
             +form-field__error({ error: 'email', message: `${errLbl} has invalid format!` })
+
+            if block
+                block

http://git-wip-us.apache.org/repos/asf/ignite/blob/7da18645/modules/web-console/frontend/app/primitives/form-field/number.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/form-field/number.pug b/modules/web-console/frontend/app/primitives/form-field/number.pug
index ea90709..dfcfff3 100644
--- a/modules/web-console/frontend/app/primitives/form-field/number.pug
+++ b/modules/web-console/frontend/app/primitives/form-field/number.pug
@@ -33,9 +33,6 @@ mixin form-field__number({ label, model, name, disabled, required, placeholder,
             ng-messages=`$input.$error`
             ng-show=`($input.$dirty || $input.$touched || $input.$submitted) && $input.$invalid`
         )
-            if block
-                block
-
             if required
                 +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` })
 
@@ -46,3 +43,6 @@ mixin form-field__number({ label, model, name, disabled, required, placeholder,
             +form-field__error({ error: 'step', message: `${errLbl} step should be ${step || 1}` })
 
             +form-field__error({ error: 'number', message: 'Only numbers allowed' })
+
+            if block
+                block

http://git-wip-us.apache.org/repos/asf/ignite/blob/7da18645/modules/web-console/frontend/app/primitives/form-field/password.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/form-field/password.pug b/modules/web-console/frontend/app/primitives/form-field/password.pug
index 374d768..5103281 100644
--- a/modules/web-console/frontend/app/primitives/form-field/password.pug
+++ b/modules/web-console/frontend/app/primitives/form-field/password.pug
@@ -44,10 +44,10 @@ mixin form-field__password({ label, model, name, disabled, required, placeholder
                 ($input2.$dirty || $input2.$touched || $input2.$submitted) && $input2.$invalid
             `
         )
-            if block
-                block
-
             if required
                 +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` })
-                
+
             +form-field__error({ error: 'mismatch', message: `Password does not match the confirm password!` })
+
+            if block
+                block

http://git-wip-us.apache.org/repos/asf/ignite/blob/7da18645/modules/web-console/frontend/app/primitives/form-field/phone.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/form-field/phone.pug b/modules/web-console/frontend/app/primitives/form-field/phone.pug
index 9b60a40..4c79477 100644
--- a/modules/web-console/frontend/app/primitives/form-field/phone.pug
+++ b/modules/web-console/frontend/app/primitives/form-field/phone.pug
@@ -29,8 +29,9 @@ mixin form-field__phone({ label, model, name, disabled, required, optional, plac
             ng-messages=`$input.$error`
             ng-show=`($input.$dirty || $input.$touched || $input.$submitted) && $input.$invalid`
         )
+            if required
+                +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` })
+
             if block
                 block
 
-            if required
-                +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` })

http://git-wip-us.apache.org/repos/asf/ignite/blob/7da18645/modules/web-console/frontend/app/primitives/form-field/text.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/form-field/text.pug b/modules/web-console/frontend/app/primitives/form-field/text.pug
index ab8d14d..5a2595e 100644
--- a/modules/web-console/frontend/app/primitives/form-field/text.pug
+++ b/modules/web-console/frontend/app/primitives/form-field/text.pug
@@ -29,8 +29,8 @@ mixin form-field__text({ label, model, name, disabled, required, placeholder, ti
             ng-messages=`$input.$error`
             ng-show=`($input.$dirty || $input.$touched || $input.$submitted) && $input.$invalid`
         )
-            if block
-                block
-
             if required
                 +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` })
+
+            if block
+                block

http://git-wip-us.apache.org/repos/asf/ignite/blob/7da18645/modules/web-console/frontend/app/primitives/form-field/typeahead.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/form-field/typeahead.pug b/modules/web-console/frontend/app/primitives/form-field/typeahead.pug
index b2c62ae..ee2312f 100644
--- a/modules/web-console/frontend/app/primitives/form-field/typeahead.pug
+++ b/modules/web-console/frontend/app/primitives/form-field/typeahead.pug
@@ -48,8 +48,8 @@ mixin form-field__typeahead({ label, model, name, disabled, required, placeholde
             ng-messages=`$input.$error`
             ng-show=`($input.$dirty || $input.$touched || $input.$submitted) && $input.$invalid`
         )
-            if block
-                block
-
             if required
                 +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` })
+
+            if block
+                block

http://git-wip-us.apache.org/repos/asf/ignite/blob/7da18645/modules/web-console/frontend/app/primitives/timepicker/index.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/timepicker/index.pug b/modules/web-console/frontend/app/primitives/timepicker/index.pug
index f97a5a2..9a4dd97 100644
--- a/modules/web-console/frontend/app/primitives/timepicker/index.pug
+++ b/modules/web-console/frontend/app/primitives/timepicker/index.pug
@@ -56,8 +56,8 @@ mixin form-field__timepicker({ label, model, name, mindate, maxdate, disabled, r
             ng-messages=`$input.$error`
             ng-show=`($input.$dirty || $input.$touched || $input.$submitted) && $input.$invalid`
         )
-            if block
-                block
-
             if required
                 +form-field__error({ error: 'required', message: `${errLbl} could not be empty!` })
+
+            if block
+                block


[23/50] [abbrv] ignite git commit: IGNITE-9558 Avoid blocking transactions on client connect when possible - Fixes #4933.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetResponse.java
index 2cb75c2..584cec2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetResponse.java
@@ -206,32 +206,32 @@ public class GridNearSingleGetResponse extends GridCacheIdMessage implements Gri
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeMessage("res", res))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
-                if (!writer.writeMessage("topVer", topVer))
+            case 8:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -252,7 +252,7 @@ public class GridNearSingleGetResponse extends GridCacheIdMessage implements Gri
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -260,7 +260,7 @@ public class GridNearSingleGetResponse extends GridCacheIdMessage implements Gri
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -268,7 +268,7 @@ public class GridNearSingleGetResponse extends GridCacheIdMessage implements Gri
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -276,7 +276,7 @@ public class GridNearSingleGetResponse extends GridCacheIdMessage implements Gri
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 res = reader.readMessage("res");
 
                 if (!reader.isLastRead())
@@ -284,8 +284,8 @@ public class GridNearSingleGetResponse extends GridCacheIdMessage implements Gri
 
                 reader.incrementState();
 
-            case 7:
-                topVer = reader.readMessage("topVer");
+            case 8:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -309,7 +309,7 @@ public class GridNearSingleGetResponse extends GridCacheIdMessage implements Gri
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 9;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistRequest.java
index e71de89..dd86861 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistRequest.java
@@ -388,97 +388,97 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeBoolean("clientFirst", clientFirst))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeMessage("filter", filter))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeObjectArray("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeMessage("lockVer", lockVer))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeBoolean("needRes", needRes))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeLong("threadId", threadId))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
-                if (!writer.writeMessage("topVer", topVer))
+            case 17:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 18:
                 if (!writer.writeLong("txTimeout", txTimeout))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
+            case 19:
                 if (!writer.writeObjectArray("values", values, MessageCollectionItemType.MSG))
                     return false;
 
@@ -500,7 +500,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 clientFirst = reader.readBoolean("clientFirst");
 
                 if (!reader.isLastRead())
@@ -508,7 +508,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 filter = reader.readMessage("filter");
 
                 if (!reader.isLastRead())
@@ -516,7 +516,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -524,7 +524,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 keys = reader.readObjectArray("keys", MessageCollectionItemType.MSG, KeyCacheObject.class);
 
                 if (!reader.isLastRead())
@@ -532,7 +532,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 lockVer = reader.readMessage("lockVer");
 
                 if (!reader.isLastRead())
@@ -540,7 +540,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -548,7 +548,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 mvccSnapshot = reader.readMessage("mvccSnapshot");
 
                 if (!reader.isLastRead())
@@ -556,7 +556,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 needRes = reader.readBoolean("needRes");
 
                 if (!reader.isLastRead())
@@ -564,7 +564,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 byte opOrd;
 
                 opOrd = reader.readByte("op");
@@ -576,7 +576,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -584,7 +584,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -592,7 +592,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 threadId = reader.readLong("threadId");
 
                 if (!reader.isLastRead())
@@ -600,7 +600,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
@@ -608,15 +608,15 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 16:
-                topVer = reader.readMessage("topVer");
+            case 17:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 17:
+            case 18:
                 txTimeout = reader.readLong("txTimeout");
 
                 if (!reader.isLastRead())
@@ -624,7 +624,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 18:
+            case 19:
                 values = reader.readObjectArray("values", MessageCollectionItemType.MSG, Message.class);
 
                 if (!reader.isLastRead())
@@ -639,7 +639,7 @@ public class GridNearTxEnlistRequest extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 19;
+        return 20;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistResponse.java
index 78333a4..cf00186 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistResponse.java
@@ -190,7 +190,7 @@ public class GridNearTxEnlistResponse extends GridCacheIdMessage implements Exce
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 11;
+        return 12;
     }
 
     /** {@inheritDoc} */
@@ -208,49 +208,49 @@ public class GridNearTxEnlistResponse extends GridCacheIdMessage implements Exce
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeIgniteUuid("dhtFutId", dhtFutId))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeMessage("dhtVer", dhtVer))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeMessage("lockVer", lockVer))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeCollection("newDhtNodes", newDhtNodes, MessageCollectionItemType.UUID))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeMessage("res", res))
                     return false;
 
@@ -272,7 +272,7 @@ public class GridNearTxEnlistResponse extends GridCacheIdMessage implements Exce
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 dhtFutId = reader.readIgniteUuid("dhtFutId");
 
                 if (!reader.isLastRead())
@@ -280,7 +280,7 @@ public class GridNearTxEnlistResponse extends GridCacheIdMessage implements Exce
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 dhtVer = reader.readMessage("dhtVer");
 
                 if (!reader.isLastRead())
@@ -288,7 +288,7 @@ public class GridNearTxEnlistResponse extends GridCacheIdMessage implements Exce
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -296,7 +296,7 @@ public class GridNearTxEnlistResponse extends GridCacheIdMessage implements Exce
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -304,7 +304,7 @@ public class GridNearTxEnlistResponse extends GridCacheIdMessage implements Exce
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 lockVer = reader.readMessage("lockVer");
 
                 if (!reader.isLastRead())
@@ -312,7 +312,7 @@ public class GridNearTxEnlistResponse extends GridCacheIdMessage implements Exce
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -320,7 +320,7 @@ public class GridNearTxEnlistResponse extends GridCacheIdMessage implements Exce
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 newDhtNodes = reader.readCollection("newDhtNodes", MessageCollectionItemType.UUID);
 
                 if (!reader.isLastRead())
@@ -328,7 +328,7 @@ public class GridNearTxEnlistResponse extends GridCacheIdMessage implements Exce
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 res = reader.readMessage("res");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
index 6b5aa90..91079df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
@@ -186,13 +186,13 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
         }
 
         switch (writer.state()) {
-            case 21:
+            case 22:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 22:
+            case 23:
                 if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
                     return false;
 
@@ -214,7 +214,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
             return false;
 
         switch (reader.state()) {
-            case 21:
+            case 22:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -222,7 +222,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 22:
+            case 23:
                 mvccSnapshot = reader.readMessage("mvccSnapshot");
 
                 if (!reader.isLastRead())
@@ -242,7 +242,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 23;
+        return 24;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java
index a1a2b57..e3dcbf8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java
@@ -133,19 +133,19 @@ public class GridNearTxFinishResponse extends GridDistributedTxFinishResponse {
         }
 
         switch (writer.state()) {
-            case 6:
+            case 7:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeLong("nearThreadId", nearThreadId))
                     return false;
 
@@ -167,7 +167,7 @@ public class GridNearTxFinishResponse extends GridDistributedTxFinishResponse {
             return false;
 
         switch (reader.state()) {
-            case 6:
+            case 7:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -175,7 +175,7 @@ public class GridNearTxFinishResponse extends GridDistributedTxFinishResponse {
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -183,7 +183,7 @@ public class GridNearTxFinishResponse extends GridDistributedTxFinishResponse {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 nearThreadId = reader.readLong("nearThreadId");
 
                 if (!reader.isLastRead())
@@ -203,7 +203,7 @@ public class GridNearTxFinishResponse extends GridDistributedTxFinishResponse {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 9;
+        return 10;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
index d740737..c3eae2c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
@@ -332,43 +332,43 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
         }
 
         switch (writer.state()) {
-            case 20:
+            case 21:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 21:
+            case 22:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 22:
+            case 23:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 23:
+            case 24:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 25:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 25:
-                if (!writer.writeMessage("topVer", topVer))
+            case 26:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 26:
+            case 27:
                 if (!writer.writeString("txLbl", txLbl))
                     return false;
 
@@ -390,7 +390,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
             return false;
 
         switch (reader.state()) {
-            case 20:
+            case 21:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -398,7 +398,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 21:
+            case 22:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -406,7 +406,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 22:
+            case 23:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -414,7 +414,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 23:
+            case 24:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -422,7 +422,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 24:
+            case 25:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -430,15 +430,15 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 25:
-                topVer = reader.readMessage("topVer");
+            case 26:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 26:
+            case 27:
                 txLbl = reader.readString("txLbl");
 
                 if (!reader.isLastRead())
@@ -458,7 +458,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 27;
+        return 28;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
index e9865df..67f7116 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
@@ -376,68 +376,68 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
         }
 
         switch (writer.state()) {
-            case 10:
-                if (!writer.writeMessage("clientRemapVer", clientRemapVer))
-                    return false;
-
-                writer.incrementState();
-
             case 11:
-                if (!writer.writeMessage("dhtVer", dhtVer))
+                if (!writer.writeAffinityTopologyVersion("clientRemapVer", clientRemapVer))
                     return false;
 
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeCollection("filterFailedKeys", filterFailedKeys, MessageCollectionItemType.MSG))
+                if (!writer.writeMessage("dhtVer", dhtVer))
                     return false;
 
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeIgniteUuid("futId", futId))
+                if (!writer.writeCollection("filterFailedKeys", filterFailedKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 14:
-                if (!writer.writeInt("miniId", miniId))
+                if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
             case 15:
-                if (!writer.writeCollection("ownedValKeys", ownedValKeys, MessageCollectionItemType.MSG))
+                if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
             case 16:
-                if (!writer.writeCollection("ownedValVals", ownedValVals, MessageCollectionItemType.MSG))
+                if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
                     return false;
 
                 writer.incrementState();
 
             case 17:
-                if (!writer.writeCollection("pending", pending, MessageCollectionItemType.MSG))
+                if (!writer.writeCollection("ownedValKeys", ownedValKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 18:
-                if (!writer.writeMessage("retVal", retVal))
+                if (!writer.writeCollection("ownedValVals", ownedValVals, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 19:
-                if (!writer.writeMessage("writeVer", writeVer))
+                if (!writer.writeCollection("pending", pending, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 20:
-                if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
+                if (!writer.writeMessage("retVal", retVal))
+                    return false;
+
+                writer.incrementState();
+
+            case 21:
+                if (!writer.writeMessage("writeVer", writeVer))
                     return false;
 
                 writer.incrementState();
@@ -458,16 +458,8 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
             return false;
 
         switch (reader.state()) {
-            case 10:
-                clientRemapVer = reader.readMessage("clientRemapVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
             case 11:
-                dhtVer = reader.readMessage("dhtVer");
+                clientRemapVer = reader.readAffinityTopologyVersion("clientRemapVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -475,7 +467,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 12:
-                filterFailedKeys = reader.readCollection("filterFailedKeys", MessageCollectionItemType.MSG);
+                dhtVer = reader.readMessage("dhtVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -483,7 +475,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 13:
-                futId = reader.readIgniteUuid("futId");
+                filterFailedKeys = reader.readCollection("filterFailedKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -491,7 +483,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 14:
-                miniId = reader.readInt("miniId");
+                futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -499,7 +491,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 15:
-                ownedValKeys = reader.readCollection("ownedValKeys", MessageCollectionItemType.MSG);
+                miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -507,7 +499,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 16:
-                ownedValVals = reader.readCollection("ownedValVals", MessageCollectionItemType.MSG);
+                mvccSnapshot = reader.readMessage("mvccSnapshot");
 
                 if (!reader.isLastRead())
                     return false;
@@ -515,7 +507,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 17:
-                pending = reader.readCollection("pending", MessageCollectionItemType.MSG);
+                ownedValKeys = reader.readCollection("ownedValKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -523,7 +515,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 18:
-                retVal = reader.readMessage("retVal");
+                ownedValVals = reader.readCollection("ownedValVals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -531,7 +523,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 19:
-                writeVer = reader.readMessage("writeVer");
+                pending = reader.readCollection("pending", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -539,7 +531,15 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 20:
-                mvccSnapshot = reader.readMessage("mvccSnapshot");
+                retVal = reader.readMessage("retVal");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 21:
+                writeVer = reader.readMessage("writeVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -558,7 +558,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 21;
+        return 22;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistRequest.java
index 472937b..3b22afb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistRequest.java
@@ -297,7 +297,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 21;
+        return 22;
     }
 
     /** {@inheritDoc} */
@@ -331,109 +331,109 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeIntArray("cacheIds", cacheIds))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeBoolean("clientFirst", clientFirst))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeInt("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeMessage("lockVer", lockVer))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeInt("pageSize", pageSize))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeByteArray("paramsBytes", paramsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeIntArray("parts", parts))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeString("qry", qry))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeString("schema", schema))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 17:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 18:
                 if (!writer.writeLong("threadId", threadId))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
+            case 19:
                 if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
-            case 19:
-                if (!writer.writeMessage("topVer", topVer))
+            case 20:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 20:
+            case 21:
                 if (!writer.writeLong("txTimeout", txTimeout))
                     return false;
 
@@ -455,7 +455,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 cacheIds = reader.readIntArray("cacheIds");
 
                 if (!reader.isLastRead())
@@ -463,7 +463,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 clientFirst = reader.readBoolean("clientFirst");
 
                 if (!reader.isLastRead())
@@ -471,7 +471,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 flags = reader.readInt("flags");
 
                 if (!reader.isLastRead())
@@ -479,7 +479,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -487,7 +487,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 lockVer = reader.readMessage("lockVer");
 
                 if (!reader.isLastRead())
@@ -495,7 +495,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -503,7 +503,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 mvccSnapshot = reader.readMessage("mvccSnapshot");
 
                 if (!reader.isLastRead())
@@ -511,7 +511,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 pageSize = reader.readInt("pageSize");
 
                 if (!reader.isLastRead())
@@ -519,7 +519,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 paramsBytes = reader.readByteArray("paramsBytes");
 
                 if (!reader.isLastRead())
@@ -527,7 +527,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 parts = reader.readIntArray("parts");
 
                 if (!reader.isLastRead())
@@ -535,7 +535,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 qry = reader.readString("qry");
 
                 if (!reader.isLastRead())
@@ -543,7 +543,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 schema = reader.readString("schema");
 
                 if (!reader.isLastRead())
@@ -551,7 +551,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -559,7 +559,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 16:
+            case 17:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -567,7 +567,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 17:
+            case 18:
                 threadId = reader.readLong("threadId");
 
                 if (!reader.isLastRead())
@@ -575,7 +575,7 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 18:
+            case 19:
                 timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
@@ -583,15 +583,15 @@ public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 19:
-                topVer = reader.readMessage("topVer");
+            case 20:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 20:
+            case 21:
                 txTimeout = reader.readLong("txTimeout");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistResponse.java
index d628de1..2715f89 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistResponse.java
@@ -166,7 +166,7 @@ public class GridNearTxQueryEnlistResponse extends GridCacheIdMessage implements
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 10;
+        return 11;
     }
 
     /** {@inheritDoc} */
@@ -184,47 +184,48 @@ public class GridNearTxQueryEnlistResponse extends GridCacheIdMessage implements
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeMessage("lockVer", lockVer))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
-                if (!writer.writeBoolean("removeMapping", removeMapping))
+            case 8:
+                if (!writer.writeCollection("newDhtNodes", newDhtNodes, MessageCollectionItemType.UUID))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
-                if (!writer.writeLong("res", res))
+            case 9:
+                if (!writer.writeBoolean("removeMapping", removeMapping))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
-                if (!writer.writeCollection("newDhtNodes", newDhtNodes, MessageCollectionItemType.UUID))
+            case 10:
+                if (!writer.writeLong("res", res))
                     return false;
 
                 writer.incrementState();
+
         }
 
         return true;
@@ -241,7 +242,7 @@ public class GridNearTxQueryEnlistResponse extends GridCacheIdMessage implements
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -249,7 +250,7 @@ public class GridNearTxQueryEnlistResponse extends GridCacheIdMessage implements
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -257,7 +258,7 @@ public class GridNearTxQueryEnlistResponse extends GridCacheIdMessage implements
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 lockVer = reader.readMessage("lockVer");
 
                 if (!reader.isLastRead())
@@ -265,7 +266,7 @@ public class GridNearTxQueryEnlistResponse extends GridCacheIdMessage implements
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -273,29 +274,30 @@ public class GridNearTxQueryEnlistResponse extends GridCacheIdMessage implements
 
                 reader.incrementState();
 
-            case 7:
-                removeMapping = reader.readBoolean("removeMapping");
+            case 8:
+                newDhtNodes = reader.readCollection("newDhtNodes", MessageCollectionItemType.UUID);
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 8:
-                res = reader.readLong("res");
+            case 9:
+                removeMapping = reader.readBoolean("removeMapping");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 9:
-                newDhtNodes = reader.readCollection("newDhtNodes", MessageCollectionItemType.UUID);
+            case 10:
+                res = reader.readLong("res");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
+
         }
 
         return reader.afterMessageRead(GridNearTxQueryEnlistResponse.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistRequest.java
index f350d50..8e10a7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistRequest.java
@@ -335,85 +335,85 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeBoolean("clientFirst", clientFirst))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeObjectArray("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeMessage("lockVer", lockVer))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeLong("threadId", threadId))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
-                if (!writer.writeMessage("topVer", topVer))
+            case 15:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeLong("txTimeout", txTimeout))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 17:
                 if (!writer.writeObjectArray("values", values, MessageCollectionItemType.MSG))
                     return false;
 
@@ -435,7 +435,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 clientFirst = reader.readBoolean("clientFirst");
 
                 if (!reader.isLastRead())
@@ -443,7 +443,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -451,7 +451,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 keys = reader.readObjectArray("keys", MessageCollectionItemType.MSG, KeyCacheObject.class);
 
                 if (!reader.isLastRead())
@@ -459,7 +459,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 lockVer = reader.readMessage("lockVer");
 
                 if (!reader.isLastRead())
@@ -467,7 +467,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -475,7 +475,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 mvccSnapshot = reader.readMessage("mvccSnapshot");
 
                 if (!reader.isLastRead())
@@ -483,7 +483,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 byte opOrd;
 
                 opOrd = reader.readByte("op");
@@ -495,7 +495,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -503,7 +503,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -511,7 +511,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 threadId = reader.readLong("threadId");
 
                 if (!reader.isLastRead())
@@ -519,7 +519,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
@@ -527,15 +527,15 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 14:
-                topVer = reader.readMessage("topVer");
+            case 15:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 txTimeout = reader.readLong("txTimeout");
 
                 if (!reader.isLastRead())
@@ -543,7 +543,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 16:
+            case 17:
                 values = reader.readObjectArray("values", MessageCollectionItemType.MSG, CacheObject.class);
 
                 if (!reader.isLastRead())
@@ -558,7 +558,7 @@ public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 17;
+        return 18;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistResponse.java
index 48c63bc..2a0c632 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistResponse.java
@@ -102,7 +102,7 @@ public class GridNearTxQueryResultsEnlistResponse extends GridNearTxQueryEnlistR
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 12;
+        return 13;
     }
 
     /** {@inheritDoc} */
@@ -120,17 +120,18 @@ public class GridNearTxQueryResultsEnlistResponse extends GridNearTxQueryEnlistR
         }
 
         switch (writer.state()) {
-            case 10:
+            case 11:
                 if (!writer.writeIgniteUuid("dhtFutId", dhtFutId))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeMessage("dhtVer", dhtVer))
                     return false;
 
                 writer.incrementState();
+
         }
 
         return true;
@@ -147,7 +148,7 @@ public class GridNearTxQueryResultsEnlistResponse extends GridNearTxQueryEnlistR
             return false;
 
         switch (reader.state()) {
-            case 10:
+            case 11:
                 dhtFutId = reader.readIgniteUuid("dhtFutId");
 
                 if (!reader.isLastRead())
@@ -155,13 +156,14 @@ public class GridNearTxQueryResultsEnlistResponse extends GridNearTxQueryEnlistR
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 dhtVer = reader.readMessage("dhtVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
+
         }
 
         return reader.afterMessageRead(GridNearTxQueryResultsEnlistResponse.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearUnlockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearUnlockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearUnlockRequest.java
index 14b5438..3b2ccb4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearUnlockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearUnlockRequest.java
@@ -84,7 +84,7 @@ public class GridNearUnlockRequest extends GridDistributedUnlockRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 9;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryId.java
index 89f09db..f3b3150 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryId.java
@@ -76,6 +76,7 @@ public class MvccAckRequestTxAndQueryId extends MvccAckRequestTx {
                     return false;
 
                 writer.incrementState();
+
         }
 
         return true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java
index c5a8494..73d3f94 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.mvcc.msg;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccLongList;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
@@ -33,6 +34,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 /**
  *
  */
+@IgniteCodeGeneratingFail
 public class MvccSnapshotResponse implements MvccMessage, MvccSnapshot, MvccLongList {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccTxSnapshotRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccTxSnapshotRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccTxSnapshotRequest.java
index cd30eb8..4cf6f65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccTxSnapshotRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccTxSnapshotRequest.java
@@ -81,6 +81,7 @@ public class MvccTxSnapshotRequest implements MvccMessage {
                     return false;
 
                 writer.incrementState();
+
         }
 
         return true;
@@ -101,6 +102,7 @@ public class MvccTxSnapshotRequest implements MvccMessage {
                     return false;
 
                 reader.incrementState();
+
         }
 
         return reader.afterMessageRead(MvccTxSnapshotRequest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastRequest.java
index ffd9a67..5c46ca6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastRequest.java
@@ -79,13 +79,13 @@ public class PartitionCountersNeighborcastRequest extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeCollection("updCntrs", updCntrs, MessageCollectionItemType.MSG))
                     return false;
 
@@ -107,7 +107,7 @@ public class PartitionCountersNeighborcastRequest extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -115,7 +115,7 @@ public class PartitionCountersNeighborcastRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 updCntrs = reader.readCollection("updCntrs", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -135,7 +135,7 @@ public class PartitionCountersNeighborcastRequest extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 5;
+        return 6;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastResponse.java
index 547539d..2f88cde 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastResponse.java
@@ -62,7 +62,7 @@ public class PartitionCountersNeighborcastResponse extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
@@ -84,7 +84,7 @@ public class PartitionCountersNeighborcastResponse extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -104,7 +104,7 @@ public class PartitionCountersNeighborcastResponse extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 4;
+        return 5;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
index b7205b6..21c6363 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
@@ -513,128 +513,128 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
         }
 
         switch (writer.state()) {
-            case 3:
-                if (!writer.writeBoolean("all", all))
-                    return false;
-
-                writer.incrementState();
-
             case 4:
-                if (!writer.writeByteArray("argsBytes", argsBytes))
+                if (!writer.writeBoolean("all", all))
                     return false;
 
                 writer.incrementState();
 
             case 5:
-                if (!writer.writeString("cacheName", cacheName))
+                if (!writer.writeByteArray("argsBytes", argsBytes))
                     return false;
 
                 writer.incrementState();
 
             case 6:
-                if (!writer.writeBoolean("cancel", cancel))
+                if (!writer.writeString("cacheName", cacheName))
                     return false;
 
                 writer.incrementState();
 
             case 7:
-                if (!writer.writeString("clause", clause))
+                if (!writer.writeBoolean("cancel", cancel))
                     return false;
 
                 writer.incrementState();
 
             case 8:
-                if (!writer.writeString("clsName", clsName))
+                if (!writer.writeString("clause", clause))
                     return false;
 
                 writer.incrementState();
 
             case 9:
-                if (!writer.writeBoolean("fields", fields))
+                if (!writer.writeString("clsName", clsName))
                     return false;
 
                 writer.incrementState();
 
             case 10:
-                if (!writer.writeLong("id", id))
+                if (!writer.writeBoolean("fields", fields))
                     return false;
 
                 writer.incrementState();
 
             case 11:
-                if (!writer.writeBoolean("incBackups", incBackups))
+                if (!writer.writeLong("id", id))
                     return false;
 
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeBoolean("incMeta", incMeta))
+                if (!writer.writeBoolean("incBackups", incBackups))
                     return false;
 
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeBoolean("keepBinary", keepBinary))
+                if (!writer.writeBoolean("incMeta", incMeta))
                     return false;
 
                 writer.incrementState();
 
             case 14:
-                if (!writer.writeByteArray("keyValFilterBytes", keyValFilterBytes))
+                if (!writer.writeBoolean("keepBinary", keepBinary))
                     return false;
 
                 writer.incrementState();
 
             case 15:
-                if (!writer.writeInt("pageSize", pageSize))
+                if (!writer.writeByteArray("keyValFilterBytes", keyValFilterBytes))
                     return false;
 
                 writer.incrementState();
 
             case 16:
-                if (!writer.writeInt("part", part))
+                if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
                     return false;
 
                 writer.incrementState();
 
             case 17:
-                if (!writer.writeByteArray("rdcBytes", rdcBytes))
+                if (!writer.writeInt("pageSize", pageSize))
                     return false;
 
                 writer.incrementState();
 
             case 18:
-                if (!writer.writeUuid("subjId", subjId))
+                if (!writer.writeInt("part", part))
                     return false;
 
                 writer.incrementState();
 
             case 19:
-                if (!writer.writeInt("taskHash", taskHash))
+                if (!writer.writeByteArray("rdcBytes", rdcBytes))
                     return false;
 
                 writer.incrementState();
 
             case 20:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
             case 21:
-                if (!writer.writeByteArray("transBytes", transBytes))
+                if (!writer.writeInt("taskHash", taskHash))
                     return false;
 
                 writer.incrementState();
 
             case 22:
-                if (!writer.writeByte("type", type != null ? (byte)type.ordinal() : -1))
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
             case 23:
-                if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
+                if (!writer.writeByteArray("transBytes", transBytes))
+                    return false;
+
+                writer.incrementState();
+
+            case 24:
+                if (!writer.writeByte("type", type != null ? (byte)type.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
@@ -655,7 +655,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 all = reader.readBoolean("all");
 
                 if (!reader.isLastRead())
@@ -663,7 +663,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 argsBytes = reader.readByteArray("argsBytes");
 
                 if (!reader.isLastRead())
@@ -671,7 +671,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 cacheName = reader.readString("cacheName");
 
                 if (!reader.isLastRead())
@@ -679,7 +679,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 cancel = reader.readBoolean("cancel");
 
                 if (!reader.isLastRead())
@@ -687,7 +687,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 clause = reader.readString("clause");
 
                 if (!reader.isLastRead())
@@ -695,7 +695,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 clsName = reader.readString("clsName");
 
                 if (!reader.isLastRead())
@@ -703,7 +703,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 fields = reader.readBoolean("fields");
 
                 if (!reader.isLastRead())
@@ -711,7 +711,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 id = reader.readLong("id");
 
                 if (!reader.isLastRead())
@@ -719,7 +719,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 incBackups = reader.readBoolean("incBackups");
 
                 if (!reader.isLastRead())
@@ -727,7 +727,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 incMeta = reader.readBoolean("incMeta");
 
                 if (!reader.isLastRead())
@@ -735,7 +735,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 keepBinary = reader.readBoolean("keepBinary");
 
                 if (!reader.isLastRead())
@@ -743,7 +743,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 keyValFilterBytes = reader.readByteArray("keyValFilterBytes");
 
                 if (!reader.isLastRead())
@@ -751,7 +751,15 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
+                mvccSnapshot = reader.readMessage("mvccSnapshot");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 17:
                 pageSize = reader.readInt("pageSize");
 
                 if (!reader.isLastRead())
@@ -759,7 +767,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 16:
+            case 18:
                 part = reader.readInt("part");
 
                 if (!reader.isLastRead())
@@ -767,7 +775,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 17:
+            case 19:
                 rdcBytes = reader.readByteArray("rdcBytes");
 
                 if (!reader.isLastRead())
@@ -775,7 +783,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 18:
+            case 20:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -783,7 +791,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 19:
+            case 21:
                 taskHash = reader.readInt("taskHash");
 
                 if (!reader.isLastRead())
@@ -791,15 +799,15 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 20:
-                topVer = reader.readMessage("topVer");
+            case 22:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 21:
+            case 23:
                 transBytes = reader.readByteArray("transBytes");
 
                 if (!reader.isLastRead())
@@ -807,7 +815,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 22:
+            case 24:
                 byte typeOrd;
 
                 typeOrd = reader.readByte("type");
@@ -819,14 +827,6 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
                 reader.incrementState();
 
-            case 23:
-                mvccSnapshot = reader.readMessage("mvccSnapshot");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
         }
 
         return reader.afterMessageRead(GridCacheQueryRequest.class);
@@ -839,7 +839,7 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 24;
+        return 25;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
index 13e0915..a1650be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
@@ -287,37 +287,37 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeCollection("dataBytes", dataBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeBoolean("fields", fields))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeBoolean("finished", finished))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeCollection("metaDataBytes", metaDataBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeLong("reqId", reqId))
                     return false;
 
@@ -339,7 +339,7 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 dataBytes = reader.readCollection("dataBytes", MessageCollectionItemType.BYTE_ARR);
 
                 if (!reader.isLastRead())
@@ -347,7 +347,7 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -355,7 +355,7 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 fields = reader.readBoolean("fields");
 
                 if (!reader.isLastRead())
@@ -363,7 +363,7 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 finished = reader.readBoolean("finished");
 
                 if (!reader.isLastRead())
@@ -371,7 +371,7 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 metaDataBytes = reader.readCollection("metaDataBytes", MessageCollectionItemType.BYTE_ARR);
 
                 if (!reader.isLastRead())
@@ -379,7 +379,7 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 reqId = reader.readLong("reqId");
 
                 if (!reader.isLastRead())
@@ -399,7 +399,7 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 9;
+        return 10;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryBatchAck.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryBatchAck.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryBatchAck.java
index ef0157e..9ba9ad2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryBatchAck.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryBatchAck.java
@@ -90,15 +90,14 @@ public class CacheContinuousQueryBatchAck extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeUuid("routineId", routineId))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
-                if (!writer.writeMap("updateCntrs", updateCntrs, MessageCollectionItemType.INT,
-                    MessageCollectionItemType.LONG))
+            case 5:
+                if (!writer.writeMap("updateCntrs", updateCntrs, MessageCollectionItemType.INT, MessageCollectionItemType.LONG))
                     return false;
 
                 writer.incrementState();
@@ -119,7 +118,7 @@ public class CacheContinuousQueryBatchAck extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 routineId = reader.readUuid("routineId");
 
                 if (!reader.isLastRead())
@@ -127,9 +126,8 @@ public class CacheContinuousQueryBatchAck extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 4:
-                updateCntrs = reader.readMap("updateCntrs", MessageCollectionItemType.INT,
-                    MessageCollectionItemType.LONG, false);
+            case 5:
+                updateCntrs = reader.readMap("updateCntrs", MessageCollectionItemType.INT, MessageCollectionItemType.LONG, false);
 
                 if (!reader.isLastRead())
                     return false;
@@ -153,7 +151,7 @@ public class CacheContinuousQueryBatchAck extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 5;
+        return 6;
     }
 
     /** {@inheritDoc} */


[21/50] [abbrv] ignite git commit: IGNITE-10197 Fixed unexpected IllegalArgumentException in IgniteDbPutGetAbstractTest#testRandomPutGetRemove - Fixes #5410.

Posted by sb...@apache.org.
IGNITE-10197 Fixed unexpected IllegalArgumentException in IgniteDbPutGetAbstractTest#testRandomPutGetRemove - Fixes #5410.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


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

Branch: refs/heads/ignite-10044
Commit: 9e3bd7dd686d3392e6bbe2c7defebe260eb029f7
Parents: c9906aa
Author: ibessonov <be...@gmail.com>
Authored: Mon Nov 19 13:06:47 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Nov 19 13:06:47 2018 +0300

----------------------------------------------------------------------
 .../database/IgniteDbPutGetAbstractTest.java    | 28 +++++++------
 .../IgniteDbSingleNodeTinyPutGetTest.java       | 42 ++++++++++----------
 2 files changed, 35 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9e3bd7dd/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetAbstractTest.java
index 0b0c763..f170280 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetAbstractTest.java
@@ -44,6 +44,7 @@ import org.apache.ignite.internal.util.GridRandom;
 import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.GridTestUtils.SF;
 import org.junit.Assert;
 
 /**
@@ -810,22 +811,29 @@ public abstract class IgniteDbPutGetAbstractTest extends IgniteDbAbstractTest {
         }
     }
 
-    public void _testRandomPutGetRemove() throws Exception {
-        final IgniteCache<Integer, DbValue> cache = cache(null);
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomPutGetRemove() throws Exception {
+        final IgniteCache<Integer, DbValue> cache = cache(DEFAULT_CACHE_NAME);
 
         int cnt = KEYS_COUNT;
 
         Map<Integer, DbValue> map = new HashMap<>(cnt);
 
-        long seed = 1460943282308L; // System.currentTimeMillis();
+        long seed = System.currentTimeMillis();
+
+        int iterations = SF.apply(300_000);
 
-        X.println(" seed---> " + seed);
+        X.println("Seed: " + seed);
+
+        X.println("Iterations total: " + iterations);
 
         Random rnd = new GridRandom(seed);
 
-        for (int i = 0; i < 1000_000; i++) {
+        for (int i = 0; i < iterations; i++) {
             if (i % 5000 == 0)
-                X.println(" --> " + i);
+                X.println("Iteration #" + i);
 
             int key = rnd.nextInt(cnt);
 
@@ -833,27 +841,21 @@ public abstract class IgniteDbPutGetAbstractTest extends IgniteDbAbstractTest {
 
             switch (rnd.nextInt(3)) {
                 case 0:
-                    X.println("Put: " + key + " = " + v0);
-
                     assertEquals(map.put(key, v0), cache.getAndPut(key, v0));
 
                 case 1:
-                    X.println("Get: " + key);
-
                     assertEquals(map.get(key), cache.get(key));
 
                     break;
 
                 case 2:
-                    X.println("Rmv: " + key);
-
                     assertEquals(map.remove(key), cache.getAndRemove(key));
 
                     assertNull(cache.get(key));
             }
         }
 
-//        assertEquals(map.size(), cache.size(CachePeekMode.ALL));
+        assertEquals(map.size(), cache.size());
 
         for (Integer key : map.keySet())
             assertEquals(map.get(key), cache.get(key));

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e3bd7dd/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeTinyPutGetTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeTinyPutGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeTinyPutGetTest.java
index 53d299e..345eb86 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeTinyPutGetTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbSingleNodeTinyPutGetTest.java
@@ -29,10 +29,8 @@ public class IgniteDbSingleNodeTinyPutGetTest extends IgniteDbSingleNodePutGetTe
         return true;
     }
 
-    /**
-     * @throws Exception If fail.
-     */
-    public void testPutGetTiny() throws Exception {
+    /** */
+    public void testPutGetTiny() {
         IgniteEx ig = grid(0);
 
         IgniteCache<Short, Byte> cache = ig.cache("tiny");
@@ -60,92 +58,92 @@ public class IgniteDbSingleNodeTinyPutGetTest extends IgniteDbSingleNodePutGetTe
     }
 
     /** {@inheritDoc} */
-    @Override public void testPutGetSimple() throws Exception {
+    @Override public void testPutGetSimple() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testPutGetLarge() throws Exception {
+    @Override public void testPutGetLarge() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testPutGetOverwrite() throws Exception {
+    @Override public void testPutGetOverwrite() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testOverwriteNormalSizeAfterSmallerSize() throws Exception {
+    @Override public void testOverwriteNormalSizeAfterSmallerSize() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testPutDoesNotTriggerRead() throws Exception {
+    @Override public void testPutDoesNotTriggerRead() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testPutGetMultipleObjects() throws Exception {
+    @Override public void testPutGetMultipleObjects() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testSizeClear() throws Exception {
+    @Override public void testSizeClear() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testBounds() throws Exception {
+    @Override public void testBounds() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testMultithreadedPut() throws Exception {
+    @Override public void testMultithreadedPut() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testPutGetRandomUniqueMultipleObjects() throws Exception {
+    @Override public void testPutGetRandomUniqueMultipleObjects() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testPutPrimaryUniqueSecondaryDuplicates() throws Exception {
+    @Override public void testPutPrimaryUniqueSecondaryDuplicates() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testPutGetRandomNonUniqueMultipleObjects() throws Exception {
+    @Override public void testPutGetRandomNonUniqueMultipleObjects() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testPutGetRemoveMultipleForward() throws Exception {
+    @Override public void testPutGetRemoveMultipleForward() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void _testRandomPutGetRemove() {
+    @Override public void testRandomPutGetRemove() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testPutGetRemoveMultipleBackward() throws Exception {
+    @Override public void testPutGetRemoveMultipleBackward() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testIndexOverwrite() throws Exception {
+    @Override public void testIndexOverwrite() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testObjectKey() throws Exception {
+    @Override public void testObjectKey() {
         // No-op
     }
 
     /** {@inheritDoc} */
-    @Override public void testIterators() throws Exception {
+    @Override public void testIterators() {
         // No-op
     }
 }


[18/50] [abbrv] ignite git commit: IGNITE-10192 Fixed OptimizedMarshallerTest#testAllocationOverflow throws OOME instead of expected IOE - Fixes #5400.

Posted by sb...@apache.org.
IGNITE-10192 Fixed OptimizedMarshallerTest#testAllocationOverflow throws OOME instead of expected IOE - Fixes #5400.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


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

Branch: refs/heads/ignite-10044
Commit: 96e2fddc9b7c18ec11ab181ad45633f4041a9aa3
Parents: 355ce6f
Author: ibessonov <be...@gmail.com>
Authored: Mon Nov 19 12:41:05 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Nov 19 12:41:05 2018 +0300

----------------------------------------------------------------------
 .../internal/util/io/GridUnsafeDataOutput.java  | 76 ++++++++++++--------
 .../optimized/OptimizedMarshallerTest.java      | 56 ++++++---------
 2 files changed, 66 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/96e2fddc/modules/core/src/main/java/org/apache/ignite/internal/util/io/GridUnsafeDataOutput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/io/GridUnsafeDataOutput.java b/modules/core/src/main/java/org/apache/ignite/internal/util/io/GridUnsafeDataOutput.java
index ad94889..8b664ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/io/GridUnsafeDataOutput.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/io/GridUnsafeDataOutput.java
@@ -17,8 +17,10 @@
 
 package org.apache.ignite.internal.util.io;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.util.Arrays;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -37,8 +39,15 @@ import static org.apache.ignite.internal.util.GridUnsafe.SHORT_ARR_OFF;
  * Data output based on {@code Unsafe} operations.
  */
 public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput {
+    /**
+     * Based on {@link ByteArrayOutputStream#MAX_ARRAY_SIZE} or many other similar constants in other classes.
+     * It's not safe to allocate more then this number of elements in byte array, because it can throw
+     * java.lang.OutOfMemoryError: Requested array size exceeds VM limit
+     */
+    private static final int MAX_BYTE_ARRAY_SIZE = Integer.MAX_VALUE - 8;
+
     /** */
-    private static final Long CHECK_FREQ = Long.getLong(IGNITE_MARSHAL_BUFFERS_RECHECK, 10000);
+    private static final long CHECK_FREQ = Long.getLong(IGNITE_MARSHAL_BUFFERS_RECHECK, 10000);
 
     /** Length of char buffer (for writing strings). */
     private static final int CHAR_BUF_SIZE = 256;
@@ -121,9 +130,8 @@ public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput
      * @param size Size.
      */
     private void requestFreeSize(int size) throws IOException {
-        // If arithmetic overflow occurs, off + size should be less than size.
-        if (off + size < size)
-            throw new IOException("Failed to allocate required memory (arithmetic overflow detected) " +
+        if (!canBeAllocated(off + size))
+            throw new IOException("Failed to allocate required memory (byte array size overflow detected) " +
                 "[length=" + size + ", offset=" + off + ']');
 
         size = off + size;
@@ -133,22 +141,18 @@ public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput
         long now = U.currentTimeMillis();
 
         if (size > bytes.length) {
-            byte[] newBytes = new byte[Math.max(size << 1, size)]; // Grow.
+            int newSize = size << 1;
 
-            System.arraycopy(bytes, 0, newBytes, 0, off);
+            if (!canBeAllocated(newSize))
+                newSize = MAX_BYTE_ARRAY_SIZE;
 
-            bytes = newBytes;
+            bytes = Arrays.copyOf(bytes, newSize); // Grow.
         }
         else if (now - lastCheck > CHECK_FREQ) {
             int halfSize = bytes.length >> 1;
 
-            if (maxOff < halfSize) {
-                byte[] newBytes = new byte[halfSize]; // Shrink.
-
-                System.arraycopy(bytes, 0, newBytes, 0, off);
-
-                bytes = newBytes;
-            }
+            if (maxOff < halfSize)
+                bytes = Arrays.copyOf(bytes, halfSize); // Shrink.
 
             maxOff = 0;
             lastCheck = now;
@@ -156,6 +160,15 @@ public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput
     }
 
     /**
+     * @param size Size of potential byte array to check.
+     * @return true if {@code new byte[size]} won't throw {@link OutOfMemoryError} given enough heap space.
+     * @see GridUnsafeDataOutput#MAX_BYTE_ARRAY_SIZE
+     */
+    private boolean canBeAllocated(long size) {
+        return 0 <= size && size <= MAX_BYTE_ARRAY_SIZE;
+    }
+
+    /**
      * @param size Size.
      * @throws IOException In case of error.
      */
@@ -188,9 +201,9 @@ public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput
     @Override public void writeDoubleArray(double[] arr) throws IOException {
         writeInt(arr.length);
 
-        int bytesToCp = arr.length << 3;
+        checkArrayAllocationOverflow(8, arr.length, "double");
 
-        checkArrayAllocationOverflow(bytesToCp, arr.length, "double");
+        int bytesToCp = arr.length << 3;
 
         requestFreeSize(bytesToCp);
 
@@ -222,9 +235,9 @@ public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput
     @Override public void writeCharArray(char[] arr) throws IOException {
         writeInt(arr.length);
 
-        int bytesToCp = arr.length << 1;
+        checkArrayAllocationOverflow(2, arr.length, "char");
 
-        checkArrayAllocationOverflow(bytesToCp, arr.length, "char");
+        int bytesToCp = arr.length << 1;
 
         requestFreeSize(bytesToCp);
 
@@ -247,9 +260,9 @@ public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput
     @Override public void writeLongArray(long[] arr) throws IOException {
         writeInt(arr.length);
 
-        int bytesToCp = arr.length << 3;
+        checkArrayAllocationOverflow(8, arr.length, "long");
 
-        checkArrayAllocationOverflow(bytesToCp, arr.length, "long");
+        int bytesToCp = arr.length << 3;
 
         requestFreeSize(bytesToCp);
 
@@ -272,9 +285,9 @@ public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput
     @Override public void writeFloatArray(float[] arr) throws IOException {
         writeInt(arr.length);
 
-        int bytesToCp = arr.length << 2;
+        checkArrayAllocationOverflow(4, arr.length, "float");
 
-        checkArrayAllocationOverflow(bytesToCp, arr.length, "float");
+        int bytesToCp = arr.length << 2;
 
         requestFreeSize(bytesToCp);
 
@@ -315,9 +328,9 @@ public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput
     @Override public void writeShortArray(short[] arr) throws IOException {
         writeInt(arr.length);
 
-        int bytesToCp = arr.length << 1;
+        checkArrayAllocationOverflow(2, arr.length, "short");
 
-        checkArrayAllocationOverflow(bytesToCp, arr.length, "short");
+        int bytesToCp = arr.length << 1;
 
         requestFreeSize(bytesToCp);
 
@@ -340,9 +353,9 @@ public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput
     @Override public void writeIntArray(int[] arr) throws IOException {
         writeInt(arr.length);
 
-        int bytesToCp = arr.length << 2;
+        checkArrayAllocationOverflow(4, arr.length, "int");
 
-        checkArrayAllocationOverflow(bytesToCp, arr.length, "int");
+        int bytesToCp = arr.length << 2;
 
         requestFreeSize(bytesToCp);
 
@@ -491,16 +504,17 @@ public class GridUnsafeDataOutput extends OutputStream implements GridDataOutput
     /**
      * Check for possible arithmetic overflow when trying to serialize a humongous array.
      *
-     * @param bytesToAlloc Bytes to allocate.
+     * @param bytes Number of bytes in a single array element.
      * @param arrLen Array length.
      * @param type Type of an array.
      * @throws IOException If oveflow presents and data corruption can occur.
      */
-    private void checkArrayAllocationOverflow(int bytesToAlloc, int arrLen, String type) throws IOException {
-        // If arithmetic overflow occurs, bytesToAlloc should be less than arrLen.
-        if (bytesToAlloc < arrLen)
+    private void checkArrayAllocationOverflow(int bytes, int arrLen, String type) throws IOException {
+        long bytesToAlloc = (long)arrLen * bytes;
+
+        if (!canBeAllocated(bytesToAlloc))
             throw new IOException("Failed to allocate required memory for " + type + " array " +
-                "(arithmetic overflow detected) [length=" + arrLen + ']');
+                "(byte array size overflow detected) [length=" + arrLen + ']');
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e2fddc/modules/core/src/test/java/org/apache/ignite/internal/marshaller/optimized/OptimizedMarshallerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/marshaller/optimized/OptimizedMarshallerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/marshaller/optimized/OptimizedMarshallerTest.java
index 7b1d221..94c854f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/marshaller/optimized/OptimizedMarshallerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/marshaller/optimized/OptimizedMarshallerTest.java
@@ -391,45 +391,28 @@ public class OptimizedMarshallerTest extends GridCommonAbstractTest {
     /**
      * Tests checks for arithmetic overflow when trying to serialize huge object.
      * WARNING! Requires a lot of heap space. Should not be run on CI.
+     * Minimal memory requirement is about 6-7 gigabytes of heap.
      */
     public void _testAllocationOverflow() {
         allocationOverflowCheck(() -> marshaller().marshal(new HugeObject()));
 
-        allocationOverflowCheck(() -> {
-            marshaller().marshal(new short[1<<30]);
-            marshaller().marshal(new short[1<<30]);
-            return null;
-        });
+        allocationOverflowCheck(() -> marshaller().marshal(new short[1 << 30]));
 
-        allocationOverflowCheck(() -> {
-            marshaller().marshal(new char[1<<30]);
-            marshaller().marshal(new char[1<<30]);
-            return null;
-        });
+        allocationOverflowCheck(() -> marshaller().marshal(new char[1 << 30]));
 
-        allocationOverflowCheck(() -> {
-            marshaller().marshal(new int[1<<30]);
-            marshaller().marshal(new int[1<<30]);
-            return null;
-        });
+        allocationOverflowCheck(() -> marshaller().marshal(new int[1 << 29]));
 
-        allocationOverflowCheck(() -> {
-            marshaller().marshal(new float[1<<29]);
-            marshaller().marshal(new float[1<<29]);
-            return null;
-        });
+        allocationOverflowCheck(() -> marshaller().marshal(new float[1 << 29]));
 
-        allocationOverflowCheck(() -> {
-            marshaller().marshal(new long[1<<29]);
-            marshaller().marshal(new long[1<<29]);
-            return null;
-        });
+        allocationOverflowCheck(() -> marshaller().marshal(new long[1 << 28]));
 
-        allocationOverflowCheck(() -> {
-            marshaller().marshal(new double[1<<29]);
-            marshaller().marshal(new double[1<<29]);
-            return null;
-        });
+        allocationOverflowCheck(() -> marshaller().marshal(new double[1 << 28]));
+
+        // This particular case requires about 13G of heap space.
+        // It failed because of bug in previous implementation of GridUnsafeDataOutput, mainly line
+        // "if (bytesToAlloc < arrLen)" in method "checkArrayAllocationOverflow". That check doesn't
+        // work as desired on the length in the example below.
+        allocationOverflowCheck(() -> marshaller().marshal(new long[0x2800_0000]));
     }
 
     /**
@@ -437,8 +420,9 @@ public class OptimizedMarshallerTest extends GridCommonAbstractTest {
      *
      * @param call Callable that cause allocation overflow.
      */
+    @SuppressWarnings("ThrowableNotThrown")
     private void allocationOverflowCheck(Callable<?> call) {
-        GridTestUtils.assertThrowsAnyCause(log, call, IOException.class, "Impossible to allocate required memory");
+        GridTestUtils.assertThrowsAnyCause(log, call, IOException.class, "Failed to allocate required memory");
     }
 
     /**
@@ -448,10 +432,12 @@ public class OptimizedMarshallerTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.write(new byte[1 << 31 - 2]);
-            out.write(new byte[1 << 31 - 2]);
-            out.write(new byte[1 << 31 - 2]);
-            out.write(new byte[1 << 31 - 2]);
+            byte[] bytes = new byte[1 << 31 - 2];
+
+            out.write(bytes);
+            out.write(bytes);
+            out.write(bytes);
+            out.write(bytes);
         }
 
         /** {@inheritDoc} */


[45/50] [abbrv] ignite git commit: IGNITE-9999 Added verbose logging for node recovery - Fixes #5371

Posted by sb...@apache.org.
IGNITE-9999 Added verbose logging for node recovery - Fixes #5371


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

Branch: refs/heads/ignite-10044
Commit: b48a291e1a2fc531085cba3b60ff8647ccc1639e
Parents: 74f312e
Author: Pavel Kovalenko <jo...@gmail.com>
Authored: Tue Nov 20 15:45:39 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 20 15:56:59 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   7 +
 .../processors/cache/CacheGroupContext.java     | 148 -------
 .../processors/cache/GridCacheProcessor.java    |  44 +-
 .../cache/IgniteCacheOffheapManager.java        |  11 +
 .../cache/IgniteCacheOffheapManagerImpl.java    |   7 +
 .../GridDhtPartitionsExchangeFuture.java        |  13 +-
 .../persistence/DatabaseLifecycleListener.java  |  43 +-
 .../GridCacheDatabaseSharedManager.java         | 425 +++++++++----------
 .../persistence/GridCacheOffheapManager.java    | 140 ++++++
 .../IgniteCacheDatabaseSharedManager.java       |  12 +-
 .../cache/persistence/wal/FileWALPointer.java   |   2 +-
 .../wal/serializer/RecordDataV1Serializer.java  |   2 +-
 .../db/IgniteLogicalRecoveryTest.java           | 284 ++++++++-----
 .../file/IgnitePdsDiskErrorsRecoveringTest.java |  58 ---
 .../IgniteNodeStoppedDuringDisableWALTest.java  |   3 +-
 .../persistence/db/wal/WalCompactionTest.java   |  13 +-
 16 files changed, 619 insertions(+), 593 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 3f71642..ccf7ebf 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -1040,6 +1040,13 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_ALLOW_START_CACHES_IN_PARALLEL = "IGNITE_ALLOW_START_CACHES_IN_PARALLEL";
 
     /**
+     * Allows to log additional information about all restored partitions after binary and logical recovery phases.
+     *
+     * Default is {@code true}.
+     */
+    public static final String IGNITE_RECOVERY_VERBOSE_LOGGING = "IGNITE_RECOVERY_VERBOSE_LOGGING";
+
+    /**
      * Enforces singleton.
      */
     private IgniteSystemProperties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
index 95fc08f..fc4f79d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
@@ -22,7 +22,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -42,17 +41,11 @@ import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCach
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
-import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
-import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
-import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionRecoverState;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.query.continuous.CounterSkipContext;
 import org.apache.ignite.internal.processors.query.QueryUtils;
@@ -170,9 +163,6 @@ public class CacheGroupContext {
     /** Flag indicates that cache group is under recovering and not attached to topology. */
     private final AtomicBoolean recoveryMode;
 
-    /** Flag indicates that all group partitions have restored their state from page memory / disk. */
-    private volatile boolean partitionStatesRestored;
-
     /**
      * @param ctx Context.
      * @param grpId Group ID.
@@ -791,144 +781,6 @@ public class CacheGroupContext {
     }
 
     /**
-     * Pre-create partitions that resides in page memory or WAL and restores their state.
-     */
-    public long restorePartitionStates(Map<GroupPartitionId, PartitionRecoverState> partitionRecoveryStates) throws IgniteCheckedException {
-        if (isLocal() || !affinityNode() || !dataRegion().config().isPersistenceEnabled())
-            return 0;
-
-        if (partitionStatesRestored)
-            return 0;
-
-        long processed = 0;
-
-        PageMemoryEx pageMem = (PageMemoryEx)dataRegion().pageMemory();
-
-        for (int p = 0; p < affinity().partitions(); p++) {
-            PartitionRecoverState recoverState = partitionRecoveryStates.get(new GroupPartitionId(grpId, p));
-
-            if (ctx.pageStore().exists(grpId, p)) {
-                ctx.pageStore().ensure(grpId, p);
-
-                if (ctx.pageStore().pages(grpId, p) <= 1) {
-                    if (log.isDebugEnabled())
-                        log.debug("Skipping partition on recovery (pages less than 1) " +
-                            "[grp=" + cacheOrGroupName() + ", p=" + p + "]");
-
-                    continue;
-                }
-
-                if (log.isDebugEnabled())
-                    log.debug("Creating partition on recovery (exists in page store) " +
-                        "[grp=" + cacheOrGroupName() + ", p=" + p + "]");
-
-                processed++;
-
-                GridDhtLocalPartition part = topology().forceCreatePartition(p);
-
-                offheap().onPartitionInitialCounterUpdated(p, 0);
-
-                ctx.database().checkpointReadLock();
-
-                try {
-                    long partMetaId = pageMem.partitionMetaPageId(grpId, p);
-                    long partMetaPage = pageMem.acquirePage(grpId, partMetaId);
-
-                    try {
-                        long pageAddr = pageMem.writeLock(grpId, partMetaId, partMetaPage);
-
-                        boolean changed = false;
-
-                        try {
-                            PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(pageAddr);
-
-                            if (recoverState != null) {
-                                io.setPartitionState(pageAddr, (byte) recoverState.stateId());
-
-                                changed = updateState(part, recoverState.stateId());
-
-                                if (recoverState.stateId() == GridDhtPartitionState.OWNING.ordinal()
-                                    || (recoverState.stateId() == GridDhtPartitionState.MOVING.ordinal()
-                                    && part.initialUpdateCounter() < recoverState.updateCounter())) {
-                                    part.initialUpdateCounter(recoverState.updateCounter());
-
-                                    changed = true;
-                                }
-
-                                if (log.isInfoEnabled())
-                                    log.warning("Restored partition state (from WAL) " +
-                                        "[grp=" + cacheOrGroupName() + ", p=" + p + ", state=" + part.state() +
-                                        ", updCntr=" + part.initialUpdateCounter() + "]");
-                            }
-                            else {
-                                int stateId = (int) io.getPartitionState(pageAddr);
-
-                                changed = updateState(part, stateId);
-
-                                if (log.isDebugEnabled())
-                                    log.debug("Restored partition state (from page memory) " +
-                                        "[grp=" + cacheOrGroupName() + ", p=" + p + ", state=" + part.state() +
-                                        ", updCntr=" + part.initialUpdateCounter() + ", stateId=" + stateId + "]");
-                            }
-                        }
-                        finally {
-                            pageMem.writeUnlock(grpId, partMetaId, partMetaPage, null, changed);
-                        }
-                    }
-                    finally {
-                        pageMem.releasePage(grpId, partMetaId, partMetaPage);
-                    }
-                }
-                finally {
-                    ctx.database().checkpointReadUnlock();
-                }
-            }
-            else if (recoverState != null) {
-                GridDhtLocalPartition part = topology().forceCreatePartition(p);
-
-                offheap().onPartitionInitialCounterUpdated(p, recoverState.updateCounter());
-
-                updateState(part, recoverState.stateId());
-
-                processed++;
-
-                if (log.isDebugEnabled())
-                    log.debug("Restored partition state (from WAL) " +
-                        "[grp=" + cacheOrGroupName() + ", p=" + p + ", state=" + part.state() +
-                        ", updCntr=" + part.initialUpdateCounter() + "]");
-            }
-            else {
-                if (log.isDebugEnabled())
-                    log.debug("Skipping partition on recovery (no page store OR wal state) " +
-                        "[grp=" + cacheOrGroupName() + ", p=" + p + "]");
-            }
-        }
-
-        partitionStatesRestored = true;
-
-        return processed;
-    }
-
-    /**
-     * @param part Partition to restore state for.
-     * @param stateId State enum ordinal.
-     * @return Updated flag.
-     */
-    private boolean updateState(GridDhtLocalPartition part, int stateId) {
-        if (stateId != -1) {
-            GridDhtPartitionState state = GridDhtPartitionState.fromOrdinal(stateId);
-
-            assert state != null;
-
-            part.restoreState(state == GridDhtPartitionState.EVICTED ? GridDhtPartitionState.RENTING : state);
-
-            return true;
-        }
-
-        return false;
-    }
-
-    /**
      * @return {@code True} if current cache group is in recovery mode.
      */
     public boolean isRecoveryMode() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/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 23c3623..ce81468 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
@@ -106,6 +106,8 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStor
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
 import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetastorageLifecycleListener;
 import org.apache.ignite.internal.processors.cache.persistence.metastorage.ReadOnlyMetastorage;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionRecoverState;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
@@ -5438,6 +5440,15 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         private final Map<Integer, QuerySchema> querySchemas = new ConcurrentHashMap<>();
 
         /** {@inheritDoc} */
+        @Override public void onBaselineChange() {
+            onKernalStopCaches(true);
+
+            stopCaches(true);
+
+            sharedCtx.database().cleanupRestoredCaches();
+        }
+
+        /** {@inheritDoc} */
         @Override public void onReadyForRead(ReadOnlyMetastorage metastorage) throws IgniteCheckedException {
             restoreCacheConfigurations();
         }
@@ -5449,13 +5460,44 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public void afterBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {
+        @Override public void afterBinaryMemoryRestore(GridCacheDatabaseSharedManager.RestoreBinaryState binaryState) throws IgniteCheckedException {
             for (DynamicCacheDescriptor cacheDescriptor : persistentCaches()) {
                 startCacheInRecoveryMode(cacheDescriptor);
 
                 querySchemas.put(cacheDescriptor.cacheId(), cacheDescriptor.schema().copy());
             }
         }
+
+        /** {@inheritDoc} */
+        @Override public void afterLogicalUpdatesApplied(GridCacheDatabaseSharedManager.RestoreLogicalState logicalState) throws IgniteCheckedException {
+            restorePartitionStates(cacheGroups(), logicalState.partitionRecoveryStates());
+        }
+
+        /**
+         * @param forGroups Cache groups.
+         * @param partitionStates Partition states.
+         * @throws IgniteCheckedException If failed.
+         */
+        private void restorePartitionStates(
+            Collection<CacheGroupContext> forGroups,
+            Map<GroupPartitionId, PartitionRecoverState> partitionStates
+        ) throws IgniteCheckedException {
+            long startRestorePart = U.currentTimeMillis();
+
+            if (log.isInfoEnabled())
+                log.info("Restoring partition state for local groups.");
+
+            long totalProcessed = 0;
+
+            for (CacheGroupContext grp : forGroups)
+                totalProcessed += grp.offheap().restorePartitionStates(partitionStates);
+
+            if (log.isInfoEnabled())
+                log.info("Finished restoring partition state for local groups [" +
+                    "groupsProcessed=" + forGroups.size() +
+                    ", partitionsProcessed=" + totalProcessed +
+                    ", time=" + (U.currentTimeMillis() - startRestorePart) + "ms]");
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
index 2cf302f..db09a89 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
@@ -31,6 +31,8 @@ import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow;
 import org.apache.ignite.internal.processors.cache.persistence.RootPage;
 import org.apache.ignite.internal.processors.cache.persistence.RowStore;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionRecoverState;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateResult;
@@ -82,6 +84,15 @@ public interface IgniteCacheOffheapManager {
     public void stop();
 
     /**
+     * Pre-create partitions that resides in page memory or WAL and restores their state.
+     *
+     * @param partitionRecoveryStates Partition recovery states.
+     * @return Number of processed partitions.
+     * @throws IgniteCheckedException If failed.
+     */
+    long restorePartitionStates(Map<GroupPartitionId, PartitionRecoverState> partitionRecoveryStates) throws IgniteCheckedException;
+
+    /**
      * Partition counter update callback. May be overridden by plugin-provided subclasses.
      *
      * @param part Partition.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 08ce978..6835795 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -60,6 +60,8 @@ import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapt
 import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow;
 import org.apache.ignite.internal.processors.cache.persistence.RootPage;
 import org.apache.ignite.internal.processors.cache.persistence.RowStore;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionRecoverState;
 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.DataPageIO;
@@ -258,6 +260,11 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
     }
 
     /** {@inheritDoc} */
+    @Override public long restorePartitionStates(Map<GroupPartitionId, PartitionRecoverState> partitionRecoveryStates) throws IgniteCheckedException {
+        return 0; // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void onKernalStop() {
         busyLock.block();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 44fc266..2e792f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -92,6 +92,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.Grid
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionsStateValidator;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator;
+import org.apache.ignite.internal.processors.cache.persistence.DatabaseLifecycleListener;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -890,15 +891,11 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             cctx.exchange().exchangerBlockingSectionBegin();
 
             try {
-                // Stop all recovered caches and groups.
-                cctx.cache().onKernalStopCaches(true);
+                List<DatabaseLifecycleListener> listeners = cctx.kernalContext().internalSubscriptionProcessor()
+                    .getDatabaseListeners();
 
-                cctx.cache().stopCaches(true);
-
-                cctx.database().cleanupRestoredCaches();
-
-                // Set initial node started marker.
-                cctx.database().nodeStart(null);
+                for (DatabaseLifecycleListener lsnr : listeners)
+                    lsnr.onBaselineChange();
             }
             finally {
                 cctx.exchange().exchangerBlockingSectionEnd();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java
index 1f7ba84..6762109 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java
@@ -22,6 +22,7 @@ import org.apache.ignite.IgniteCheckedException;
 /**
  *
  */
+@SuppressWarnings("RedundantThrows")
 public interface DatabaseLifecycleListener {
     /**
      * Callback executed when data regions become to start-up.
@@ -29,7 +30,15 @@ public interface DatabaseLifecycleListener {
      * @param mgr Database shared manager.
      * @throws IgniteCheckedException If failed.
      */
-    default void onInitDataRegions(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {};
+    public default void onInitDataRegions(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {}
+
+    /**
+     * Callback executed when node detected that baseline topology is changed and node is not in that baseline.
+     * It's useful to cleanup and invalidate all available data restored at that moment.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public default void onBaselineChange() throws IgniteCheckedException {}
 
     /**
      * Callback executed right before node become perform binary recovery.
@@ -37,30 +46,46 @@ public interface DatabaseLifecycleListener {
      * @param mgr Database shared manager.
      * @throws IgniteCheckedException If failed.
      */
-    default void beforeBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {};
+    public default void beforeBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {}
 
     /**
      * Callback executed when binary memory has fully restored and WAL logging is resumed.
      *
-     * @param mgr Database shared manager.
+     * @param binaryState Result of binary recovery.
      * @throws IgniteCheckedException If failed.
      */
-    default void afterBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {};
+    public default void afterBinaryMemoryRestore(GridCacheDatabaseSharedManager.RestoreBinaryState binaryState)
+        throws IgniteCheckedException {}
 
     /**
+     * Callback executed when all logical updates were applied and page memory become to fully consistent state.
      *
-     * @param mgr
-     * @throws IgniteCheckedException
+     * @param logicalState Result of logical recovery.
+     * @throws IgniteCheckedException If failed.
      */
-    default void beforeResumeWalLogging(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {};
+    public default void afterLogicalUpdatesApplied(GridCacheDatabaseSharedManager.RestoreLogicalState logicalState)
+        throws IgniteCheckedException {}
 
     /**
+     * Callback executed when all physical updates are applied and we are ready to write new physical records
+     * during logical recovery.
+     *
      * @param mgr Database shared manager.
+     * @throws IgniteCheckedException If failed.
      */
-    default void afterInitialise(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {};
+    public default void beforeResumeWalLogging(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {}
 
     /**
+     * Callback executed after all data regions are initialized.
+     *
+     * @param mgr Database shared manager.
+     */
+    public default void afterInitialise(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {}
+
+    /**
+     * Callback executed before shared manager will be stopped.
+     *
      * @param mgr Database shared manager.
      */
-    default void beforeStop(IgniteCacheDatabaseSharedManager mgr) {};
+    public default void beforeStop(IgniteCacheDatabaseSharedManager mgr) {}
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 019d1aa..c74972a 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -93,7 +93,6 @@ import org.apache.ignite.internal.pagemem.wal.record.CacheState;
 import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord;
 import org.apache.ignite.internal.pagemem.wal.record.DataEntry;
 import org.apache.ignite.internal.pagemem.wal.record.DataRecord;
-import org.apache.ignite.internal.pagemem.wal.record.MemoryRecoveryRecord;
 import org.apache.ignite.internal.pagemem.wal.record.MetastoreDataRecord;
 import org.apache.ignite.internal.pagemem.wal.record.MvccDataEntry;
 import org.apache.ignite.internal.pagemem.wal.record.MvccTxRecord;
@@ -133,6 +132,7 @@ import org.apache.ignite.internal.processors.cache.persistence.partstate.Partiti
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperation;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
 import org.apache.ignite.internal.processors.port.GridPortRecord;
@@ -153,6 +153,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.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteOutClosure;
@@ -167,7 +168,6 @@ import org.jsr166.ConcurrentLinkedHashMap;
 
 import static java.nio.file.StandardOpenOption.READ;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD;
 import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
 import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION;
@@ -192,9 +192,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     private final boolean skipSync = IgniteSystemProperties.getBoolean(IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC);
 
     /** */
-    private boolean skipCrc = IgniteSystemProperties.getBoolean(IGNITE_PDS_SKIP_CRC, false);
-
-    /** */
     private final int walRebalanceThreshold = IgniteSystemProperties.getInteger(
         IGNITE_PDS_WAL_REBALANCE_THRESHOLD, 500_000);
 
@@ -218,9 +215,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** Checkpoint file name pattern. */
     public static final Pattern CP_FILE_NAME_PATTERN = Pattern.compile("(\\d+)-(.*)-(START|END)\\.bin");
 
-    /** Node started file suffix. */
-    public static final String NODE_STARTED_FILE_NAME_SUFFIX = "-node-started.bin";
-
     /** */
     private static final String MBEAN_NAME = "DataStorageMetrics";
 
@@ -362,6 +356,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** Timeout for checkpoint read lock acquisition in milliseconds. */
     private volatile long checkpointReadLockTimeout;
 
+    /** Flag allows to log additional information about partitions during recovery phases. */
+    private final boolean recoveryVerboseLogging = IgniteSystemProperties.getBoolean(
+            IgniteSystemProperties.IGNITE_RECOVERY_VERBOSE_LOGGING, true);
+
     /**
      * @param ctx Kernal context.
      */
@@ -484,6 +482,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         final GridKernalContext kernalCtx = cctx.kernalContext();
 
         if (!kernalCtx.clientNode()) {
+            kernalCtx.internalSubscriptionProcessor().registerDatabaseListener(new MetastorageRecoveryLifecycle());
+
             checkpointer = new Checkpointer(cctx.igniteInstanceName(), "db-checkpoint-thread", log);
 
             cpHistory = new CheckpointHistory(kernalCtx);
@@ -543,7 +543,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 continue;
 
             if (log.isInfoEnabled())
-                log.info("Page memory " + region + " for " + grpDesc + " has invalidated.");
+                log.info("Page memory " + region.config().getName() + " for " + grpDesc + " has invalidated.");
 
             int partitions = grpDesc.config().getAffinity().partitions();
 
@@ -895,31 +895,40 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      * @return Last seen WAL pointer during binary memory recovery.
      * @throws IgniteCheckedException If failed.
      */
-    private WALPointer restoreBinaryMemory(Predicate<Integer> cacheGroupsPredicate) throws IgniteCheckedException {
-        assert !cctx.kernalContext().clientNode();
-
+    private RestoreBinaryState restoreBinaryMemory(Predicate<Integer> cacheGroupsPredicate) throws IgniteCheckedException {
         long time = System.currentTimeMillis();
 
-        checkpointReadLock();
-
         try {
+            log.info("Starting binary memory restore for: " + cctx.cache().cacheGroupDescriptors().keySet());
+
+            for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext()))
+                lsnr.beforeBinaryMemoryRestore(this);
+
             CheckpointStatus status = readCheckpointStatus();
 
             // First, bring memory to the last consistent checkpoint state if needed.
             // This method should return a pointer to the last valid record in the WAL.
-            WALPointer restored = performBinaryMemoryRestore(status, cacheGroupsPredicate, true);
+            RestoreBinaryState binaryState = performBinaryMemoryRestore(status, cacheGroupsPredicate, true);
+
+            WALPointer restored = binaryState.lastReadRecordPointer();
 
             if (restored == null && !status.endPtr.equals(CheckpointStatus.NULL_PTR)) {
                 throw new StorageException("The memory cannot be restored. The critical part of WAL archive is missing " +
                     "[tailWalPtr=" + restored + ", endPtr=" + status.endPtr + ']');
             }
+            else if (restored != null)
+                U.log(log, "Binary memory state restored at node startup [restoredPtr=" + restored + ']');
 
-            nodeStart(restored);
+            // Wal logging is now available.
+            cctx.wal().resumeLogging(restored);
+
+            for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext()))
+                lsnr.afterBinaryMemoryRestore(binaryState);
 
             if (log.isInfoEnabled())
                 log.info("Binary recovery performed in " + (System.currentTimeMillis() - time) + " ms.");
 
-            return restored;
+            return binaryState;
         }
         catch (IgniteCheckedException e) {
             if (X.hasCause(e, StorageException.class, IOException.class))
@@ -927,97 +936,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
             throw e;
         }
-        finally {
-            checkpointReadUnlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void nodeStart(@Nullable WALPointer ptr) throws IgniteCheckedException {
-        FileWALPointer p = (FileWALPointer)(ptr == null ? CheckpointStatus.NULL_PTR : ptr);
-
-        String fileName = U.currentTimeMillis() + NODE_STARTED_FILE_NAME_SUFFIX;
-        String tmpFileName = fileName + FilePageStoreManager.TMP_SUFFIX;
-
-        ByteBuffer buf = ByteBuffer.allocate(FileWALPointer.POINTER_SIZE);
-        buf.order(ByteOrder.nativeOrder());
-
-        try {
-            try (FileIO io = ioFactory.create(Paths.get(cpDir.getAbsolutePath(), tmpFileName).toFile(),
-                    StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)) {
-                buf.putLong(p.index());
-
-                buf.putInt(p.fileOffset());
-
-                buf.putInt(p.length());
-
-                buf.flip();
-
-                io.writeFully(buf);
-
-                buf.clear();
-
-                io.force(true);
-            }
-
-            Files.move(Paths.get(cpDir.getAbsolutePath(), tmpFileName), Paths.get(cpDir.getAbsolutePath(), fileName));
-        }
-        catch (IOException e) {
-            throw new StorageException("Failed to write node start marker: " + ptr, e);
-        }
-    }
-
-    /**
-     * Collects memory recovery pointers from node started files. See {@link #nodeStart(WALPointer)}.
-     * Each pointer associated with timestamp extracted from file.
-     * Tuples are sorted by timestamp.
-     *
-     * @return Sorted list of tuples (node started timestamp, memory recovery pointer).
-     *
-     * @throws IgniteCheckedException If failed.
-     */
-    public List<T2<Long, WALPointer>> nodeStartedPointers() throws IgniteCheckedException {
-        List<T2<Long, WALPointer>> res = new ArrayList<>();
-
-        try (DirectoryStream<Path> nodeStartedFiles = Files.newDirectoryStream(
-            cpDir.toPath(),
-            path -> path.toFile().getName().endsWith(NODE_STARTED_FILE_NAME_SUFFIX))
-        ) {
-            ByteBuffer buf = ByteBuffer.allocate(FileWALPointer.POINTER_SIZE);
-            buf.order(ByteOrder.nativeOrder());
-
-            for (Path path : nodeStartedFiles) {
-                File f = path.toFile();
-
-                String name = f.getName();
-
-                Long ts = Long.valueOf(name.substring(0, name.length() - NODE_STARTED_FILE_NAME_SUFFIX.length()));
-
-                try (FileIO io = ioFactory.create(f, READ)) {
-                    io.readFully(buf);
-
-                    buf.flip();
-
-                    FileWALPointer ptr = new FileWALPointer(
-                        buf.getLong(), buf.getInt(), buf.getInt());
-
-                    res.add(new T2<>(ts, ptr));
-
-                    buf.clear();
-                }
-                catch (IOException e) {
-                    throw new StorageException("Failed to read node started marker file: " + f.getAbsolutePath(), e);
-                }
-            }
-        }
-        catch (IOException e) {
-            throw new StorageException("Failed to retreive node started files.", e);
-        }
-
-        // Sort start markers by file timestamp.
-        res.sort(Comparator.comparingLong(IgniteBiTuple::get1));
-
-        return res;
     }
 
     /** {@inheritDoc} */
@@ -1357,7 +1275,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     cctx.database().checkpointReadLock();
 
                     try {
-                        cacheGroup.restorePartitionStates(Collections.emptyMap());
+                        cacheGroup.offheap().restorePartitionStates(Collections.emptyMap());
 
                         if (cacheGroup.localStartVersion().equals(fut.initialVersion()))
                             cacheGroup.topology().afterStateRestored(fut.initialVersion());
@@ -1504,14 +1422,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             for (IgniteBiTuple<CacheGroupContext, Boolean> tup : stoppedGrps) {
                 CacheGroupContext grp = tup.get1();
 
-                if (grp.affinityNode()) {
-                    try {
-                        cctx.pageStore().shutdownForCacheGroup(grp, tup.get2());
-                    }
-                    catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to gracefully clean page store resources for destroyed cache " +
-                            "[cache=" + grp.cacheOrGroupName() + "]", e);
-                    }
+                try {
+                    cctx.pageStore().shutdownForCacheGroup(grp, tup.get2());
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to gracefully clean page store resources for destroyed cache " +
+                        "[cache=" + grp.cacheOrGroupName() + "]", e);
                 }
             }
         }
@@ -1978,32 +1894,16 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             // Preform early regions startup before restoring state.
             initAndStartRegions(kctx.config().getDataStorageConfiguration());
 
-            for (DatabaseLifecycleListener lsnr : getDatabaseListeners(kctx))
-                lsnr.beforeBinaryMemoryRestore(this);
-
-            log.info("Starting binary memory restore for: " + cctx.cache().cacheGroupDescriptors().keySet());
-
-            cctx.pageStore().initializeForMetastorage();
-
             // Restore binary memory for all not WAL disabled cache groups.
-            WALPointer restored = restoreBinaryMemory(
+            restoreBinaryMemory(
                     g -> !initiallyGlobalWalDisabledGrps.contains(g) && !initiallyLocalWalDisabledGrps.contains(g)
             );
 
-            if (restored != null)
-                U.log(log, "Binary memory state restored at node startup [restoredPtr=" + restored + ']');
-
-            for (DatabaseLifecycleListener lsnr : getDatabaseListeners(kctx))
-                lsnr.afterBinaryMemoryRestore(this);
+            if (recoveryVerboseLogging && log.isInfoEnabled()) {
+                log.info("Partition states information after BINARY RECOVERY phase:");
 
-            cctx.wal().resumeLogging(restored);
-
-            // We should log this record to ensure that node start marker pointer will be found in compacted segment.
-            cctx.wal().log(new MemoryRecoveryRecord(System.currentTimeMillis()));
-
-            assert metaStorage == null;
-
-            metaStorage = createMetastorage(false);
+                dumpPartitionsInfo(cctx, log);
+            }
 
             CheckpointStatus status = readCheckpointStatus();
 
@@ -2014,8 +1914,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     false
             );
 
-            // Restore state for all groups.
-            restorePartitionStates(cctx.cache().cacheGroups(), logicalState.partitionRecoveryStates);
+            if (recoveryVerboseLogging && log.isInfoEnabled()) {
+                log.info("Partition states information after LOGICAL RECOVERY phase:");
+
+                dumpPartitionsInfo(cctx, log);
+            }
 
             walTail = tailPointer(logicalState.lastRead);
 
@@ -2056,32 +1959,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
-     * @param forGroups Cache groups.
-     * @param partitionStates Partition states.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void restorePartitionStates(
-        Collection<CacheGroupContext> forGroups,
-        Map<GroupPartitionId, PartitionRecoverState> partitionStates
-    ) throws IgniteCheckedException {
-        long startRestorePart = U.currentTimeMillis();
-
-        if (log.isInfoEnabled())
-            log.info("Restoring partition state for local groups.");
-
-        long totalProcessed = 0;
-
-        for (CacheGroupContext grp : forGroups)
-            totalProcessed += grp.restorePartitionStates(partitionStates);
-
-        if (log.isInfoEnabled())
-            log.info("Finished restoring partition state for local groups [" +
-                "groupsProcessed" + forGroups.size() +
-                "partitionsProcessed=" + totalProcessed +
-                ", time=" + (U.currentTimeMillis() - startRestorePart) + "ms]");
-    }
-
-    /**
      * Called when all partitions have been fully restored and pre-created on node start.
      *
      * Starts checkpointing process and initiates first checkpoint.
@@ -2108,7 +1985,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      * @throws IgniteCheckedException If failed.
      * @throws StorageException In case I/O error occurred during operations with storage.
      */
-    private WALPointer performBinaryMemoryRestore(
+    private RestoreBinaryState performBinaryMemoryRestore(
         CheckpointStatus status,
         Predicate<Integer> cacheGroupsPredicate,
         boolean finalizeState
@@ -2257,7 +2134,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         if (!finalizeState)
             return null;
 
-        WALPointer lastReadPtr = restoreBinaryState.lastReadRecordPointer();
+        FileWALPointer lastReadPtr = restoreBinaryState.lastReadRecordPointer();
 
         if (status.needRestoreMemory()) {
             if (restoreBinaryState.needApplyBinaryUpdate())
@@ -2274,7 +2151,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
         cpHistory.initialize(retreiveHistory());
 
-        return lastReadPtr != null ? lastReadPtr.next() : null;
+        // Move pointer position to the end of last read record.
+        restoreBinaryState.lastRead = lastReadPtr != null ? lastReadPtr.next() : lastReadPtr;
+
+        return restoreBinaryState;
     }
 
     /**
@@ -2311,61 +2191,61 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      * @param it WalIterator.
      * @param recPredicate Wal record filter.
      * @param entryPredicate Entry filter.
-     * @param partitionRecoveryStates Partition to restore state.
      */
     public void applyUpdatesOnRecovery(
         @Nullable WALIterator it,
-        IgnitePredicate<IgniteBiTuple<WALPointer, WALRecord>> recPredicate,
-        IgnitePredicate<DataEntry> entryPredicate,
-        Map<GroupPartitionId, PartitionRecoverState> partitionRecoveryStates
+        IgniteBiPredicate<WALPointer, WALRecord> recPredicate,
+        IgnitePredicate<DataEntry> entryPredicate
     ) throws IgniteCheckedException {
+        if (it == null)
+            return;
+
         cctx.walState().runWithOutWAL(() -> {
-            if (it != null) {
-                while (it.hasNext()) {
-                    IgniteBiTuple<WALPointer, WALRecord> next = it.next();
+            while (it.hasNext()) {
+                IgniteBiTuple<WALPointer, WALRecord> next = it.next();
 
-                    WALRecord rec = next.get2();
+                WALRecord rec = next.get2();
 
-                    if (!recPredicate.apply(next))
-                        break;
+                if (!recPredicate.apply(next.get1(), rec))
+                    break;
 
-                    switch (rec.type()) {
-                        case MVCC_DATA_RECORD:
+                switch (rec.type()) {
+                    case MVCC_DATA_RECORD:
                         case DATA_RECORD:
-                            checkpointReadLock();
+                        checkpointReadLock();
 
-                            try {
-                                DataRecord dataRec = (DataRecord)rec;
+                        try {
+                            DataRecord dataRec = (DataRecord)rec;
 
-                                for (DataEntry dataEntry : dataRec.writeEntries()) {
-                                    if (entryPredicate.apply(dataEntry)) {
-                                        checkpointReadLock();
+                            for (DataEntry dataEntry : dataRec.writeEntries()) {
+                                if (entryPredicate.apply(dataEntry)) {
+                                    checkpointReadLock();
 
-                                        try {
-                                            int cacheId = dataEntry.cacheId();
+                                    try {
+                                        int cacheId = dataEntry.cacheId();
 
-                                            GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
+                                        GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
 
-                                            if (cacheCtx != null)
-                                                applyUpdate(cacheCtx, dataEntry);
-                                            else if (log != null)
-                                                log.warning("Cache is not started. Updates cannot be applied " +
-                                                    "[cacheId=" + cacheId + ']');
-                                        }
-                                        finally {
-                                            checkpointReadUnlock();
-                                        }
+                                        if (cacheCtx != null)
+                                            applyUpdate(cacheCtx, dataEntry);
+                                        else if (log != null)
+                                            log.warning("Cache is not started. Updates cannot be applied " +
+                                                "[cacheId=" + cacheId + ']');
+                                    }
+                                    finally {
+                                        checkpointReadUnlock();
                                     }
                                 }
                             }
-                            catch (IgniteCheckedException e) {
-                                throw new IgniteException(e);
-                            }
-                            finally {
-                                checkpointReadUnlock();
-                            }
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                        finally {
+                            checkpointReadUnlock();
+                        }
 
-                            break;
+                        break;
 
                         case MVCC_TX_RECORD:
                             checkpointReadLock();
@@ -2386,23 +2266,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
                             break;
 
-                        default:
-                            // Skip other records.
-                    }
+                    default:
+                        // Skip other records.
                 }
             }
-
-            checkpointReadLock();
-
-            try {
-                restorePartitionStates(cctx.cache().cacheGroups(), partitionRecoveryStates);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
-            finally {
-                checkpointReadUnlock();
-            }
         });
     }
 
@@ -2538,6 +2405,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             log.info("Finished applying WAL changes [updatesApplied=" + applied +
                 ", time=" + (U.currentTimeMillis() - start) + " ms]");
 
+        for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext()))
+            lsnr.afterLogicalUpdatesApplied(restoreLogicalState);
+
         return restoreLogicalState;
     }
 
@@ -4223,12 +4093,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /**
      *
      */
-    private static class CheckpointStatus {
+    public static class CheckpointStatus {
         /** Null checkpoint UUID. */
         private static final UUID NULL_UUID = new UUID(0L, 0L);
 
         /** Null WAL pointer. */
-        private static final WALPointer NULL_PTR = new FileWALPointer(0, 0, 0);
+        public static final WALPointer NULL_PTR = new FileWALPointer(0, 0, 0);
 
         /** */
         private long cpStartTs;
@@ -4693,6 +4563,97 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
+     * Method dumps partitions info see {@link #dumpPartitionsInfo(CacheGroupContext, IgniteLogger)}
+     * for all persistent cache groups.
+     *
+     * @param cctx Shared context.
+     * @param log Logger.
+     * @throws IgniteCheckedException If failed.
+     */
+    private static void dumpPartitionsInfo(GridCacheSharedContext cctx, IgniteLogger log) throws IgniteCheckedException {
+        for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
+            if (grp.isLocal() || !grp.persistenceEnabled())
+                continue;
+
+            dumpPartitionsInfo(grp, log);
+        }
+    }
+
+    /**
+     * Retrieves from page memory meta information about given {@code grp} group partitions
+     * and dumps this information to log INFO level.
+     *
+     * @param grp Cache group.
+     * @param log Logger.
+     * @throws IgniteCheckedException If failed.
+     */
+    private static void dumpPartitionsInfo(CacheGroupContext grp, IgniteLogger log) throws IgniteCheckedException {
+        PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+
+        IgnitePageStoreManager pageStore = grp.shared().pageStore();
+
+        assert pageStore != null : "Persistent cache should have initialize page store manager.";
+
+        for (int p = 0; p < grp.affinity().partitions(); p++) {
+            if (!pageStore.exists(grp.groupId(), p))
+                continue;
+
+            pageStore.ensure(grp.groupId(), p);
+
+            if (pageStore.pages(grp.groupId(), p) <= 1) {
+                log.info("Partition [id=" + p + ", state=N/A (only file header) ]");
+
+                continue;
+            }
+
+            long partMetaId = pageMem.partitionMetaPageId(grp.groupId(), p);
+            long partMetaPage = pageMem.acquirePage(grp.groupId(), partMetaId);
+
+            try {
+                long pageAddr = pageMem.readLock(grp.groupId(), partMetaId, partMetaPage);
+
+                try {
+                    PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(pageAddr);
+
+                    GridDhtPartitionState partitionState = GridDhtPartitionState.fromOrdinal(io.getPartitionState(pageAddr));
+
+                    String state = partitionState != null ? partitionState.toString() : "N/A";
+
+                    long updateCounter = io.getUpdateCounter(pageAddr);
+                    long size = io.getSize(pageAddr);
+
+                    log.info("Partition [grp=" + grp.cacheOrGroupName()
+                            + ", id=" + p
+                            + ", state=" + state
+                            + ", counter=" + updateCounter
+                            + ", size=" + size + "]");
+                }
+                finally {
+                    pageMem.readUnlock(grp.groupId(), partMetaId, partMetaPage);
+                }
+            }
+            finally {
+                pageMem.releasePage(grp.groupId(), partMetaId, partMetaPage);
+            }
+        }
+    }
+
+    /**
+     * Recovery lifecycle for read-write metastorage.
+     */
+    private class MetastorageRecoveryLifecycle implements DatabaseLifecycleListener {
+        @Override public void beforeBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {
+            cctx.pageStore().initializeForMetastorage();
+        }
+
+        @Override public void afterBinaryMemoryRestore(RestoreBinaryState binaryState) throws IgniteCheckedException {
+            assert metaStorage == null;
+
+            metaStorage = createMetastorage(false);
+        }
+    }
+
+    /**
      * Abstract class for create restore context.
      */
     private abstract class RestoreStateContext {
@@ -4764,12 +4725,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                                 // Filter data entries by group id.
                                 List<DataEntry> filteredEntries = dataRecord.writeEntries().stream()
                                         .filter(entry -> {
-                                            if (entry == null)
-                                                return false;
-
                                             int cacheId = entry.cacheId();
 
-                                            return cctx != null && cctx.cacheContext(cacheId) != null && cacheGroupPredicate.test(cctx.cacheContext(cacheId).groupId());
+                                            return cctx.cacheContext(cacheId) != null && cacheGroupPredicate.test(cctx.cacheContext(cacheId).groupId());
                                         })
                                         .collect(Collectors.toList());
 
@@ -4805,7 +4763,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
          *
          * @return Last read WAL record pointer.
          */
-        public WALPointer lastReadRecordPointer() {
+        public FileWALPointer lastReadRecordPointer() {
             return lastRead;
         }
 
@@ -4823,7 +4781,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /**
      * Restore memory context. Tracks the safety of binary recovery.
      */
-    private class RestoreBinaryState extends RestoreStateContext {
+    public class RestoreBinaryState extends RestoreStateContext {
         /** Checkpoint status. */
         private final CheckpointStatus status;
 
@@ -4898,7 +4856,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /**
      * Restore logical state context. Tracks the safety of logical recovery.
      */
-    private class RestoreLogicalState extends RestoreStateContext {
+    public class RestoreLogicalState extends RestoreStateContext {
         /** States of partitions recovered during applying logical updates. */
         private final Map<GroupPartitionId, PartitionRecoverState> partitionRecoveryStates = new HashMap<>();
 
@@ -4908,6 +4866,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         public RestoreLogicalState(long lastArchivedSegment, Predicate<Integer> cacheGroupsPredicate) {
             super(lastArchivedSegment, cacheGroupsPredicate, false);
         }
+
+        /**
+         * @return Map of restored partition states for cache groups.
+         */
+        public Map<GroupPartitionId, PartitionRecoverState> partitionRecoveryStates() {
+            return Collections.unmodifiableMap(partitionRecoveryStates);
+        }
     }
 
     /** Indicates checkpoint read lock acquisition failure which did not lead to node invalidation. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index f24900f..73cb878 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -71,6 +71,7 @@ import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemor
 import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
 import org.apache.ignite.internal.processors.cache.persistence.partstate.PagesAllocationRange;
 import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionRecoverState;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageMetaIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionCountersIO;
@@ -112,6 +113,9 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
     /** */
     private ReuseListImpl reuseList;
 
+    /** Flag indicates that all group partitions have restored their state from page memory / disk. */
+    private volatile boolean partitionStatesRestored;
+
     /** {@inheritDoc} */
     @Override protected void initPendingTree(GridCacheContext cctx) throws IgniteCheckedException {
         // No-op. Per-partition PendingTree should be used.
@@ -396,6 +400,142 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
             tryAddEmptyPartitionToSnapshot(store, ctx);
     }
 
+    /** {@inheritDoc} */
+    @Override public long restorePartitionStates(Map<GroupPartitionId, PartitionRecoverState> partitionRecoveryStates) throws IgniteCheckedException {
+        if (grp.isLocal() || !grp.affinityNode() || !grp.dataRegion().config().isPersistenceEnabled())
+            return 0;
+
+        if (partitionStatesRestored)
+            return 0;
+
+        long processed = 0;
+
+        PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+
+        for (int p = 0; p < grp.affinity().partitions(); p++) {
+            PartitionRecoverState recoverState = partitionRecoveryStates.get(new GroupPartitionId(grp.groupId(), p));
+
+            if (ctx.pageStore().exists(grp.groupId(), p)) {
+                ctx.pageStore().ensure(grp.groupId(), p);
+
+                if (ctx.pageStore().pages(grp.groupId(), p) <= 1) {
+                    if (log.isDebugEnabled())
+                        log.debug("Skipping partition on recovery (pages less than 1) " +
+                            "[grp=" + grp.cacheOrGroupName() + ", p=" + p + "]");
+
+                    continue;
+                }
+
+                if (log.isDebugEnabled())
+                    log.debug("Creating partition on recovery (exists in page store) " +
+                        "[grp=" + grp.cacheOrGroupName() + ", p=" + p + "]");
+
+                processed++;
+
+                GridDhtLocalPartition part = grp.topology().forceCreatePartition(p);
+
+                onPartitionInitialCounterUpdated(p, 0);
+
+                ctx.database().checkpointReadLock();
+
+                try {
+                    long partMetaId = pageMem.partitionMetaPageId(grp.groupId(), p);
+                    long partMetaPage = pageMem.acquirePage(grp.groupId(), partMetaId);
+
+                    try {
+                        long pageAddr = pageMem.writeLock(grp.groupId(), partMetaId, partMetaPage);
+
+                        boolean changed = false;
+
+                        try {
+                            PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(pageAddr);
+
+                            if (recoverState != null) {
+                                io.setPartitionState(pageAddr, (byte) recoverState.stateId());
+
+                                changed = updateState(part, recoverState.stateId());
+
+                                if (recoverState.stateId() == GridDhtPartitionState.OWNING.ordinal()
+                                    || (recoverState.stateId() == GridDhtPartitionState.MOVING.ordinal()
+                                    && part.initialUpdateCounter() < recoverState.updateCounter())) {
+                                    part.initialUpdateCounter(recoverState.updateCounter());
+
+                                    changed = true;
+                                }
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Restored partition state (from WAL) " +
+                                        "[grp=" + grp.cacheOrGroupName() + ", p=" + p + ", state=" + part.state() +
+                                        ", updCntr=" + part.initialUpdateCounter() + "]");
+                            }
+                            else {
+                                int stateId = (int) io.getPartitionState(pageAddr);
+
+                                changed = updateState(part, stateId);
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Restored partition state (from page memory) " +
+                                        "[grp=" + grp.cacheOrGroupName() + ", p=" + p + ", state=" + part.state() +
+                                        ", updCntr=" + part.initialUpdateCounter() + ", stateId=" + stateId + "]");
+                            }
+                        }
+                        finally {
+                            pageMem.writeUnlock(grp.groupId(), partMetaId, partMetaPage, null, changed);
+                        }
+                    }
+                    finally {
+                        pageMem.releasePage(grp.groupId(), partMetaId, partMetaPage);
+                    }
+                }
+                finally {
+                    ctx.database().checkpointReadUnlock();
+                }
+            }
+            else if (recoverState != null) {
+                GridDhtLocalPartition part = grp.topology().forceCreatePartition(p);
+
+                onPartitionInitialCounterUpdated(p, recoverState.updateCounter());
+
+                updateState(part, recoverState.stateId());
+
+                processed++;
+
+                if (log.isDebugEnabled())
+                    log.debug("Restored partition state (from WAL) " +
+                        "[grp=" + grp.cacheOrGroupName() + ", p=" + p + ", state=" + part.state() +
+                        ", updCntr=" + part.initialUpdateCounter() + "]");
+            }
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Skipping partition on recovery (no page store OR wal state) " +
+                        "[grp=" + grp.cacheOrGroupName() + ", p=" + p + "]");
+            }
+        }
+
+        partitionStatesRestored = true;
+
+        return processed;
+    }
+
+    /**
+     * @param part Partition to restore state for.
+     * @param stateId State enum ordinal.
+     * @return Updated flag.
+     */
+    private boolean updateState(GridDhtLocalPartition part, int stateId) {
+        if (stateId != -1) {
+            GridDhtPartitionState state = GridDhtPartitionState.fromOrdinal(stateId);
+
+            assert state != null;
+
+            part.restoreState(state == GridDhtPartitionState.EVICTED ? GridDhtPartitionState.RENTING : state);
+
+            return true;
+        }
+
+        return false;
+    }
+
     /**
      * Check that we need to snapshot this partition and add it to map.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index 4966bca..7fc70d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -697,21 +697,11 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * Creates file with current timestamp and specific "node-started.bin" suffix
-     * and writes into memory recovery pointer.
-     *
-     * @param ptr Memory recovery wal pointer.
-     */
-    public void nodeStart(@Nullable WALPointer ptr) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /**
      * @param memPlcName Name of {@link DataRegion} to obtain {@link DataRegionMetrics} for.
      * @return {@link DataRegionMetrics} snapshot for specified {@link DataRegion} or {@code null} if
      * no {@link DataRegion} is configured for specified name.
      */
-    @Nullable public DataRegionMetrics memoryMetrics(String memPlcName) {
+    public @Nullable DataRegionMetrics memoryMetrics(String memPlcName) {
         if (!F.isEmpty(memMetricsMap)) {
             DataRegionMetrics memMetrics = memMetricsMap.get(memPlcName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
index 6ea7e00..5e59178 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
@@ -80,7 +80,7 @@ public class FileWALPointer implements WALPointer, Comparable<FileWALPointer> {
     }
 
     /** {@inheritDoc} */
-    @Override public WALPointer next() {
+    @Override public FileWALPointer next() {
         if (len == 0)
             throw new IllegalStateException("Failed to calculate next WAL pointer " +
                 "(this pointer is a terminal): " + this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
index 183e147..926e403 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
@@ -230,7 +230,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
         if (encSpi instanceof NoopEncryptionSpi)
             return false;
 
-        if (!(rec instanceof WalRecordCacheGroupAware) || rec instanceof MetastoreDataRecord)
+        if (!(rec instanceof WalRecordCacheGroupAware))
             return false;
 
         return needEncryption(((WalRecordCacheGroupAware)rec).groupId());


[38/50] [abbrv] ignite git commit: IGNITE-9811 Web Console: Do not block "Save" button in case some inputs are invalid and show validation when clicked.

Posted by sb...@apache.org.
IGNITE-9811 Web Console: Do not block "Save" button in case some inputs are invalid and show validation when clicked.


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

Branch: refs/heads/ignite-10044
Commit: 070819a528dfe944f39cee8f08331891820242f7
Parents: 4fb1fc3
Author: Ilya Borisov <kl...@gmail.com>
Authored: Tue Nov 20 17:34:38 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Nov 20 17:34:38 2018 +0700

----------------------------------------------------------------------
 .../components/form-field-size/controller.js    | 19 +++++++++++++++++-
 .../components/form-field-size/template.pug     |  1 +
 .../form-field/showValidationError.directive.js | 21 +++++++++++++++++---
 .../page-configure/components/pcValidation.js   | 18 ++++++++++++++++-
 .../app/components/page-profile/controller.js   | 12 +++++++++--
 .../app/components/page-profile/template.pug    |  5 +----
 .../app/components/page-signin/controller.ts    |  3 +++
 7 files changed, 68 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/070819a5/modules/web-console/frontend/app/components/form-field/components/form-field-size/controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/form-field/components/form-field-size/controller.js b/modules/web-console/frontend/app/components/form-field/components/form-field-size/controller.js
index 7f2f1b1..e158c0a 100644
--- a/modules/web-console/frontend/app/components/form-field/components/form-field-size/controller.js
+++ b/modules/web-console/frontend/app/components/form-field/components/form-field-size/controller.js
@@ -67,7 +67,8 @@ export default class PCFormFieldSizeController {
     }
 
     $onDestroy() {
-        this.$element = null;
+        delete this.$element[0].focus;
+        this.$element = this.inputElement = null;
     }
 
     $onInit() {
@@ -84,6 +85,8 @@ export default class PCFormFieldSizeController {
             this.ngModel.$validators.max = (value) => this.ngModel.$isEmpty(value) || value === void 0 || value <= this.max;
 
         this.ngModel.$validators.step = (value) => this.ngModel.$isEmpty(value) || value === void 0 || Math.floor(value) === value;
+        this.inputElement = this.$element[0].querySelector('input');
+        this.$element[0].focus = () => this.inputElement.focus();
     }
 
     $onChanges(changes) {
@@ -143,4 +146,18 @@ export default class PCFormFieldSizeController {
         this.sizesMenu = PCFormFieldSizeController.sizeTypes.bytes;
         this.sizeScale = this.chooseSizeScale();
     }
+
+    notifyAboutError() {
+        if (this.$element)
+            this.$element.find('.form-field__error [bs-tooltip]').trigger('mouseenter');
+    }
+
+    hideError() {
+        if (this.$element)
+            this.$element.find('.form-field__error [bs-tooltip]').trigger('mouseleave');
+    }
+
+    triggerBlur() {
+        this.$element[0].dispatchEvent(new FocusEvent('blur', {relatedTarget: this.inputElement}));
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/070819a5/modules/web-console/frontend/app/components/form-field/components/form-field-size/template.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/form-field/components/form-field-size/template.pug b/modules/web-console/frontend/app/components/form-field/components/form-field-size/template.pug
index 147440b..6a2974d 100644
--- a/modules/web-console/frontend/app/components/form-field/components/form-field-size/template.pug
+++ b/modules/web-console/frontend/app/components/form-field/components/form-field-size/template.pug
@@ -40,6 +40,7 @@ include /app/helpers/jade/mixins
         ng-required='$ctrl.required'
         ng-disabled='$ctrl.ngDisabled'
         ignite-form-field-input-autofocus='{{$ctrl.autofocus}}'
+        ng-on-blur='$ctrl.triggerBlur()'
     )
     button.select-toggle(
         bs-select

http://git-wip-us.apache.org/repos/asf/ignite/blob/070819a5/modules/web-console/frontend/app/components/form-field/showValidationError.directive.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/form-field/showValidationError.directive.js b/modules/web-console/frontend/app/components/form-field/showValidationError.directive.js
index 8720c56..31a8f2d 100644
--- a/modules/web-console/frontend/app/components/form-field/showValidationError.directive.js
+++ b/modules/web-console/frontend/app/components/form-field/showValidationError.directive.js
@@ -21,8 +21,17 @@
  */
 export function directive($timeout) {
     return {
-        require: ['ngModel', '?^^bsCollapseTarget', '?^^igniteFormField', '?^^panelCollapsible'],
-        link(scope, el, attr, [ngModel, bsCollapseTarget, igniteFormField, panelCollapsible]) {
+        require: ['ngModel', '?^^bsCollapseTarget', '?^^igniteFormField', '?formFieldSize', '?^^panelCollapsible'],
+        link(scope, el, attr, [ngModel, bsCollapseTarget, igniteFormField, formFieldSize, panelCollapsible]) {
+            const formFieldController = igniteFormField || formFieldSize;
+
+            let onBlur;
+
+            scope.$on('$destroy', () => {
+                el[0].removeEventListener('blur', onBlur);
+                onBlur = null;
+            });
+
             const off = scope.$on('$showValidationError', (e, target) => {
                 if (target !== ngModel)
                     return;
@@ -32,6 +41,12 @@ export function directive($timeout) {
                 bsCollapseTarget && bsCollapseTarget.open();
                 panelCollapsible && panelCollapsible.open();
 
+                if (!onBlur && formFieldController) {
+                    onBlur = () => formFieldController.hideError();
+
+                    el[0].addEventListener('blur', onBlur, {passive: true});
+                }
+
                 $timeout(() => {
                     if (el[0].scrollIntoViewIfNeeded)
                         el[0].scrollIntoViewIfNeeded();
@@ -41,7 +56,7 @@ export function directive($timeout) {
                     if (!attr.bsSelect)
                         $timeout(() => el[0].focus());
 
-                    igniteFormField && igniteFormField.notifyAboutError();
+                    formFieldController && formFieldController.notifyAboutError();
                 });
             });
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/070819a5/modules/web-console/frontend/app/components/page-configure/components/pcValidation.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/page-configure/components/pcValidation.js b/modules/web-console/frontend/app/components/page-configure/components/pcValidation.js
index 1702c35..697e9e9 100644
--- a/modules/web-console/frontend/app/components/page-configure/components/pcValidation.js
+++ b/modules/web-console/frontend/app/components/page-configure/components/pcValidation.js
@@ -21,21 +21,37 @@ export class IgniteFormField {
     static animName = 'ignite-form-field__error-blink';
     static eventName = 'webkitAnimationEnd oAnimationEnd msAnimationEnd animationend';
     static $inject = ['$element', '$scope'];
+
     constructor($element, $scope) {
         Object.assign(this, {$element});
         this.$scope = $scope;
     }
+
     $postLink() {
         this.onAnimEnd = () => this.$element.removeClass(IgniteFormField.animName);
         this.$element.on(IgniteFormField.eventName, this.onAnimEnd);
     }
+
     $onDestroy() {
         this.$element.off(IgniteFormField.eventName, this.onAnimEnd);
         this.$element = this.onAnimEnd = null;
     }
+
     notifyAboutError() {
-        if (this.$element) this.$element.addClass(IgniteFormField.animName);
+        if (!this.$element)
+            return;
+
+        this.$element.addClass(IgniteFormField.animName);
+        this.$element.find('.form-field__error [bs-tooltip]').trigger('mouseenter');
     }
+
+    hideError() {
+        if (!this.$element)
+            return;
+
+        this.$element.find('.form-field__error [bs-tooltip]').trigger('mouseleave');
+    }
+
     /**
      * Exposes control in $scope
      * @param {ng.INgModelController} control

http://git-wip-us.apache.org/repos/asf/ignite/blob/070819a5/modules/web-console/frontend/app/components/page-profile/controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/page-profile/controller.js b/modules/web-console/frontend/app/components/page-profile/controller.js
index 8586656..a2f344e 100644
--- a/modules/web-console/frontend/app/components/page-profile/controller.js
+++ b/modules/web-console/frontend/app/components/page-profile/controller.js
@@ -19,7 +19,7 @@ import _ from 'lodash';
 
 export default class PageProfileController {
     static $inject = [
-        '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteCountries', 'User'
+        '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteCountries', 'User', 'IgniteFormUtils'
     ];
 
     /**
@@ -32,8 +32,9 @@ export default class PageProfileController {
      * @param {import('app/services/Confirm.service').Confirm} Confirm
      * @param {ReturnType<typeof import('app/services/Countries.service').default>} Countries
      * @param {ReturnType<typeof import('app/modules/user/User.service').default>} User
+     * @param {ReturnType<typeof import('app/services/FormUtils.service').default>} FormUtils
      */
-    constructor($root, $scope, $http, LegacyUtils, Messages, Focus, Confirm, Countries, User) {
+    constructor($root, $scope, $http, LegacyUtils, Messages, Focus, Confirm, Countries, User, FormUtils) {
         this.$root = $root;
         this.$scope = $scope;
         this.$http = $http;
@@ -43,6 +44,7 @@ export default class PageProfileController {
         this.Confirm = Confirm;
         this.Countries = Countries;
         this.User = User;
+        this.FormUtils = FormUtils;
     }
 
     $onInit() {
@@ -69,6 +71,12 @@ export default class PageProfileController {
     }
 
     saveUser() {
+        if (this.form.$invalid) {
+            this.FormUtils.triggerValidation(this.form);
+
+            return;
+        }
+
         return this.$http.post('/api/v1/profile/save', this.ui.user)
             .then(this.User.load)
             .then(() => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/070819a5/modules/web-console/frontend/app/components/page-profile/template.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/page-profile/template.pug b/modules/web-console/frontend/app/components/page-profile/template.pug
index 3c5fb52..68c321f 100644
--- a/modules/web-console/frontend/app/components/page-profile/template.pug
+++ b/modules/web-console/frontend/app/components/page-profile/template.pug
@@ -154,9 +154,6 @@ div
 
     footer
         a.btn-ignite.btn-ignite--link-success(type='button' ui-sref='default-state') Cancel
-        button.btn-ignite.btn-ignite--success(
-            ng-click='$ctrl.saveUser()'
-            ng-disabled='$ctrl.form.$invalid'
-        )
+        button.btn-ignite.btn-ignite--success(ng-click='$ctrl.saveUser()')
             svg.icon-left(ignite-icon='checkmark')
             | Save Changes

http://git-wip-us.apache.org/repos/asf/ignite/blob/070819a5/modules/web-console/frontend/app/components/page-signin/controller.ts
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/page-signin/controller.ts b/modules/web-console/frontend/app/components/page-signin/controller.ts
index 724da41..c461b86 100644
--- a/modules/web-console/frontend/app/components/page-signin/controller.ts
+++ b/modules/web-console/frontend/app/components/page-signin/controller.ts
@@ -66,7 +66,10 @@ export default class {
 
         return this.Auth.signin(this.data.email, this.data.password).catch((res) => {
             this.IgniteMessages.showError(null, res.data);
+
             this.setServerError(res.data);
+
+            this.IgniteFormUtils.triggerValidation(this.form);
         });
     }
 }


[14/50] [abbrv] ignite git commit: IGNITE-10278: MVCC: Checkpoint read lock must be acquired before calling MvccProcessor.updateState. This closes #5401.

Posted by sb...@apache.org.
IGNITE-10278: MVCC: Checkpoint read lock must be acquired before calling MvccProcessor.updateState. This closes #5401.


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

Branch: refs/heads/ignite-10044
Commit: 43ee832482b6cd9237970c097ffc15dadc2c3eb5
Parents: 8526add
Author: Slava Koptilin <sl...@gmail.com>
Authored: Sat Nov 17 15:18:08 2018 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Sat Nov 17 15:18:08 2018 +0300

----------------------------------------------------------------------
 .../cache/persistence/GridCacheDatabaseSharedManager.java       | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/43ee8324/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 393e137..019d1aa 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -2368,6 +2368,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             break;
 
                         case MVCC_TX_RECORD:
+                            checkpointReadLock();
+
                             try {
                                 MvccTxRecord txRecord = (MvccTxRecord)rec;
 
@@ -2378,6 +2380,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             catch (IgniteCheckedException e) {
                                 throw new IgniteException(e);
                             }
+                            finally {
+                                checkpointReadUnlock();
+                            }
 
                             break;
 


[05/50] [abbrv] ignite git commit: IGNITE-10273: Thin clents now getting the last affinity mapping in all cases.

Posted by sb...@apache.org.
IGNITE-10273: Thin clents now getting the last affinity mapping in all
cases.

This closes #5399


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

Branch: refs/heads/ignite-10044
Commit: 64b19a99abdc4e253d0aeb53ba860c8a2f30c51d
Parents: eed26bc
Author: Igor Sapego <is...@apache.org>
Authored: Fri Nov 16 15:40:24 2018 +0300
Committer: Igor Sapego <is...@apache.org>
Committed: Fri Nov 16 15:40:24 2018 +0300

----------------------------------------------------------------------
 .../cache/ClientCacheNodePartitionsRequest.java |  3 +--
 .../thin-client-test/src/cache_client_test.cpp  | 27 ++++++++++++++++++++
 2 files changed, 28 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/64b19a99/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java
index b9bf80e..377d26f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java
@@ -50,8 +50,7 @@ public class ClientCacheNodePartitionsRequest extends ClientCacheRequest {
         IgniteCache cache = cache(ctx);
 
         GridDiscoveryManager discovery = ctx.kernalContext().discovery();
-        Collection<ClusterNode> nodes = discovery.cacheNodes(cache.getName(),
-            new AffinityTopologyVersion(discovery.topologyVersion()));
+        Collection<ClusterNode> nodes = discovery.discoCache().cacheNodes(cache.getName());
 
         Affinity aff = ctx.kernalContext().affinity().affinityProxy(cache.getName());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/64b19a99/modules/platforms/cpp/thin-client-test/src/cache_client_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/thin-client-test/src/cache_client_test.cpp b/modules/platforms/cpp/thin-client-test/src/cache_client_test.cpp
index 24ff48f..4031d46 100644
--- a/modules/platforms/cpp/thin-client-test/src/cache_client_test.cpp
+++ b/modules/platforms/cpp/thin-client-test/src/cache_client_test.cpp
@@ -814,6 +814,33 @@ BOOST_AUTO_TEST_CASE(CacheClientDefaultDynamicCache)
     }
 }
 
+BOOST_AUTO_TEST_CASE(CacheClientDefaultDynamicCacheThreeNodes)
+{
+    StartNode("node1");
+    StartNode("node2");
+
+    IgniteClientConfiguration cfg;
+    cfg.SetEndPoints("127.0.0.1:11110..11120");
+
+    IgniteClient client = IgniteClient::Start(cfg);
+
+    cache::CacheClient<std::string, int64_t> cache =
+        client.CreateCache<std::string, int64_t>("defaultdynamic3");
+
+    cache.RefreshAffinityMapping();
+
+    for (int64_t i = 1; i < 1000; ++i)
+        cache.Put(ignite::common::LexicalCast<std::string>(i * 39916801), i * 5039);
+
+    for (int64_t i = 1; i < 1000; ++i)
+    {
+        int64_t val;
+        LocalPeek(cache, ignite::common::LexicalCast<std::string>(i * 39916801), val);
+
+        BOOST_CHECK_EQUAL(val, i * 5039);
+    }
+}
+
 BOOST_AUTO_TEST_CASE(CacheClientGetAllContainers)
 {
     IgniteClientConfiguration cfg;


[07/50] [abbrv] ignite git commit: merge

Posted by sb...@apache.org.
 merge


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

Branch: refs/heads/ignite-10044
Commit: bc1ac15c36fb459f221c9b723ad82a6e43a5bfca
Parents: fb8fd07
Author: sboikov <sb...@apache.org>
Authored: Fri Nov 16 16:38:07 2018 +0300
Committer: sboikov <sb...@apache.org>
Committed: Fri Nov 16 16:38:07 2018 +0300

----------------------------------------------------------------------
 .../distributed/IgniteCachePartitionLossPolicySelfTest.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bc1ac15c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
index f02563d..322a79f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
@@ -553,12 +553,12 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
 
             for (Map<Integer, Semaphore> map : lostMap) {
                 for (Map.Entry<Integer, Semaphore> entry : map.entrySet())
-                    assertTrue("Failed to wait for partition LOST event for partition:" + entry.getKey(), entry.getValue().tryAcquire(1));
+                    assertTrue("Failed to wait for partition LOST event for partition: " + entry.getKey(), entry.getValue().tryAcquire(1));
             }
 
             for (Map<Integer, Semaphore> map : lostMap) {
                 for (Map.Entry<Integer, Semaphore> entry : map.entrySet())
-                    assertFalse("Partition LOST event raised twice for partition:" + entry.getKey(), entry.getValue().tryAcquire(1));
+                    assertFalse("Partition LOST event raised twice for partition: " + entry.getKey(), entry.getValue().tryAcquire(1));
             }
 
             return parts;


[32/50] [abbrv] ignite git commit: IGNITE-9339 Form-field-size improvements. - Fixes #5440.

Posted by sb...@apache.org.
IGNITE-9339 Form-field-size improvements. - Fixes #5440.

Signed-off-by: Andrey Novikov <an...@gridgain.com>


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

Branch: refs/heads/ignite-10044
Commit: 7c632e68d4b48480b22cdc950c67944415b8bd5e
Parents: 351d080
Author: Ilya Borisov <Kl...@gmail.com>
Authored: Tue Nov 20 11:30:03 2018 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Tue Nov 20 11:30:03 2018 +0700

----------------------------------------------------------------------
 .../web-console/e2e/testcafe/components/FormField.js | 11 +++++------
 .../components/form-field-size/controller.js         |  6 ++++++
 .../form-field/components/form-field-size/index.js   |  3 ++-
 .../form-field/components/form-field-size/style.scss | 15 +++++++++++++++
 .../components/form-field-size/template.pug          |  1 +
 5 files changed, 29 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7c632e68/modules/web-console/e2e/testcafe/components/FormField.js
----------------------------------------------------------------------
diff --git a/modules/web-console/e2e/testcafe/components/FormField.js b/modules/web-console/e2e/testcafe/components/FormField.js
index b0e8967..0c6c4eb 100644
--- a/modules/web-console/e2e/testcafe/components/FormField.js
+++ b/modules/web-console/e2e/testcafe/components/FormField.js
@@ -32,13 +32,12 @@ export class FormField {
         if (id)
             this._selector = Selector(`#${id}`).parent(this.constructor.ROOT_SELECTOR);
         else if (label) {
-            this._selector = Selector(() => {
-                return Array
-                    .from(window.document.querySelectorAll(this.constructor.LABEL_SELECTOR))
-                    .filter((el) => el.textContent.contains(label))
-                    .map((el) => el.parent(this.constructor.ROOT_SELECTOR))
+            this._selector = Selector((LABEL_SELECTOR, ROOT_SELECTOR, label) => {
+                return [].slice.call((window.document.querySelectorAll(LABEL_SELECTOR)))
+                    .filter((el) => el.textContent.includes(label))
+                    .map((el) => el.closest(ROOT_SELECTOR))
                     .pop();
-            });
+            })(this.constructor.LABEL_SELECTOR, this.constructor.ROOT_SELECTOR, label);
         } else if (model)
             this._selector = AngularJSSelector.byModel(model).parent(this.constructor.ROOT_SELECTOR);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7c632e68/modules/web-console/frontend/app/components/form-field/components/form-field-size/controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/form-field/components/form-field-size/controller.js b/modules/web-console/frontend/app/components/form-field/components/form-field-size/controller.js
index 7663e9d..7f2f1b1 100644
--- a/modules/web-console/frontend/app/components/form-field/components/form-field-size/controller.js
+++ b/modules/web-console/frontend/app/components/form-field/components/form-field-size/controller.js
@@ -28,6 +28,8 @@ export default class PCFormFieldSizeController {
     onScaleChange;
     /** @type {ng.IFormController} */
     innerForm;
+    /** @type {boolean?} */
+    autofocus;
 
     static $inject = ['$element', '$attrs'];
 
@@ -38,6 +40,10 @@ export default class PCFormFieldSizeController {
             {label: 'Mb', value: 1024 * 1024},
             {label: 'Gb', value: 1024 * 1024 * 1024}
         ],
+        gigabytes: [
+            {label: 'Gb', value: 1},
+            {label: 'Tb', value: 1024}
+        ],
         seconds: [
             {label: 'ns', value: 1 / 1000},
             {label: 'ms', value: 1},

http://git-wip-us.apache.org/repos/asf/ignite/blob/7c632e68/modules/web-console/frontend/app/components/form-field/components/form-field-size/index.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/form-field/components/form-field-size/index.js b/modules/web-console/frontend/app/components/form-field/components/form-field-size/index.js
index 5e08df2..e7da7d5 100644
--- a/modules/web-console/frontend/app/components/form-field/components/form-field-size/index.js
+++ b/modules/web-console/frontend/app/components/form-field/components/form-field-size/index.js
@@ -36,6 +36,7 @@ export default {
         sizeType: '@?',
         sizeScaleLabel: '@?',
         onScaleChange: '&?',
-        ngDisabled: '<?'
+        ngDisabled: '<?',
+        autofocus: '<?'
     }
 };

http://git-wip-us.apache.org/repos/asf/ignite/blob/7c632e68/modules/web-console/frontend/app/components/form-field/components/form-field-size/style.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/form-field/components/form-field-size/style.scss b/modules/web-console/frontend/app/components/form-field/components/form-field-size/style.scss
index c522200..8b86e05 100644
--- a/modules/web-console/frontend/app/components/form-field/components/form-field-size/style.scss
+++ b/modules/web-console/frontend/app/components/form-field/components/form-field-size/style.scss
@@ -16,5 +16,20 @@
  */
 
 form-field-size {
+    --scale-select-width: 60px;
+    --error-area-width: 36px;
+    --default-input-padding-horizontal: 10px;
+
 	display: block;
+
+    .form-field__error {
+        width: var(--error-area-width);
+        position: absolute;
+        right: var(--scale-select-width);
+        bottom: 0;
+    }
+
+    &.ng-invalid input, input.ng-invalid {
+        padding-right: var(--error-area-width);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7c632e68/modules/web-console/frontend/app/components/form-field/components/form-field-size/template.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/form-field/components/form-field-size/template.pug b/modules/web-console/frontend/app/components/form-field/components/form-field-size/template.pug
index b712a67..147440b 100644
--- a/modules/web-console/frontend/app/components/form-field/components/form-field-size/template.pug
+++ b/modules/web-console/frontend/app/components/form-field/components/form-field-size/template.pug
@@ -39,6 +39,7 @@ include /app/helpers/jade/mixins
         max='{{ $ctrl.max ? $ctrl.max / $ctrl.sizeScale.value : "" }}'
         ng-required='$ctrl.required'
         ng-disabled='$ctrl.ngDisabled'
+        ignite-form-field-input-autofocus='{{$ctrl.autofocus}}'
     )
     button.select-toggle(
         bs-select


[08/50] [abbrv] ignite git commit: IGNITE-12364 Reworked Sending Full Message logging - Fixes #5413.

Posted by sb...@apache.org.
IGNITE-12364 Reworked Sending Full Message logging - Fixes #5413.

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/34c059a1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/34c059a1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/34c059a1

Branch: refs/heads/ignite-10044
Commit: 34c059a15b3505c8937045b2b973874286bc148a
Parents: 64b19a9
Author: Pavel Voronkin <pv...@gridgain.com>
Authored: Fri Nov 16 18:56:28 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Nov 16 18:59:31 2018 +0300

----------------------------------------------------------------------
 .../preloader/GridDhtPartitionsExchangeFuture.java   | 10 ----------
 .../spi/communication/tcp/TcpCommunicationSpi.java   | 15 +++++++++++++--
 2 files changed, 13 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/34c059a1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index c8471c7..f762d3d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -1910,8 +1910,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             .map(singleMessage -> fullMsg.copy().joinedNodeAffinity(affinityForJoinedNodes))
             .orElse(null);
 
-        long time = System.currentTimeMillis();
-
         // Prepare and send full messages for given nodes.
         nodes.stream()
             .map(node -> {
@@ -1964,9 +1962,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     U.error(log, "Failed to send partitions [node=" + node + ']', e);
                 }
             });
-
-        if (log.isInfoEnabled())
-            log.info("Sending Full Message performed in " + (System.currentTimeMillis() - time) + " ms.");
     }
 
     /**
@@ -3377,16 +3372,11 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         nodes.addAll(sndResNodes);
                 }
 
-                time = System.currentTimeMillis();
-
                 if (!nodes.isEmpty())
                     sendAllPartitions(msg, nodes, mergedJoinExchMsgs0, joinedNodeAff);
 
                 partitionsSent = true;
 
-                if (log.isInfoEnabled())
-                    log.info("Sending Full Message to all nodes performed in " + (System.currentTimeMillis() - time) + " ms.");
-
                 if (!stateChangeExchange())
                     onDone(exchCtx.events().topologyVersion(), null);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/34c059a1/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 427db08..3e3f7f9 100755
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -269,6 +269,9 @@ import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastRecei
 @IgniteSpiMultipleInstancesSupport(true)
 @IgniteSpiConsistencyChecked(optional = false)
 public class TcpCommunicationSpi extends IgniteSpiAdapter implements CommunicationSpi<Message> {
+    /** Time threshold to log too long connection establish. */
+    private static final int CONNECTION_ESTABLISH_THRESHOLD_MS = 100;
+
     /** IPC error message. */
     public static final String OUT_OF_RESOURCES_TCP_MSG = "Failed to allocate shared memory segment " +
         "(switching to TCP, may be slower).";
@@ -2983,10 +2986,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
         connectGate.enter();
 
         try {
+            final long start = System.currentTimeMillis();
+
             GridCommunicationClient client = createTcpClient(node, connIdx);
 
-            if (log.isDebugEnabled())
-                log.debug("TCP client created: " + client);
+            final long time = System.currentTimeMillis() - start;
+
+            if (time > CONNECTION_ESTABLISH_THRESHOLD_MS) {
+                if (log.isInfoEnabled())
+                    log.info("TCP client created [client=" + client + ", duration=" + time + "ms]");
+            }
+            else if (log.isDebugEnabled())
+                log.debug("TCP client created [client=" + client + ", duration=" + time + "ms]");
 
             return client;
         }


[41/50] [abbrv] ignite git commit: IGNITE-10217: SQL: store unwrapped key fields in secondary indexes instead of full key when possible. Closes #5407.

Posted by sb...@apache.org.
IGNITE-10217: SQL: store unwrapped key fields in secondary indexes instead of full key when possible. Closes #5407.


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

Branch: refs/heads/ignite-10044
Commit: a9b5c8c8aee9085f082fd524e288949853dd81f1
Parents: c8fd687
Author: Yuriy Gerzhedovich <yg...@gridgain.com>
Authored: Tue Nov 20 14:47:15 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Nov 20 14:47:15 2018 +0300

----------------------------------------------------------------------
 .../IgnitePKIndexesMigrationToUnwrapPkTest.java |   6 +-
 .../processors/query/h2/H2TableDescriptor.java  |  28 ++-
 .../internal/processors/query/h2/H2Utils.java   |  13 +
 .../index/ComplexPrimaryKeyUnwrapSelfTest.java  |  18 +-
 .../ComplexSecondaryKeyUnwrapSelfTest.java      | 249 +++++++++++++++++++
 .../IgniteBinaryCacheQueryTestSuite.java        |   2 +
 6 files changed, 300 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a9b5c8c8/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java
index 22ccae0..316e574 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java
@@ -145,10 +145,12 @@ public class IgnitePKIndexesMigrationToUnwrapPkTest extends IgnitePersistenceCom
      */
     @NotNull private static void initializeTable(IgniteEx igniteEx, String tblName) {
         executeSql(igniteEx, "CREATE TABLE " + tblName + " (id int, name varchar, age int, company varchar, city varchar, " +
-            "primary key (id, name, city))");
+            "primary key (id, name, city)) WITH \"affinity_key=name\"");
 
-        executeSql(igniteEx, "INSERT INTO " + tblName + " (id, name, age, company, city) VALUES(1,'name',2,'company', 'city')");
+        executeSql(igniteEx, "CREATE INDEX ON " + tblName + "(city, age)");
 
+        for (int i = 0; i < 1000; i++)
+            executeSql(igniteEx, "INSERT INTO " + tblName + " (id, name, age, company, city) VALUES(?,'name',2,'company', 'city')", i);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9b5c8c8/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
index aa66939..6c20727 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
@@ -204,7 +204,7 @@ public class H2TableDescriptor implements GridH2SystemIndexFactory {
         if (affCol != null && H2Utils.equals(affCol, keyCol))
             affCol = null;
 
-        List<IndexColumn> unwrappedKeyCols = extractKeyColumns(tbl, keyCol, affCol);
+        List<IndexColumn> unwrappedKeyAndAffinityCols = extractKeyColumns(tbl, keyCol, affCol);
 
         List<IndexColumn> wrappedKeyCols = H2Utils.treeIndexColumns(tbl.rowDescriptor(),
             new ArrayList<>(2), keyCol, affCol);
@@ -224,7 +224,7 @@ public class H2TableDescriptor implements GridH2SystemIndexFactory {
             tbl,
             true,
             false,
-            unwrappedKeyCols,
+            unwrappedKeyAndAffinityCols,
             wrappedKeyCols,
             -1
         );
@@ -271,14 +271,24 @@ public class H2TableDescriptor implements GridH2SystemIndexFactory {
 
             // Add explicit affinity key index if nothing alike was found.
             if (!affIdxFound) {
-                List<IndexColumn> columns = H2Utils.treeIndexColumns(tbl.rowDescriptor(), new ArrayList<>(2), affCol, keyCol);
+                List<IndexColumn> unwrappedKeyCols = extractKeyColumns(tbl, keyCol, null);
+
+                ArrayList<IndexColumn> colsWithUnwrappedKey = new ArrayList<>(unwrappedKeyCols.size());
+
+                colsWithUnwrappedKey.add(affCol);
+
+                //We need to reorder PK columns to have affinity key as first column, that's why we can't use simple PK columns
+                H2Utils.addUniqueColumns(colsWithUnwrappedKey, unwrappedKeyCols);
+
+                List<IndexColumn> cols = H2Utils.treeIndexColumns(tbl.rowDescriptor(), new ArrayList<>(2), affCol, keyCol);
+
                 idxs.add(idx.createSortedIndex(
                     AFFINITY_KEY_IDX_NAME,
                     tbl,
                     false,
                     true,
-                    columns,
-                    columns,
+                    colsWithUnwrappedKey,
+                    cols,
                     -1)
                 );
             }
@@ -378,6 +388,12 @@ public class H2TableDescriptor implements GridH2SystemIndexFactory {
         GridH2RowDescriptor desc = tbl.rowDescriptor();
 
         if (idxDesc.type() == QueryIndexType.SORTED) {
+            List<IndexColumn> unwrappedKeyCols = extractKeyColumns(tbl, keyCol, affCol);
+
+            List<IndexColumn> colsWithUnwrappedKey = new ArrayList<>(cols);
+
+            H2Utils.addUniqueColumns(colsWithUnwrappedKey, unwrappedKeyCols);
+
             cols = H2Utils.treeIndexColumns(desc, cols, keyCol, affCol);
 
             return idx.createSortedIndex(
@@ -385,7 +401,7 @@ public class H2TableDescriptor implements GridH2SystemIndexFactory {
                 tbl,
                 false,
                 false,
-                cols,
+                colsWithUnwrappedKey,
                 cols,
                 idxDesc.inlineSize()
             );

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9b5c8c8/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
index b9d9d8e..865754f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
@@ -285,4 +285,17 @@ public class H2Utils {
 
         return resCur;
     }
+
+    /**
+     * Add only new columns to destination list.
+     *
+     * @param dest List of index columns to add new elements from src list.
+     * @param src List of IndexColumns to add to dest list.
+     */
+    public static void addUniqueColumns(List<IndexColumn> dest, List<IndexColumn> src) {
+        for (IndexColumn col : src) {
+            if (!containsColumn(dest, col))
+                dest.add(col);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9b5c8c8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ComplexPrimaryKeyUnwrapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ComplexPrimaryKeyUnwrapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ComplexPrimaryKeyUnwrapSelfTest.java
index 8f4eaf0..376a0d4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ComplexPrimaryKeyUnwrapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ComplexPrimaryKeyUnwrapSelfTest.java
@@ -88,18 +88,19 @@ public class ComplexPrimaryKeyUnwrapSelfTest extends GridCommonAbstractTest {
                 put("uuid", "'1'");
             }
         };
+
         for (Map.Entry<String, String> entry : types.entrySet()) {
 
             String tblName = createTableName();
 
             String type = entry.getKey();
-            String value = entry.getValue();
+            String val = entry.getValue();
 
             executeSql("CREATE TABLE " + tblName +
                 " (id " + type + " , name varchar, age int, company varchar, city varchar," +
                 " primary key (id))");
 
-            checkUsingIndexes(tblName, value);
+            checkUsingIndexes(tblName, val);
         }
     }
 
@@ -128,18 +129,19 @@ public class ComplexPrimaryKeyUnwrapSelfTest extends GridCommonAbstractTest {
                 put("uuid", "'1'");
             }
         };
+
         for (Map.Entry<String, String> entry : types.entrySet()) {
 
             String tblName = createTableName();
 
             String type = entry.getKey();
-            String value = entry.getValue();
+            String val = entry.getValue();
 
             executeSql("CREATE TABLE " + tblName +
                 " (id " + type + " , name varchar, age int, company varchar, city varchar," +
                 " primary key (id)) WITH \"affinity_key=id\"");
 
-            checkUsingIndexes(tblName, value);
+            checkUsingIndexes(tblName, val);
         }
     }
 
@@ -160,18 +162,18 @@ public class ComplexPrimaryKeyUnwrapSelfTest extends GridCommonAbstractTest {
      *
      * @param tblName name of table which should be checked to using PK indexes.
      */
-    private void checkUsingIndexes(String tblName, String idValue) {
+    private void checkUsingIndexes(String tblName, String idVal) {
         String explainSQL = "explain SELECT * FROM " + tblName + " WHERE ";
 
-        List<List<?>> results = executeSql(explainSQL + "id=" + idValue);
+        List<List<?>> results = executeSql(explainSQL + "id=" + idVal);
 
         assertUsingPkIndex(results);
 
-        results = executeSql(explainSQL + "id=" + idValue + " and name=''");
+        results = executeSql(explainSQL + "id=" + idVal + " and name=''");
 
         assertUsingPkIndex(results);
 
-        results = executeSql(explainSQL + "id=" + idValue + " and name='' and city='' and age=0");
+        results = executeSql(explainSQL + "id=" + idVal + " and name='' and city='' and age=0");
 
         assertUsingPkIndex(results);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9b5c8c8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ComplexSecondaryKeyUnwrapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ComplexSecondaryKeyUnwrapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ComplexSecondaryKeyUnwrapSelfTest.java
new file mode 100644
index 0000000..fe3cbfa
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ComplexSecondaryKeyUnwrapSelfTest.java
@@ -0,0 +1,249 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test of creating and using secondary indexes for tables created through SQL.
+ */
+@SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
+public class ComplexSecondaryKeyUnwrapSelfTest extends GridCommonAbstractTest {
+
+    /** Counter to generate unique table names. */
+    private static int tblCnt = 0;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * Test secondary index with complex PK. Columns for secondary and PK indexes are intersect.
+     */
+    public void testSecondaryIndexWithIntersectColumnsComplexPk() {
+        String tblName = createTableName();
+
+        executeSql("CREATE TABLE " + tblName + " (id int, name varchar, age int, company varchar, city varchar, " +
+            "primary key (name, city))");
+
+        executeSql("CREATE INDEX ON " + tblName + "(id, name, city)");
+
+        checkUsingIndexes(tblName, "'1'");
+    }
+
+    /**
+     * Test using secondary index with simple PK.
+     */
+    public void testSecondaryIndexSimplePk() {
+        HashMap<String, String> types = new HashMap() {
+            {
+                put("boolean", "1");
+                put("char", "'1'");
+                put("varchar", "'1'");
+                put("real", "1");
+                put("number", "1");
+                put("int", "1");
+                put("long", "1");
+                put("float", "1");
+                put("double", "1");
+                put("tinyint", "1");
+                put("smallint", "1");
+                put("bigint", "1");
+                put("varchar_ignorecase", "'1'");
+                put("time", "'11:11:11'");
+                put("timestamp", "'20018-11-02 11:11:11'");
+                put("uuid", "'1'");
+            }
+        };
+
+        for (Map.Entry<String, String> entry : types.entrySet()) {
+
+            String tblName = createTableName();
+
+            String type = entry.getKey();
+            String val = entry.getValue();
+
+            executeSql("CREATE TABLE " + tblName +
+                " (id int, name " + type + ", age int, company varchar, city varchar," +
+                " primary key (name))");
+
+            executeSql("CREATE INDEX ON " + tblName + "(id, name, city)");
+
+            checkUsingIndexes(tblName, val);
+        }
+    }
+
+    /**
+     * Check using secondary indexes for few cases.
+     *
+     * @param tblName name of table which should be checked to using secondary indexes.
+     * @param nameVal Value for name param.
+     */
+    private void checkUsingIndexes(String tblName, String nameVal) {
+        String explainSQL = "explain SELECT * FROM " + tblName + " WHERE ";
+
+        List<List<?>> results = executeSql(explainSQL + "id=1");
+
+        assertUsingSecondaryIndex(results);
+
+        results = executeSql(explainSQL + "id=1 and name=" + nameVal);
+
+        assertUsingSecondaryIndex(results);
+
+        results = executeSql(explainSQL + "id=1 and name=" + nameVal + " and age=0");
+
+        assertUsingSecondaryIndex(results);
+    }
+
+    /**
+     * Check that explain plan result shown using Secondary index and don't use scan.
+     *
+     * @param results result of execut explain plan query.
+     */
+    private void assertUsingSecondaryIndex(List<List<?>> results) {
+        assertEquals(2, results.size());
+
+        String explainPlan = (String)results.get(0).get(0);
+
+        assertTrue(explainPlan, explainPlan.contains("_idx\": "));
+
+        assertFalse(explainPlan, explainPlan.contains("_SCAN_"));
+    }
+
+    /**
+     * Create unique table name.
+     *
+     * @return unique name of table.
+     */
+    private String createTableName() {
+        return "TST_TABLE_" + tblCnt++;
+    }
+
+    /**
+     * Run SQL statement on default node.
+     *
+     * @param stmt Statement to run.
+     * @param args arguments of statements
+     * @return Run result.
+     */
+    private List<List<?>> executeSql(String stmt, Object... args) {
+        return executeSql(node(), stmt, args);
+    }
+
+    /**
+     * Run SQL statement on specified node.
+     *
+     * @param node node to execute query.
+     * @param stmt Statement to run.
+     * @param args arguments of statements
+     * @return Run result.
+     */
+    private List<List<?>> executeSql(IgniteEx node, String stmt, Object... args) {
+        return node.context().query().querySqlFields(new SqlFieldsQuery(stmt).setArgs(args), true).getAll();
+    }
+
+    /**
+     * @return Node to initiate operations from.
+     */
+    private IgniteEx node() {
+        return grid(0);
+    }
+
+    /**
+     *
+     */
+    static class TestKey {
+        /** */
+        @QuerySqlField
+        private int id;
+
+        /**
+         * @param id ID.
+         */
+        public TestKey(int id) {
+            this.id = id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestKey testKey = (TestKey)o;
+
+            return id == testKey.id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return id;
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestValue {
+        /** */
+        @QuerySqlField()
+        private String name;
+        /** */
+        @QuerySqlField()
+        private String company;
+        /** */
+        @QuerySqlField()
+        private String city;
+        /** */
+        @QuerySqlField()
+        private int age;
+
+        /**
+         * @param age Age.
+         * @param name Name.
+         * @param company Company.
+         * @param city City.
+         */
+        public TestValue(int age, String name, String company, String city) {
+            this.age = age;
+            this.name = name;
+            this.company = company;
+            this.city = city;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9b5c8c8/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index ddfeaef..25cb473 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -251,6 +251,8 @@ public class IgniteBinaryCacheQueryTestSuite extends TestSuite {
         suite.addTestSuite(ExplainSelfTest.class);
         suite.addTestSuite(RunningQueriesTest.class);
 
+        suite.addTestSuite(ComplexPrimaryKeyUnwrapSelfTest.class);
+
         suite.addTestSuite(PartitionedSqlTest.class);
         suite.addTestSuite(ReplicatedSqlTest.class);
 


[26/50] [abbrv] ignite git commit: IGNITE-9558 Avoid blocking transactions on client connect when possible - Fixes #4933.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 1726fd6..55c5d6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -22,7 +22,6 @@ import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -35,6 +34,7 @@ import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.TimeUnit;
@@ -135,6 +135,7 @@ import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE;
 import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap.PARTIAL_COUNTERS_MAP_SINCE;
+import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.nextDumpTimeout;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader.DFLT_PRELOAD_RESEND_TIMEOUT;
 
@@ -177,19 +178,23 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     @Nullable private volatile GridDhtPartitionsExchangeFuture lastInitializedFut;
 
     /** */
-    private final AtomicReference<GridDhtTopologyFuture> lastFinishedFut = new AtomicReference<>();
+    private final AtomicReference<GridDhtPartitionsExchangeFuture> lastFinishedFut = new AtomicReference<>();
 
     /** */
     private final ConcurrentMap<AffinityTopologyVersion, AffinityReadyFuture> readyFuts =
         new ConcurrentSkipListMap<>();
 
+    /** */
+    private final ConcurrentNavigableMap<AffinityTopologyVersion, AffinityTopologyVersion> lastAffTopVers =
+        new ConcurrentSkipListMap<>();
+
     /**
      * Latest started rebalance topology version but possibly not finished yet. Value {@code NONE}
      * means that previous rebalance is undefined and the new one should be initiated.
      *
      * Should not be used to determine latest rebalanced topology.
      */
-    private volatile AffinityTopologyVersion rebTopVer = AffinityTopologyVersion.NONE;
+    private volatile AffinityTopologyVersion rebTopVer = NONE;
 
     /** */
     private GridFutureAdapter<?> reconnectExchangeFut;
@@ -900,11 +905,11 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param fut Finished future.
      */
-    public void lastFinishedFuture(GridDhtTopologyFuture fut) {
+    public void lastFinishedFuture(GridDhtPartitionsExchangeFuture fut) {
         assert fut != null && fut.isDone() : fut;
 
         while (true) {
-            GridDhtTopologyFuture cur = lastFinishedFut.get();
+            GridDhtPartitionsExchangeFuture cur = lastFinishedFut.get();
 
             if (fut.topologyVersion() != null && (cur == null || fut.topologyVersion().compareTo(cur.topologyVersion()) > 0)) {
                 if (lastFinishedFut.compareAndSet(cur, fut))
@@ -920,6 +925,17 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * @return Future or {@code null} is future is already completed.
      */
     @Nullable public IgniteInternalFuture<AffinityTopologyVersion> affinityReadyFuture(AffinityTopologyVersion ver) {
+        GridDhtPartitionsExchangeFuture lastInitializedFut0 = lastInitializedFut;
+
+        if (lastInitializedFut0 != null && lastInitializedFut0.initialVersion().compareTo(ver) == 0
+            && lastInitializedFut0.changedAffinity()) {
+            if (log.isTraceEnabled())
+                log.trace("Return lastInitializedFut for topology ready future " +
+                    "[ver=" + ver + ", fut=" + lastInitializedFut0 + ']');
+
+            return lastInitializedFut0;
+        }
+
         AffinityTopologyVersion topVer = exchFuts.readyTopVer();
 
         if (topVer.compareTo(ver) >= 0) {
@@ -985,6 +1001,48 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
+     *
+     * @param topVer Topology version.
+     * @return Last topology version before the provided one when affinity was modified.
+     */
+    public AffinityTopologyVersion lastAffinityChangedTopologyVersion(AffinityTopologyVersion topVer) {
+        if (topVer.topologyVersion() <= 0)
+            return topVer;
+
+        AffinityTopologyVersion lastAffTopVer = lastAffTopVers.get(topVer);
+
+        return lastAffTopVer != null ? lastAffTopVer : topVer;
+    }
+
+    /**
+     *
+     * @param topVer Topology version.
+     * @param lastAffTopVer Last topology version before the provided one when affinity was modified.
+     * @return {@code True} if data was modified.
+     */
+    public boolean lastAffinityChangedTopologyVersion(AffinityTopologyVersion topVer, AffinityTopologyVersion lastAffTopVer) {
+        assert lastAffTopVer.compareTo(topVer) <= 0;
+
+        if (lastAffTopVer.topologyVersion() <= 0 || lastAffTopVer.equals(topVer))
+            return false;
+
+        while (true) {
+            AffinityTopologyVersion old = lastAffTopVers.putIfAbsent(topVer, lastAffTopVer);
+
+            if (old == null)
+                return true;
+
+            if (lastAffTopVer.compareTo(old) < 0) {
+                if (lastAffTopVers.replace(topVer, old, lastAffTopVer))
+                    return true;
+            }
+            else
+                return false;
+        }
+
+    }
+
+    /**
      * @param evt Discovery event.
      * @return Affinity topology version.
      */
@@ -1056,7 +1114,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             return;
         }
 
-        ClusterNode oldest = cctx.discovery().oldestAliveServerNode(AffinityTopologyVersion.NONE);
+        ClusterNode oldest = cctx.discovery().oldestAliveServerNode(NONE);
 
         if (oldest == null) {
             if (log.isDebugEnabled())
@@ -1554,30 +1612,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         if (log.isDebugEnabled())
             log.debug("Exchange done [topVer=" + topVer + ", err=" + err + ']');
 
-        if (err == null) {
+        if (err == null)
             exchFuts.readyTopVer(topVer);
 
-            for (Map.Entry<AffinityTopologyVersion, AffinityReadyFuture> entry : readyFuts.entrySet()) {
-                if (entry.getKey().compareTo(topVer) <= 0) {
-                    if (log.isDebugEnabled())
-                        log.debug("Completing created topology ready future " +
-                            "[ver=" + topVer + ", fut=" + entry.getValue() + ']');
-
-                    entry.getValue().onDone(topVer);
-                }
-            }
-        }
-        else {
-            for (Map.Entry<AffinityTopologyVersion, AffinityReadyFuture> entry : readyFuts.entrySet()) {
-                if (entry.getKey().compareTo(initTopVer) <= 0) {
-                    if (log.isDebugEnabled())
-                        log.debug("Completing created topology ready future with error " +
-                            "[ver=" + entry.getKey() + ", fut=" + entry.getValue() + ']');
-
-                    entry.getValue().onDone(err);
-                }
-            }
-        }
+        completeAffReadyFuts(err == null ? topVer : initTopVer, err);
 
         ExchangeFutureSet exchFuts0 = exchFuts;
 
@@ -1596,6 +1634,28 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         }
     }
 
+    /** */
+    private void completeAffReadyFuts(AffinityTopologyVersion topVer, @Nullable Throwable err) {
+        for (Map.Entry<AffinityTopologyVersion, AffinityReadyFuture> entry : readyFuts.entrySet()) {
+            if (entry.getKey().compareTo(topVer) <= 0) {
+                if (err == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Completing created topology ready future " +
+                            "[ver=" + topVer + ", fut=" + entry.getValue() + ']');
+
+                    entry.getValue().onDone(topVer);
+                }
+                else {
+                    if (log.isDebugEnabled())
+                        log.debug("Completing created topology ready future with error " +
+                            "[ver=" + entry.getKey() + ", fut=" + entry.getValue() + ']');
+
+                    entry.getValue().onDone(err);
+                }
+            }
+        }
+    }
+
     /**
      * @param fut Future.
      * @return {@code True} if added.
@@ -2230,6 +2290,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         break;
                     }
 
+                    if (!fut.changedAffinity()) {
+                        if (log.isInfoEnabled())
+                            log.info("Stop merge, no-affinity exchange found: " + evt);
+
+                        break;
+                    }
+
                     ClusterNode node = evt.eventNode();
 
                     if ((evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT) &&
@@ -2402,6 +2469,38 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         return false;
     }
 
+    /** */
+    public boolean affinityChanged(AffinityTopologyVersion from, AffinityTopologyVersion to) {
+        if (lastAffinityChangedTopologyVersion(to).compareTo(from) >= 0)
+            return false;
+
+        Collection<GridDhtPartitionsExchangeFuture> history = exchFuts.values();
+
+        boolean fromFound = false;
+
+        for (GridDhtPartitionsExchangeFuture fut : history) {
+            if (!fromFound) {
+                int cmp = fut.initialVersion().compareTo(from);
+
+                if (cmp > 0) // We don't have history, so return true for safety
+                    return true;
+                else if (cmp == 0)
+                    fromFound = true;
+                else if (fut.isDone() && fut.topologyVersion().compareTo(from) >= 0)
+                    return true; // Temporary solution for merge exchange case
+            }
+            else {
+                if (fut.changedAffinity())
+                    return true;
+
+                if (fut.initialVersion().compareTo(to) >= 0)
+                    return false;
+            }
+        }
+
+        return true;
+    }
+
     /**
      * Exchange future thread. All exchanges happen only by one thread and next
      * exchange will not start until previous one completes.
@@ -2756,6 +2855,20 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                 crd = newCrd = !srvNodes.isEmpty() && srvNodes.get(0).isLocal();
                             }
 
+                            if (!exchFut.changedAffinity()) {
+                                GridDhtPartitionsExchangeFuture lastFut = lastFinishedFut.get();
+
+                                if (lastFut != null) {
+                                    if (!lastFut.changedAffinity()) {
+                                        AffinityTopologyVersion lastAffVer = cctx.exchange().lastAffinityChangedTopologyVersion(lastFut.initialVersion());
+
+                                        cctx.exchange().lastAffinityChangedTopologyVersion(exchFut.initialVersion(), lastAffVer);
+                                    }
+                                    else
+                                        cctx.exchange().lastAffinityChangedTopologyVersion(exchFut.initialVersion(), lastFut.initialVersion());
+                                }
+                            }
+
                             exchFut.init(newCrd);
 
                             int dumpCnt = 0;
@@ -2849,7 +2962,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                     continue;
 
                                 if (grp.preloader().rebalanceRequired(rebTopVer, exchFut))
-                                    rebTopVer = AffinityTopologyVersion.NONE;
+                                    rebTopVer = NONE;
 
                                 changed |= grp.topology().afterExchange(exchFut);
                             }
@@ -2860,9 +2973,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                         // Schedule rebalance if force rebalance or force reassign occurs.
                         if (exchFut == null)
-                            rebTopVer = AffinityTopologyVersion.NONE;
+                            rebTopVer = NONE;
 
-                        if (!cctx.kernalContext().clientNode() && rebTopVer.equals(AffinityTopologyVersion.NONE)) {
+                        if (!cctx.kernalContext().clientNode() && rebTopVer.equals(NONE)) {
                             assignsMap = new HashMap<>();
 
                             IgniteCacheSnapshotManager snp = cctx.snapshot();
@@ -2893,7 +3006,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         busy = false;
                     }
 
-                    if (assignsMap != null && rebTopVer.equals(AffinityTopologyVersion.NONE)) {
+                    if (assignsMap != null && rebTopVer.equals(NONE)) {
                         int size = assignsMap.size();
 
                         NavigableMap<Integer, List<Integer>> orderMap = new TreeMap<>();
@@ -3076,7 +3189,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         /** */
         private final AtomicReference<AffinityTopologyVersion> readyTopVer =
-            new AtomicReference<>(AffinityTopologyVersion.NONE);
+            new AtomicReference<>(NONE);
 
         /**
          * Creates ordered, not strict list set.
@@ -3084,20 +3197,15 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
          * @param histSize Max history size.
          */
         private ExchangeFutureSet(int histSize) {
-            super(new Comparator<GridDhtPartitionsExchangeFuture>() {
-                @Override public int compare(
-                    GridDhtPartitionsExchangeFuture f1,
-                    GridDhtPartitionsExchangeFuture f2
-                ) {
-                    AffinityTopologyVersion t1 = f1.exchangeId().topologyVersion();
-                    AffinityTopologyVersion t2 = f2.exchangeId().topologyVersion();
-
-                    assert t1.topologyVersion() > 0;
-                    assert t2.topologyVersion() > 0;
-
-                    // Reverse order.
-                    return t2.compareTo(t1);
-                }
+            super((f1, f2) -> {
+                AffinityTopologyVersion t1 = f1.exchangeId().topologyVersion();
+                AffinityTopologyVersion t2 = f2.exchangeId().topologyVersion();
+
+                assert t1.topologyVersion() > 0;
+                assert t2.topologyVersion() > 0;
+
+                // Reverse order.
+                return t2.compareTo(t1);
             }, /*not strict*/false);
 
             this.histSize = histSize;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridChangeGlobalStateMessageResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridChangeGlobalStateMessageResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridChangeGlobalStateMessageResponse.java
index e49be49..4cf9e23 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridChangeGlobalStateMessageResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridChangeGlobalStateMessageResponse.java
@@ -116,13 +116,13 @@ public class GridChangeGlobalStateMessageResponse extends GridCacheMessage {
         }
 
         switch (writer.state()) {
-            case 2:
+            case 3:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 3:
+            case 4:
                 if (!writer.writeUuid("requestId", requestId))
                     return false;
 
@@ -144,7 +144,7 @@ public class GridChangeGlobalStateMessageResponse extends GridCacheMessage {
             return false;
 
         switch (reader.state()) {
-            case 2:
+            case 3:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -152,7 +152,7 @@ public class GridChangeGlobalStateMessageResponse extends GridCacheMessage {
 
                 reader.incrementState();
 
-            case 3:
+            case 4:
                 requestId = reader.readUuid("requestId");
 
                 if (!reader.isLastRead())
@@ -172,7 +172,7 @@ public class GridChangeGlobalStateMessageResponse extends GridCacheMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 4;
+        return 5;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObject.java
index 8f8ceb6..9c4eeee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObject.java
@@ -48,4 +48,4 @@ public interface KeyCacheObject extends CacheObject {
      * @return Copy of this object with given partition set.
      */
     public KeyCacheObject copy(int part);
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateAckMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateAckMessage.java
index 7c24106..f8012a7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateAckMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateAckMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaOperationStatusMessage;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -31,6 +32,7 @@ import java.util.UUID;
 /**
  * WAL state ack message (sent from participant node to coordinator).
  */
+@IgniteCodeGeneratingFail
 public class WalStateAckMessage implements Message {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java
index c092132..c420aeb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java
@@ -213,37 +213,37 @@ public class GridCacheTtlUpdateRequest extends GridCacheIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeCollection("nearKeys", nearKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeCollection("nearVers", nearVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
-                if (!writer.writeMessage("topVer", topVer))
+            case 7:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeLong("ttl", ttl))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeCollection("vers", vers, MessageCollectionItemType.MSG))
                     return false;
 
@@ -265,7 +265,7 @@ public class GridCacheTtlUpdateRequest extends GridCacheIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -273,7 +273,7 @@ public class GridCacheTtlUpdateRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 nearKeys = reader.readCollection("nearKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -281,7 +281,7 @@ public class GridCacheTtlUpdateRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 nearVers = reader.readCollection("nearVers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -289,15 +289,15 @@ public class GridCacheTtlUpdateRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 6:
-                topVer = reader.readMessage("topVer");
+            case 7:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 ttl = reader.readLong("ttl");
 
                 if (!reader.isLastRead())
@@ -305,7 +305,7 @@ public class GridCacheTtlUpdateRequest extends GridCacheIdMessage {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 vers = reader.readCollection("vers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -325,7 +325,7 @@ public class GridCacheTtlUpdateRequest extends GridCacheIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 9;
+        return 10;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryRequest.java
index 45d1f1a..90ce234 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryRequest.java
@@ -148,37 +148,37 @@ public class GridCacheTxRecoveryRequest extends GridDistributedBaseMessage {
         }
 
         switch (writer.state()) {
-            case 7:
+            case 8:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeBoolean("nearTxCheck", nearTxCheck))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeMessage("nearXidVer", nearXidVer))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeBoolean("sys", sys))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeInt("txNum", txNum))
                     return false;
 
@@ -200,7 +200,7 @@ public class GridCacheTxRecoveryRequest extends GridDistributedBaseMessage {
             return false;
 
         switch (reader.state()) {
-            case 7:
+            case 8:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -208,7 +208,7 @@ public class GridCacheTxRecoveryRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -216,7 +216,7 @@ public class GridCacheTxRecoveryRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 nearTxCheck = reader.readBoolean("nearTxCheck");
 
                 if (!reader.isLastRead())
@@ -224,7 +224,7 @@ public class GridCacheTxRecoveryRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 nearXidVer = reader.readMessage("nearXidVer");
 
                 if (!reader.isLastRead())
@@ -232,7 +232,7 @@ public class GridCacheTxRecoveryRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 sys = reader.readBoolean("sys");
 
                 if (!reader.isLastRead())
@@ -240,7 +240,7 @@ public class GridCacheTxRecoveryRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 txNum = reader.readInt("txNum");
 
                 if (!reader.isLastRead())
@@ -260,7 +260,7 @@ public class GridCacheTxRecoveryRequest extends GridDistributedBaseMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 13;
+        return 14;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryResponse.java
index a9ac26b..1ef44a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryResponse.java
@@ -129,19 +129,19 @@ public class GridCacheTxRecoveryResponse extends GridDistributedBaseMessage impl
         }
 
         switch (writer.state()) {
-            case 7:
+            case 8:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeBoolean("success", success))
                     return false;
 
@@ -163,7 +163,7 @@ public class GridCacheTxRecoveryResponse extends GridDistributedBaseMessage impl
             return false;
 
         switch (reader.state()) {
-            case 7:
+            case 8:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -171,7 +171,7 @@ public class GridCacheTxRecoveryResponse extends GridDistributedBaseMessage impl
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -179,7 +179,7 @@ public class GridCacheTxRecoveryResponse extends GridDistributedBaseMessage impl
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 success = reader.readBoolean("success");
 
                 if (!reader.isLastRead())
@@ -199,7 +199,7 @@ public class GridCacheTxRecoveryResponse extends GridDistributedBaseMessage impl
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 10;
+        return 11;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java
index fc209aa..8536e48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java
@@ -161,25 +161,25 @@ public abstract class GridDistributedBaseMessage extends GridCacheIdMessage impl
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeByteArray("candsByIdxBytes", candsByIdxBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeCollection("committedVers", committedVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeCollection("rolledbackVers", rolledbackVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeMessage("ver", ver))
                     return false;
 
@@ -201,7 +201,7 @@ public abstract class GridDistributedBaseMessage extends GridCacheIdMessage impl
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 candsByIdxBytes = reader.readByteArray("candsByIdxBytes");
 
                 if (!reader.isLastRead())
@@ -209,7 +209,7 @@ public abstract class GridDistributedBaseMessage extends GridCacheIdMessage impl
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 committedVers = reader.readCollection("committedVers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -217,7 +217,7 @@ public abstract class GridDistributedBaseMessage extends GridCacheIdMessage impl
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 rolledbackVers = reader.readCollection("rolledbackVers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -225,7 +225,7 @@ public abstract class GridDistributedBaseMessage extends GridCacheIdMessage impl
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 ver = reader.readMessage("ver");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
index 25a557c..ca78763 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
@@ -366,79 +366,79 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
         }
 
         switch (writer.state()) {
-            case 7:
+            case 8:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeBoolean("isInTx", isInTx))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeBoolean("isInvalidate", isInvalidate))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeBoolean("isRead", isRead))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeByte("isolation", isolation != null ? (byte)isolation.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeMessage("nearXidVer", nearXidVer))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeUuid("nodeId", nodeId))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 17:
                 if (!writer.writeBooleanArray("retVals", retVals))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 18:
                 if (!writer.writeLong("threadId", threadId))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
+            case 19:
                 if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
-            case 19:
+            case 20:
                 if (!writer.writeInt("txSize", txSize))
                     return false;
 
@@ -460,7 +460,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
             return false;
 
         switch (reader.state()) {
-            case 7:
+            case 8:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -468,7 +468,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -476,7 +476,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 isInTx = reader.readBoolean("isInTx");
 
                 if (!reader.isLastRead())
@@ -484,7 +484,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 isInvalidate = reader.readBoolean("isInvalidate");
 
                 if (!reader.isLastRead())
@@ -492,7 +492,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 isRead = reader.readBoolean("isRead");
 
                 if (!reader.isLastRead())
@@ -500,7 +500,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 byte isolationOrd;
 
                 isolationOrd = reader.readByte("isolation");
@@ -512,7 +512,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -520,7 +520,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 nearXidVer = reader.readMessage("nearXidVer");
 
                 if (!reader.isLastRead())
@@ -528,7 +528,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 nodeId = reader.readUuid("nodeId");
 
                 if (!reader.isLastRead())
@@ -536,7 +536,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 16:
+            case 17:
                 retVals = reader.readBooleanArray("retVals");
 
                 if (!reader.isLastRead())
@@ -544,7 +544,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 17:
+            case 18:
                 threadId = reader.readLong("threadId");
 
                 if (!reader.isLastRead())
@@ -552,7 +552,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 18:
+            case 19:
                 timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
@@ -560,7 +560,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 19:
+            case 20:
                 txSize = reader.readInt("txSize");
 
                 if (!reader.isLastRead())
@@ -580,7 +580,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 20;
+        return 21;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java
index 4b21896..2d4de9c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java
@@ -221,19 +221,19 @@ public class GridDistributedLockResponse extends GridDistributedBaseMessage {
         }
 
         switch (writer.state()) {
-            case 7:
+            case 8:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG))
                     return false;
 
@@ -255,7 +255,7 @@ public class GridDistributedLockResponse extends GridDistributedBaseMessage {
             return false;
 
         switch (reader.state()) {
-            case 7:
+            case 8:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -263,7 +263,7 @@ public class GridDistributedLockResponse extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -271,7 +271,7 @@ public class GridDistributedLockResponse extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 vals = reader.readCollection("vals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -291,7 +291,7 @@ public class GridDistributedLockResponse extends GridDistributedBaseMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 10;
+        return 11;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java
index ea9336b..a1af470 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java
@@ -325,85 +325,85 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
         }
 
         switch (writer.state()) {
-            case 7:
+            case 8:
                 if (!writer.writeMessage("baseVer", baseVer))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeBoolean("commit", commit))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeMessage("commitVer", commitVer))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeBoolean("invalidate", invalidate))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeByte("plc", plc))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 17:
                 if (!writer.writeBoolean("sys", sys))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 18:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
+            case 19:
                 if (!writer.writeLong("threadId", threadId))
                     return false;
 
                 writer.incrementState();
 
-            case 19:
-                if (!writer.writeMessage("topVer", topVer))
+            case 20:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 20:
+            case 21:
                 if (!writer.writeInt("txSize", txSize))
                     return false;
 
@@ -425,7 +425,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
             return false;
 
         switch (reader.state()) {
-            case 7:
+            case 8:
                 baseVer = reader.readMessage("baseVer");
 
                 if (!reader.isLastRead())
@@ -433,7 +433,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 commit = reader.readBoolean("commit");
 
                 if (!reader.isLastRead())
@@ -441,7 +441,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 commitVer = reader.readMessage("commitVer");
 
                 if (!reader.isLastRead())
@@ -449,7 +449,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -457,7 +457,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -465,7 +465,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 invalidate = reader.readBoolean("invalidate");
 
                 if (!reader.isLastRead())
@@ -473,7 +473,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 plc = reader.readByte("plc");
 
                 if (!reader.isLastRead())
@@ -481,7 +481,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -489,7 +489,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 byte syncModeOrd;
 
                 syncModeOrd = reader.readByte("syncMode");
@@ -501,7 +501,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 16:
+            case 17:
                 sys = reader.readBoolean("sys");
 
                 if (!reader.isLastRead())
@@ -509,7 +509,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 17:
+            case 18:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -517,7 +517,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 18:
+            case 19:
                 threadId = reader.readLong("threadId");
 
                 if (!reader.isLastRead())
@@ -525,15 +525,15 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
                 reader.incrementState();
 
-            case 19:
-                topVer = reader.readMessage("topVer");
+            case 20:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 20:
+            case 21:
                 txSize = reader.readInt("txSize");
 
                 if (!reader.isLastRead())
@@ -553,7 +553,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage i
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 21;
+        return 22;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
index c36e633..5fdf970 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
@@ -145,25 +145,25 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
         }
 
         switch (writer.state()) {
-            case 2:
+            case 3:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 3:
+            case 4:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeInt("part", part))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeMessage("txId", txId))
                     return false;
 
@@ -185,7 +185,7 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
             return false;
 
         switch (reader.state()) {
-            case 2:
+            case 3:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -193,7 +193,7 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
 
                 reader.incrementState();
 
-            case 3:
+            case 4:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -201,7 +201,7 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 part = reader.readInt("part");
 
                 if (!reader.isLastRead())
@@ -209,7 +209,7 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 txId = reader.readMessage("txId");
 
                 if (!reader.isLastRead())
@@ -229,7 +229,7 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 6;
+        return 7;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
index a5aa0d8..96eeee2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
@@ -505,79 +505,79 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
         }
 
         switch (writer.state()) {
-            case 7:
+            case 8:
                 if (!writer.writeByte("concurrency", concurrency != null ? (byte)concurrency.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeCollection("dhtVerKeys", dhtVerKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeCollection("dhtVerVals", dhtVerVals, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeByte("isolation", isolation != null ? (byte)isolation.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeByte("plc", plc))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeCollection("reads", reads, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeLong("threadId", threadId))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 17:
                 if (!writer.writeMap("txNodesMsg", txNodesMsg, MessageCollectionItemType.UUID, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 18:
                 if (!writer.writeInt("txSize", txSize))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
+            case 19:
                 if (!writer.writeMessage("writeVer", writeVer))
                     return false;
 
                 writer.incrementState();
 
-            case 19:
+            case 20:
                 if (!writer.writeCollection("writes", writes, MessageCollectionItemType.MSG))
                     return false;
 
@@ -599,7 +599,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
             return false;
 
         switch (reader.state()) {
-            case 7:
+            case 8:
                 byte concurrencyOrd;
 
                 concurrencyOrd = reader.readByte("concurrency");
@@ -611,7 +611,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 dhtVerKeys = reader.readCollection("dhtVerKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -619,7 +619,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 dhtVerVals = reader.readCollection("dhtVerVals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -627,7 +627,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -635,7 +635,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 byte isolationOrd;
 
                 isolationOrd = reader.readByte("isolation");
@@ -647,7 +647,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 plc = reader.readByte("plc");
 
                 if (!reader.isLastRead())
@@ -655,7 +655,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 reads = reader.readCollection("reads", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -663,7 +663,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 threadId = reader.readLong("threadId");
 
                 if (!reader.isLastRead())
@@ -671,7 +671,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
@@ -679,7 +679,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 16:
+            case 17:
                 txNodesMsg = reader.readMap("txNodesMsg", MessageCollectionItemType.UUID, MessageCollectionItemType.MSG, false);
 
                 if (!reader.isLastRead())
@@ -687,7 +687,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 17:
+            case 18:
                 txSize = reader.readInt("txSize");
 
                 if (!reader.isLastRead())
@@ -695,7 +695,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 18:
+            case 19:
                 writeVer = reader.readMessage("writeVer");
 
                 if (!reader.isLastRead())
@@ -703,7 +703,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 19:
+            case 20:
                 writes = reader.readCollection("writes", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -723,7 +723,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 20;
+        return 21;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java
index 58e9492..c26880e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java
@@ -178,19 +178,19 @@ public class GridDistributedTxPrepareResponse extends GridDistributedBaseMessage
         }
 
         switch (writer.state()) {
-            case 7:
+            case 8:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeInt("part", part))
                     return false;
 
@@ -212,7 +212,7 @@ public class GridDistributedTxPrepareResponse extends GridDistributedBaseMessage
             return false;
 
         switch (reader.state()) {
-            case 7:
+            case 8:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -220,7 +220,7 @@ public class GridDistributedTxPrepareResponse extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -228,7 +228,7 @@ public class GridDistributedTxPrepareResponse extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 part = reader.readInt("part");
 
                 if (!reader.isLastRead())
@@ -248,7 +248,7 @@ public class GridDistributedTxPrepareResponse extends GridDistributedBaseMessage
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 10;
+        return 11;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
index ca2bdab..001eb61 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
@@ -122,7 +122,7 @@ public class GridDistributedUnlockRequest extends GridDistributedBaseMessage {
         }
 
         switch (writer.state()) {
-            case 7:
+            case 8:
                 if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
@@ -144,7 +144,7 @@ public class GridDistributedUnlockRequest extends GridDistributedBaseMessage {
             return false;
 
         switch (reader.state()) {
-            case 7:
+            case 8:
                 keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -164,7 +164,7 @@ public class GridDistributedUnlockRequest extends GridDistributedBaseMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 9;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/ClientCacheDhtTopologyFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/ClientCacheDhtTopologyFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/ClientCacheDhtTopologyFuture.java
index 4b48f5a..f468590 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/ClientCacheDhtTopologyFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/ClientCacheDhtTopologyFuture.java
@@ -81,6 +81,11 @@ public class ClientCacheDhtTopologyFuture extends GridDhtTopologyFutureAdapter {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean changedAffinity() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return "ClientCacheDhtTopologyFuture [topVer=" + topVer + ']';
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
index 44c7b88..cf7018a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
@@ -109,7 +109,7 @@ public class GridDhtAffinityAssignmentRequest extends GridCacheGroupIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 6;
+        return 7;
     }
 
     /** {@inheritDoc} */
@@ -127,20 +127,20 @@ public class GridDhtAffinityAssignmentRequest extends GridCacheGroupIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
-                if (!writer.writeMessage("topVer", topVer))
+            case 6:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -161,7 +161,7 @@ public class GridDhtAffinityAssignmentRequest extends GridCacheGroupIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -169,7 +169,7 @@ public class GridDhtAffinityAssignmentRequest extends GridCacheGroupIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -177,8 +177,8 @@ public class GridDhtAffinityAssignmentRequest extends GridCacheGroupIdMessage {
 
                 reader.incrementState();
 
-            case 5:
-                topVer = reader.readMessage("topVer");
+            case 6:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
index 5b0de08..e8b40e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
@@ -215,7 +215,7 @@ public class GridDhtAffinityAssignmentResponse extends GridCacheGroupIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 9;
     }
 
     /**
@@ -272,32 +272,32 @@ public class GridDhtAffinityAssignmentResponse extends GridCacheGroupIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeByteArray("affAssignmentIdsBytes", affAssignmentIdsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeByteArray("idealAffAssignmentBytes", idealAffAssignmentBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeByteArray("partBytes", partBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
-                if (!writer.writeMessage("topVer", topVer))
+            case 8:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -318,7 +318,7 @@ public class GridDhtAffinityAssignmentResponse extends GridCacheGroupIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 affAssignmentIdsBytes = reader.readByteArray("affAssignmentIdsBytes");
 
                 if (!reader.isLastRead())
@@ -326,7 +326,7 @@ public class GridDhtAffinityAssignmentResponse extends GridCacheGroupIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -334,7 +334,7 @@ public class GridDhtAffinityAssignmentResponse extends GridCacheGroupIdMessage {
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 idealAffAssignmentBytes = reader.readByteArray("idealAffAssignmentBytes");
 
                 if (!reader.isLastRead())
@@ -342,7 +342,7 @@ public class GridDhtAffinityAssignmentResponse extends GridCacheGroupIdMessage {
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 partBytes = reader.readByteArray("partBytes");
 
                 if (!reader.isLastRead())
@@ -350,8 +350,8 @@ public class GridDhtAffinityAssignmentResponse extends GridCacheGroupIdMessage {
 
                 reader.incrementState();
 
-            case 7:
-                topVer = reader.readMessage("topVer");
+            case 8:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index faada54..17bf4ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -1270,8 +1270,14 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
      * @param curVer Current topology version.
      * @return {@code True} if cache affinity changed and operation should be remapped.
      */
-    protected final boolean needRemap(AffinityTopologyVersion expVer, AffinityTopologyVersion curVer) {
-        if (expVer.equals(curVer))
+    protected final boolean needRemap(AffinityTopologyVersion expVer, AffinityTopologyVersion curVer,
+        Collection<KeyCacheObject> keys) {
+        if (curVer.equals(expVer))
+            return false;
+
+        AffinityTopologyVersion lastAffChangedTopVer = ctx.shared().exchange().lastAffinityChangedTopologyVersion(expVer);
+
+        if (curVer.compareTo(lastAffChangedTopVer) >= 0 && curVer.compareTo(expVer) <= 0)
             return false;
 
         // TODO IGNITE-7164 check mvcc crd for mvcc enabled txs.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
index 090ea68..95786be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
@@ -377,67 +377,67 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
         }
 
         switch (writer.state()) {
-            case 20:
+            case 21:
                 if (!writer.writeLong("accessTtl", accessTtl))
                     return false;
 
                 writer.incrementState();
 
-            case 21:
+            case 22:
                 if (!writer.writeBitSet("invalidateEntries", invalidateEntries))
                     return false;
 
                 writer.incrementState();
 
-            case 22:
+            case 23:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 23:
+            case 24:
                 if (!writer.writeCollection("nearKeys", nearKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 25:
                 if (!writer.writeObjectArray("ownedKeys", ownedKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 25:
+            case 26:
                 if (!writer.writeObjectArray("ownedValues", ownedValues, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 26:
+            case 27:
                 if (!writer.writeBitSet("preloadKeys", preloadKeys))
                     return false;
 
                 writer.incrementState();
 
-            case 27:
+            case 28:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 28:
+            case 29:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 29:
-                if (!writer.writeMessage("topVer", topVer))
+            case 30:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 30:
+            case 31:
                 if (!writer.writeString("txLbl", txLbl))
                     return false;
 
@@ -459,7 +459,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
             return false;
 
         switch (reader.state()) {
-            case 20:
+            case 21:
                 accessTtl = reader.readLong("accessTtl");
 
                 if (!reader.isLastRead())
@@ -467,7 +467,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 21:
+            case 22:
                 invalidateEntries = reader.readBitSet("invalidateEntries");
 
                 if (!reader.isLastRead())
@@ -475,7 +475,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 22:
+            case 23:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -483,7 +483,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 23:
+            case 24:
                 nearKeys = reader.readCollection("nearKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -491,7 +491,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 24:
+            case 25:
                 ownedKeys = reader.readObjectArray("ownedKeys", MessageCollectionItemType.MSG, KeyCacheObject.class);
 
                 if (!reader.isLastRead())
@@ -499,7 +499,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 25:
+            case 26:
                 ownedValues = reader.readObjectArray("ownedValues", MessageCollectionItemType.MSG, GridCacheVersion.class);
 
                 if (!reader.isLastRead())
@@ -507,7 +507,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 26:
+            case 27:
                 preloadKeys = reader.readBitSet("preloadKeys");
 
                 if (!reader.isLastRead())
@@ -515,7 +515,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 27:
+            case 28:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -523,7 +523,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 28:
+            case 29:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -531,15 +531,15 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 29:
-                topVer = reader.readMessage("topVer");
+            case 30:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 30:
+            case 31:
                 txLbl = reader.readString("txLbl");
 
                 if (!reader.isLastRead())
@@ -559,7 +559,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 31;
+        return 32;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
index 87abd6c..63c07e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
@@ -207,25 +207,25 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
         }
 
         switch (writer.state()) {
-            case 10:
+            case 11:
                 if (!writer.writeCollection("invalidParts", invalidParts, MessageCollectionItemType.INT))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeCollection("nearEvicted", nearEvicted, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeCollection("preloadEntries", preloadEntries, MessageCollectionItemType.MSG))
                     return false;
 
@@ -247,7 +247,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
             return false;
 
         switch (reader.state()) {
-            case 10:
+            case 11:
                 invalidParts = reader.readCollection("invalidParts", MessageCollectionItemType.INT);
 
                 if (!reader.isLastRead())
@@ -255,7 +255,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -263,7 +263,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 nearEvicted = reader.readCollection("nearEvicted", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -271,7 +271,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 preloadEntries = reader.readCollection("preloadEntries", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -291,7 +291,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 14;
+        return 15;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
index 650ddc7..2a4e103 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
@@ -85,4 +85,10 @@ public interface GridDhtTopologyFuture extends IgniteInternalFuture<AffinityTopo
         boolean read,
         @Nullable Object key,
         @Nullable Collection<?> keys);
+
+    /**
+     *
+     * @return {@code True} if this exchange changed affinity.
+     */
+    public boolean changedAffinity();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 5437991..60463b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -1082,7 +1082,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
             }
 
             try {
-                if (top != null && needRemap(req.topologyVersion(), top.readyTopologyVersion())) {
+                if (top != null && needRemap(req.topologyVersion(), top.readyTopologyVersion(), req.keys())) {
                     if (log.isDebugEnabled()) {
                         log.debug("Client topology version mismatch, need remap lock request [" +
                             "reqTopVer=" + req.topologyVersion() +
@@ -2122,7 +2122,10 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
 
                 GridDhtTopologyFuture topFut = top.topologyVersionFuture();
 
-                if (!topFut.isDone() || !topFut.topologyVersion().equals(topVer)) {
+                boolean done = topFut.isDone();
+
+                if (!done || !(topFut.topologyVersion().compareTo(topVer) >= 0
+                    && ctx.shared().exchange().lastAffinityChangedTopologyVersion(topFut.initialVersion()).compareTo(topVer) <= 0)) {
                     // TODO IGNITE-7164 Wait for topology change, remap client TX in case affinity was changed.
                     top.readUnlock();
 


[24/50] [abbrv] ignite git commit: IGNITE-9558 Avoid blocking transactions on client connect when possible - Fixes #4933.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
index 124ae44..80c45ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
@@ -167,26 +167,26 @@ public class GridDhtForceKeysRequest extends GridCacheIdMessage implements GridC
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
-                if (!writer.writeMessage("topVer", topVer))
+            case 7:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -207,7 +207,7 @@ public class GridDhtForceKeysRequest extends GridCacheIdMessage implements GridC
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -215,7 +215,7 @@ public class GridDhtForceKeysRequest extends GridCacheIdMessage implements GridC
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -223,7 +223,7 @@ public class GridDhtForceKeysRequest extends GridCacheIdMessage implements GridC
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -231,8 +231,8 @@ public class GridDhtForceKeysRequest extends GridCacheIdMessage implements GridC
 
                 reader.incrementState();
 
-            case 6:
-                topVer = reader.readMessage("topVer");
+            case 7:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -251,7 +251,7 @@ public class GridDhtForceKeysRequest extends GridCacheIdMessage implements GridC
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 7;
+        return 8;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
index 977e9ba..ab85df3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
@@ -213,31 +213,31 @@ public class GridDhtForceKeysResponse extends GridCacheIdMessage implements Grid
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeCollection("infos", infos, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeCollection("missedKeys", missedKeys, MessageCollectionItemType.MSG))
                     return false;
 
@@ -259,7 +259,7 @@ public class GridDhtForceKeysResponse extends GridCacheIdMessage implements Grid
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -267,7 +267,7 @@ public class GridDhtForceKeysResponse extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -275,7 +275,7 @@ public class GridDhtForceKeysResponse extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 infos = reader.readCollection("infos", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -283,7 +283,7 @@ public class GridDhtForceKeysResponse extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -291,7 +291,7 @@ public class GridDhtForceKeysResponse extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 missedKeys = reader.readCollection("missedKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -311,7 +311,7 @@ public class GridDhtForceKeysResponse extends GridCacheIdMessage implements Grid
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 9;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandLegacyMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandLegacyMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandLegacyMessage.java
index 46e9ceb..cd7741b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandLegacyMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandLegacyMessage.java
@@ -285,49 +285,49 @@ public class GridDhtPartitionDemandLegacyMessage extends GridCacheGroupIdMessage
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeCollection("historicalParts", historicalParts, MessageCollectionItemType.INT))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeCollection("parts", parts, MessageCollectionItemType.INT))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeMap("partsCntrs", partsCntrs, MessageCollectionItemType.INT, MessageCollectionItemType.LONG))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
-                if (!writer.writeMessage("topVer", topVer))
+            case 8:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeByteArray("topicBytes", topicBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeLong("updateSeq", updateSeq))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeInt("workerId", workerId))
                     return false;
 
@@ -349,7 +349,7 @@ public class GridDhtPartitionDemandLegacyMessage extends GridCacheGroupIdMessage
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 historicalParts = reader.readCollection("historicalParts", MessageCollectionItemType.INT);
 
                 if (!reader.isLastRead())
@@ -357,7 +357,7 @@ public class GridDhtPartitionDemandLegacyMessage extends GridCacheGroupIdMessage
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 parts = reader.readCollection("parts", MessageCollectionItemType.INT);
 
                 if (!reader.isLastRead())
@@ -365,7 +365,7 @@ public class GridDhtPartitionDemandLegacyMessage extends GridCacheGroupIdMessage
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 partsCntrs = reader.readMap("partsCntrs", MessageCollectionItemType.INT, MessageCollectionItemType.LONG, false);
 
                 if (!reader.isLastRead())
@@ -373,7 +373,7 @@ public class GridDhtPartitionDemandLegacyMessage extends GridCacheGroupIdMessage
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
@@ -381,15 +381,15 @@ public class GridDhtPartitionDemandLegacyMessage extends GridCacheGroupIdMessage
 
                 reader.incrementState();
 
-            case 7:
-                topVer = reader.readMessage("topVer");
+            case 8:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 topicBytes = reader.readByteArray("topicBytes");
 
                 if (!reader.isLastRead())
@@ -397,7 +397,7 @@ public class GridDhtPartitionDemandLegacyMessage extends GridCacheGroupIdMessage
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 updateSeq = reader.readLong("updateSeq");
 
                 if (!reader.isLastRead())
@@ -405,7 +405,7 @@ public class GridDhtPartitionDemandLegacyMessage extends GridCacheGroupIdMessage
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 workerId = reader.readInt("workerId");
 
                 if (!reader.isLastRead())
@@ -415,7 +415,7 @@ public class GridDhtPartitionDemandLegacyMessage extends GridCacheGroupIdMessage
 
         }
 
-        return reader.afterMessageRead(GridDhtPartitionDemandMessage.class);
+        return reader.afterMessageRead(GridDhtPartitionDemandLegacyMessage.class);
     }
 
     /** {@inheritDoc} */
@@ -425,7 +425,7 @@ public class GridDhtPartitionDemandLegacyMessage extends GridCacheGroupIdMessage
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 11;
+        return 12;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
index dc6162b..bae3264 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
@@ -21,6 +21,7 @@ import java.io.Externalizable;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheGroupIdMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
@@ -35,6 +36,7 @@ import org.jetbrains.annotations.NotNull;
 /**
  * Partition demand request.
  */
+@IgniteCodeGeneratingFail
 public class GridDhtPartitionDemandMessage extends GridCacheGroupIdMessage {
     /** */
     private static final long serialVersionUID = 0L;
@@ -259,37 +261,37 @@ public class GridDhtPartitionDemandMessage extends GridCacheGroupIdMessage {
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeByteArray("partsBytes", partsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
-                if (!writer.writeMessage("topVer", topVer))
+            case 6:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeByteArray("topicBytes", topicBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeLong("rebalanceId", rebalanceId))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeInt("workerId", workerId))
                     return false;
 
@@ -311,7 +313,7 @@ public class GridDhtPartitionDemandMessage extends GridCacheGroupIdMessage {
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 partsBytes = reader.readByteArray("partsBytes");
 
                 if (!reader.isLastRead())
@@ -319,7 +321,7 @@ public class GridDhtPartitionDemandMessage extends GridCacheGroupIdMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
@@ -327,15 +329,15 @@ public class GridDhtPartitionDemandMessage extends GridCacheGroupIdMessage {
 
                 reader.incrementState();
 
-            case 5:
-                topVer = reader.readMessage("topVer");
+            case 6:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 topicBytes = reader.readByteArray("topicBytes");
 
                 if (!reader.isLastRead())
@@ -343,7 +345,7 @@ public class GridDhtPartitionDemandMessage extends GridCacheGroupIdMessage {
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 rebalanceId = reader.readLong("rebalanceId");
 
                 if (!reader.isLastRead())
@@ -351,7 +353,7 @@ public class GridDhtPartitionDemandMessage extends GridCacheGroupIdMessage {
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 workerId = reader.readInt("workerId");
 
                 if (!reader.isLastRead())
@@ -371,7 +373,7 @@ public class GridDhtPartitionDemandMessage extends GridCacheGroupIdMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 9;
+        return 10;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
index 4d5137e..0352564 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
@@ -247,7 +247,7 @@ public class GridDhtPartitionExchangeId implements Message, Comparable<GridDhtPa
                 writer.incrementState();
 
             case 2:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -282,7 +282,7 @@ public class GridDhtPartitionExchangeId implements Message, Comparable<GridDhtPa
                 reader.incrementState();
 
             case 2:
-                topVer = reader.readMessage("topVer");
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -310,4 +310,4 @@ public class GridDhtPartitionExchangeId implements Message, Comparable<GridDhtPa
             "nodeId", U.id8(nodeId),
             "evt", U.gridEventName(evt));
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
index 284700a..3034fb9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectMap;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
@@ -45,6 +46,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 /**
  * Partition supply message.
  */
+@IgniteCodeGeneratingFail
 public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage implements GridCacheDeployable {
     /** */
     private static final long serialVersionUID = 0L;
@@ -282,55 +284,55 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeCollection("clean", clean, MessageCollectionItemType.INT))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeLong("estimatedKeysCnt", estimatedKeysCnt))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeMap("infos", infos, MessageCollectionItemType.INT, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeMap("keysPerCache", keysPerCache, MessageCollectionItemType.INT, MessageCollectionItemType.LONG))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeMap("last", last, MessageCollectionItemType.INT, MessageCollectionItemType.LONG))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeCollection("missed", missed, MessageCollectionItemType.INT))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeInt("msgSize", msgSize))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
-                if (!writer.writeMessage("topVer", topVer))
+            case 11:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 // Keep 'updateSeq' name for compatibility.
                 if (!writer.writeLong("updateSeq", rebalanceId))
                     return false;
@@ -353,7 +355,7 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 clean = reader.readCollection("clean", MessageCollectionItemType.INT);
 
                 if (!reader.isLastRead())
@@ -361,7 +363,7 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 estimatedKeysCnt = reader.readLong("estimatedKeysCnt");
 
                 if (!reader.isLastRead())
@@ -369,7 +371,7 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 infos = reader.readMap("infos", MessageCollectionItemType.INT, MessageCollectionItemType.MSG, false);
 
                 if (!reader.isLastRead())
@@ -377,7 +379,7 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 keysPerCache = reader.readMap("keysPerCache", MessageCollectionItemType.INT, MessageCollectionItemType.LONG, false);
 
                 if (!reader.isLastRead())
@@ -385,7 +387,7 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 last = reader.readMap("last", MessageCollectionItemType.INT, MessageCollectionItemType.LONG, false);
 
                 if (!reader.isLastRead())
@@ -393,7 +395,7 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 missed = reader.readCollection("missed", MessageCollectionItemType.INT);
 
                 if (!reader.isLastRead())
@@ -401,7 +403,7 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 msgSize = reader.readInt("msgSize");
 
                 if (!reader.isLastRead())
@@ -409,15 +411,15 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
 
                 reader.incrementState();
 
-            case 10:
-                topVer = reader.readMessage("topVer");
+            case 11:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 // Keep 'updateSeq' name for compatibility.
                 rebalanceId = reader.readLong("updateSeq");
 
@@ -438,7 +440,7 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 12;
+        return 13;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java
index b6bff0e..b209cdb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java
@@ -101,7 +101,7 @@ public class GridDhtPartitionSupplyMessageV2 extends GridDhtPartitionSupplyMessa
         }
 
         switch (writer.state()) {
-            case 12:
+            case 13:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
@@ -123,7 +123,7 @@ public class GridDhtPartitionSupplyMessageV2 extends GridDhtPartitionSupplyMessa
             return false;
 
         switch (reader.state()) {
-            case 12:
+            case 13:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -148,6 +148,6 @@ public class GridDhtPartitionSupplyMessageV2 extends GridDhtPartitionSupplyMessa
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 13;
+        return 14;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
index 84cc792..e2884e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
@@ -145,7 +145,7 @@ public abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 5;
+        return 6;
     }
 
     /** {@inheritDoc} */
@@ -163,19 +163,19 @@ public abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage
         }
 
         switch (writer.state()) {
-            case 2:
+            case 3:
                 if (!writer.writeMessage("exchId", exchId))
                     return false;
 
                 writer.incrementState();
 
-            case 3:
+            case 4:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeMessage("lastVer", lastVer))
                     return false;
 
@@ -197,7 +197,7 @@ public abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage
             return false;
 
         switch (reader.state()) {
-            case 2:
+            case 3:
                 exchId = reader.readMessage("exchId");
 
                 if (!reader.isLastRead())
@@ -205,7 +205,7 @@ public abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 3:
+            case 4:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -213,7 +213,7 @@ public abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 lastVer = reader.readMessage("lastVer");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index f762d3d..a79c95f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -574,6 +574,16 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         return exchActions != null && exchActions.changedBaseline();
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean changedAffinity() {
+        DiscoveryEvent firstDiscoEvt0 = firstDiscoEvt;
+
+        assert firstDiscoEvt0 != null;
+
+        return firstDiscoEvt0.type() == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT
+            || !firstDiscoEvt0.eventNode().isClient() || firstDiscoEvt0.eventNode().isLocal();
+    }
+
     /**
      * @return {@code True} if there are caches to start.
      */
@@ -582,7 +592,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
-     * @return First event discovery event.
+     * @return First event discovery event.1
      *
      */
     public DiscoveryEvent firstEvent() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
index 4e895ed..fbaa241 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
@@ -633,74 +633,74 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
         }
 
         switch (writer.state()) {
-            case 5:
+            case 6:
                 if (!writer.writeMap("dupPartsData", dupPartsData, MessageCollectionItemType.INT, MessageCollectionItemType.INT))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeByteArray("errsBytes", errsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeMap("idealAffDiff", idealAffDiff, MessageCollectionItemType.INT, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeMap("joinedNodeAff", joinedNodeAff, MessageCollectionItemType.INT, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeByteArray("partCntrsBytes2", partCntrsBytes2))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeByteArray("partHistSuppliersBytes", partHistSuppliersBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeByteArray("partsBytes", partsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeByteArray("partsSizesBytes", partsSizesBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeByteArray("partsToReloadBytes", partsToReloadBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
-                if (!writer.writeMessage("resTopVer", resTopVer))
+            case 16:
+                if (!writer.writeAffinityTopologyVersion("resTopVer", resTopVer))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
-                if (!writer.writeMessage("topVer", topVer))
+            case 17:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -721,7 +721,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
             return false;
 
         switch (reader.state()) {
-            case 5:
+            case 6:
                 dupPartsData = reader.readMap("dupPartsData", MessageCollectionItemType.INT, MessageCollectionItemType.INT, false);
 
                 if (!reader.isLastRead())
@@ -729,7 +729,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 errsBytes = reader.readByteArray("errsBytes");
 
                 if (!reader.isLastRead())
@@ -737,7 +737,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 idealAffDiff = reader.readMap("idealAffDiff", MessageCollectionItemType.INT, MessageCollectionItemType.MSG, false);
 
                 if (!reader.isLastRead())
@@ -745,7 +745,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 joinedNodeAff = reader.readMap("joinedNodeAff", MessageCollectionItemType.INT, MessageCollectionItemType.MSG, false);
 
                 if (!reader.isLastRead())
@@ -753,7 +753,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 partCntrsBytes = reader.readByteArray("partCntrsBytes");
 
                 if (!reader.isLastRead())
@@ -761,7 +761,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 partCntrsBytes2 = reader.readByteArray("partCntrsBytes2");
 
                 if (!reader.isLastRead())
@@ -769,7 +769,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 partHistSuppliersBytes = reader.readByteArray("partHistSuppliersBytes");
 
                 if (!reader.isLastRead())
@@ -777,7 +777,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 partsBytes = reader.readByteArray("partsBytes");
 
                 if (!reader.isLastRead())
@@ -785,7 +785,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 partsSizesBytes = reader.readByteArray("partsSizesBytes");
 
                 if (!reader.isLastRead())
@@ -793,7 +793,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 partsToReloadBytes = reader.readByteArray("partsToReloadBytes");
 
                 if (!reader.isLastRead())
@@ -801,16 +801,16 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 15:
-                resTopVer = reader.readMessage("resTopVer");
+            case 16:
+                resTopVer = reader.readAffinityTopologyVersion("resTopVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 16:
-                topVer = reader.readMessage("topVer");
+            case 17:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -829,7 +829,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 17;
+        return 18;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
index 088fb31..b5de155 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
@@ -473,65 +473,66 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
         }
 
         switch (writer.state()) {
-            case 5:
+            case 6:
+                if (!writer.writeMessage("activeQryTrackers", activeQryTrackers))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
                 if (!writer.writeBoolean("client", client))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 8:
                 if (!writer.writeMap("dupPartsData", dupPartsData, MessageCollectionItemType.INT, MessageCollectionItemType.INT))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 9:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 10:
                 if (!writer.writeMessage("finishMsg", finishMsg))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 11:
                 if (!writer.writeCollection("grpsAffRequest", grpsAffRequest, MessageCollectionItemType.INT))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 12:
                 if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 13:
                 if (!writer.writeByteArray("partHistCntrsBytes", partHistCntrsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 14:
                 if (!writer.writeByteArray("partsBytes", partsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 15:
                 if (!writer.writeByteArray("partsSizesBytes", partsSizesBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
-                if (!writer.writeMessage("activeQryTrackers", activeQryTrackers))
-                    return false;
-
-                writer.incrementState();
         }
 
         return true;
@@ -548,7 +549,15 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
             return false;
 
         switch (reader.state()) {
-            case 5:
+            case 6:
+                activeQryTrackers = reader.readMessage("activeQryTrackers");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
                 client = reader.readBoolean("client");
 
                 if (!reader.isLastRead())
@@ -556,7 +565,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 6:
+            case 8:
                 dupPartsData = reader.readMap("dupPartsData", MessageCollectionItemType.INT, MessageCollectionItemType.INT, false);
 
                 if (!reader.isLastRead())
@@ -564,7 +573,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 7:
+            case 9:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -572,7 +581,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 8:
+            case 10:
                 finishMsg = reader.readMessage("finishMsg");
 
                 if (!reader.isLastRead())
@@ -580,7 +589,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 9:
+            case 11:
                 grpsAffRequest = reader.readCollection("grpsAffRequest", MessageCollectionItemType.INT);
 
                 if (!reader.isLastRead())
@@ -588,7 +597,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 10:
+            case 12:
                 partCntrsBytes = reader.readByteArray("partCntrsBytes");
 
                 if (!reader.isLastRead())
@@ -596,7 +605,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 11:
+            case 13:
                 partHistCntrsBytes = reader.readByteArray("partHistCntrsBytes");
 
                 if (!reader.isLastRead())
@@ -604,7 +613,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 12:
+            case 14:
                 partsBytes = reader.readByteArray("partsBytes");
 
                 if (!reader.isLastRead())
@@ -612,7 +621,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 13:
+            case 15:
                 partsSizesBytes = reader.readByteArray("partsSizesBytes");
 
                 if (!reader.isLastRead())
@@ -620,13 +629,6 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 14:
-                activeQryTrackers = reader.readMessage("activeQryTrackers");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
         }
 
         return reader.afterMessageRead(GridDhtPartitionsSingleMessage.class);
@@ -639,7 +641,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 15;
+        return 16;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java
index 0be0f37..26d3cde 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java
@@ -89,7 +89,7 @@ public class GridDhtPartitionsSingleRequest extends GridDhtPartitionsAbstractMes
         }
 
         switch (writer.state()) {
-            case 5:
+            case 6:
                 if (!writer.writeMessage("restoreExchId", restoreExchId))
                     return false;
 
@@ -111,7 +111,7 @@ public class GridDhtPartitionsSingleRequest extends GridDhtPartitionsAbstractMes
             return false;
 
         switch (reader.state()) {
-            case 5:
+            case 6:
                 restoreExchId = reader.readMessage("restoreExchId");
 
                 if (!reader.isLastRead())
@@ -131,7 +131,7 @@ public class GridDhtPartitionsSingleRequest extends GridDhtPartitionsAbstractMes
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 6;
+        return 7;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java
index bad1b61..9c69fdf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java
@@ -103,10 +103,11 @@ public class LatchAckMessage implements Message {
                 writer.incrementState();
 
             case 2:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
+
         }
 
         return true;
@@ -137,12 +138,13 @@ public class LatchAckMessage implements Message {
                 reader.incrementState();
 
             case 2:
-                topVer = reader.readMessage("topVer");
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
+
         }
 
         return reader.afterMessageRead(LatchAckMessage.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
index 45f3282..6418dc7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
@@ -313,9 +313,18 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @Override public GridDhtTopologyFuture topologyVersionFuture() {
-        assert topReadyFut != null;
+        GridDhtTopologyFuture topReadyFut0 = topReadyFut;
 
-        return topReadyFut;
+        assert topReadyFut0 != null;
+
+        if (!topReadyFut0.changedAffinity()) {
+            GridDhtTopologyFuture lastFut = ctx.exchange().lastFinishedFuture();
+
+            if (lastFut != null)
+                return lastFut;
+        }
+
+        return topReadyFut0;
     }
 
     /** {@inheritDoc} */
@@ -748,45 +757,47 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                 long updateSeq = this.updateSeq.incrementAndGet();
 
-                for (int p = 0; p < partitions; p++) {
-                    GridDhtLocalPartition locPart = localPartition0(p, topVer, false, true);
-
-                    if (partitionLocalNode(p, topVer)) {
-                        // Prepare partition to rebalance if it's not happened on full map update phase.
-                        if (locPart == null || locPart.state() == RENTING || locPart.state() == EVICTED)
-                            locPart = rebalancePartition(p, false);
+                if (!ctx.localNode().isClient()) {
+                    for (int p = 0; p < partitions; p++) {
+                        GridDhtLocalPartition locPart = localPartition0(p, topVer, false, true);
 
-                        GridDhtPartitionState state = locPart.state();
+                        if (partitionLocalNode(p, topVer)) {
+                            // Prepare partition to rebalance if it's not happened on full map update phase.
+                            if (locPart == null || locPart.state() == RENTING || locPart.state() == EVICTED)
+                                locPart = rebalancePartition(p, false);
 
-                        if (state == MOVING) {
-                            if (grp.rebalanceEnabled()) {
-                                Collection<ClusterNode> owners = owners(p);
+                            GridDhtPartitionState state = locPart.state();
 
-                                // If an owner node left during exchange, then new exchange should be started with detecting lost partitions.
-                                if (!F.isEmpty(owners)) {
-                                    if (log.isDebugEnabled())
-                                        log.debug("Will not own partition (there are owners to rebalance from) " +
-                                            "[grp=" + grp.cacheOrGroupName() + ", p=" + p + ", owners = " + owners + ']');
+                            if (state == MOVING) {
+                                if (grp.rebalanceEnabled()) {
+                                    Collection<ClusterNode> owners = owners(p);
+
+                                    // If an owner node left during exchange, then new exchange should be started with detecting lost partitions.
+                                    if (!F.isEmpty(owners)) {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Will not own partition (there are owners to rebalance from) " +
+                                                "[grp=" + grp.cacheOrGroupName() + ", p=" + p + ", owners = " + owners + ']');
+                                    }
                                 }
+                                else
+                                    updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
                             }
-                            else
-                                updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
                         }
-                    }
-                    else {
-                        if (locPart != null) {
-                            GridDhtPartitionState state = locPart.state();
+                        else {
+                            if (locPart != null) {
+                                GridDhtPartitionState state = locPart.state();
 
-                            if (state == MOVING) {
-                                locPart.rent(false);
+                                if (state == MOVING) {
+                                    locPart.rent(false);
 
-                                updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
+                                    updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
 
-                                changed = true;
+                                    changed = true;
 
-                                if (log.isDebugEnabled()) {
-                                    log.debug("Evicting " + state + " partition (it does not belong to affinity) [" +
-                                        "grp=" + grp.cacheOrGroupName() + ", p=" + locPart.id() + ']');
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("Evicting " + state + " partition (it does not belong to affinity) [" +
+                                            "grp=" + grp.cacheOrGroupName() + ", p=" + locPart.id() + ']');
+                                    }
                                 }
                             }
                         }
@@ -1131,25 +1142,38 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
             List<ClusterNode> nodes = null;
 
-            if (!topVer.equals(diffFromAffinityVer)) {
-                LT.warn(log, "Requested topology version does not match calculated diff, will require full iteration to" +
-                    "calculate mapping [grp=" + grp.cacheOrGroupName() + ", topVer=" + topVer +
-                    ", diffVer=" + diffFromAffinityVer + "]");
+            AffinityTopologyVersion diffVer = diffFromAffinityVer;
 
-                nodes = new ArrayList<>();
+            if (!diffVer.equals(topVer)) {
+                LT.warn(log, "Requested topology version does not match calculated diff, need to check if " +
+                    "affinity has changed [grp=" + grp.cacheOrGroupName() + ", topVer=" + topVer +
+                    ", diffVer=" + diffVer + "]");
 
-                nodes.addAll(affNodes);
+                boolean affChanged;
 
-                for (Map.Entry<UUID, GridDhtPartitionMap> entry : node2part.entrySet()) {
-                    GridDhtPartitionState state = entry.getValue().get(p);
+                if (diffVer.compareTo(topVer) < 0)
+                    affChanged = ctx.exchange().affinityChanged(diffVer, topVer);
+                else
+                    affChanged = ctx.exchange().affinityChanged(topVer, diffVer);
 
-                    ClusterNode n = ctx.discovery().node(entry.getKey());
+                if (affChanged) {
+                    LT.warn(log, "Requested topology version does not match calculated diff, will require full iteration to" +
+                        "calculate mapping [grp=" + grp.cacheOrGroupName() + ", topVer=" + topVer +
+                        ", diffVer=" + diffVer + "]");
 
-                    if (n != null && state != null && (state == MOVING || state == OWNING || state == RENTING)
-                        && !nodes.contains(n) && (topVer.topologyVersion() < 0 || n.order() <= topVer.topologyVersion())) {
-                        nodes.add(n);
-                    }
+                    nodes = new ArrayList<>();
+
+                    nodes.addAll(affNodes);
+
+                    for (Map.Entry<UUID, GridDhtPartitionMap> entry : node2part.entrySet()) {
+                        GridDhtPartitionState state = entry.getValue().get(p);
 
+                        ClusterNode n = ctx.discovery().node(entry.getKey());
+
+                        if (n != null && state != null && (state == MOVING || state == OWNING || state == RENTING)
+                            && !nodes.contains(n) && (topVer.topologyVersion() < 0 || n.order() <= topVer.topologyVersion()))
+                            nodes.add(n);
+                    }
                 }
 
                 return nodes;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/CacheVersionedValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/CacheVersionedValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/CacheVersionedValue.java
index 9670f8a..c19d486 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/CacheVersionedValue.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/CacheVersionedValue.java
@@ -174,4 +174,4 @@ public class CacheVersionedValue implements Message {
     @Override public String toString() {
         return S.toString(CacheVersionedValue.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
index 2531143..247a1f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
@@ -375,79 +375,79 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeLong("accessTtl", accessTtl))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeLong("createTtl", createTtl))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeCollection("readersFlags", readersFlags, MessageCollectionItemType.BOOLEAN))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
-                if (!writer.writeMessage("topVer", topVer))
+            case 14:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeString("txLbl", txLbl))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeMessage("ver", ver))
                     return false;
 
@@ -469,7 +469,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 accessTtl = reader.readLong("accessTtl");
 
                 if (!reader.isLastRead())
@@ -477,7 +477,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 createTtl = reader.readLong("createTtl");
 
                 if (!reader.isLastRead())
@@ -485,7 +485,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -493,7 +493,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -501,7 +501,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -509,7 +509,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -517,7 +517,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 mvccSnapshot = reader.readMessage("mvccSnapshot");
 
                 if (!reader.isLastRead())
@@ -525,7 +525,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 readersFlags = reader.readCollection("readersFlags", MessageCollectionItemType.BOOLEAN);
 
                 if (!reader.isLastRead())
@@ -533,7 +533,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -541,7 +541,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -549,15 +549,15 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 13:
-                topVer = reader.readMessage("topVer");
+            case 14:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 txLbl = reader.readString("txLbl");
 
                 if (!reader.isLastRead())
@@ -565,7 +565,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 ver = reader.readMessage("ver");
 
                 if (!reader.isLastRead())
@@ -585,7 +585,7 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 16;
+        return 17;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
index b4e4424..578c46b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
@@ -228,43 +228,43 @@ public class GridNearGetResponse extends GridCacheIdMessage implements GridCache
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeCollection("entries", entries, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeCollection("invalidParts", invalidParts, MessageCollectionItemType.INT))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
-                if (!writer.writeMessage("topVer", topVer))
+            case 9:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeMessage("ver", ver))
                     return false;
 
@@ -286,7 +286,7 @@ public class GridNearGetResponse extends GridCacheIdMessage implements GridCache
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 entries = reader.readCollection("entries", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -294,7 +294,7 @@ public class GridNearGetResponse extends GridCacheIdMessage implements GridCache
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
@@ -302,7 +302,7 @@ public class GridNearGetResponse extends GridCacheIdMessage implements GridCache
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -310,7 +310,7 @@ public class GridNearGetResponse extends GridCacheIdMessage implements GridCache
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 invalidParts = reader.readCollection("invalidParts", MessageCollectionItemType.INT);
 
                 if (!reader.isLastRead())
@@ -318,7 +318,7 @@ public class GridNearGetResponse extends GridCacheIdMessage implements GridCache
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -326,15 +326,15 @@ public class GridNearGetResponse extends GridCacheIdMessage implements GridCache
 
                 reader.incrementState();
 
-            case 8:
-                topVer = reader.readMessage("topVer");
+            case 9:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 ver = reader.readMessage("ver");
 
                 if (!reader.isLastRead())
@@ -354,7 +354,7 @@ public class GridNearGetResponse extends GridCacheIdMessage implements GridCache
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 10;
+        return 11;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
index fb7295c..ce913aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
@@ -377,61 +377,61 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
         }
 
         switch (writer.state()) {
-            case 20:
+            case 21:
                 if (!writer.writeLong("accessTtl", accessTtl))
                     return false;
 
                 writer.incrementState();
 
-            case 21:
+            case 22:
                 if (!writer.writeLong("createTtl", createTtl))
                     return false;
 
                 writer.incrementState();
 
-            case 22:
+            case 23:
                 if (!writer.writeObjectArray("dhtVers", dhtVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 23:
+            case 24:
                 if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 25:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 25:
+            case 26:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 26:
+            case 27:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 27:
+            case 28:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 28:
-                if (!writer.writeMessage("topVer", topVer))
+            case 29:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 29:
+            case 30:
                 if(!writer.writeString("txLbl", txLbl))
                     return false;
 
@@ -452,7 +452,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
             return false;
 
         switch (reader.state()) {
-            case 20:
+            case 21:
                 accessTtl = reader.readLong("accessTtl");
 
                 if (!reader.isLastRead())
@@ -460,7 +460,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 21:
+            case 22:
                 createTtl = reader.readLong("createTtl");
 
                 if (!reader.isLastRead())
@@ -468,7 +468,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 22:
+            case 23:
                 dhtVers = reader.readObjectArray("dhtVers", MessageCollectionItemType.MSG, GridCacheVersion.class);
 
                 if (!reader.isLastRead())
@@ -476,7 +476,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 23:
+            case 24:
                 filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class);
 
                 if (!reader.isLastRead())
@@ -484,7 +484,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 24:
+            case 25:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -492,7 +492,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 25:
+            case 26:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -500,7 +500,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 26:
+            case 27:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -508,7 +508,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 27:
+            case 28:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -516,15 +516,15 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 28:
-                topVer = reader.readMessage("topVer");
+            case 29:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 29:
+            case 30:
                 txLbl = reader.readString("txLbl");
 
                 if(!reader.isLastRead())
@@ -544,7 +544,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 30;
+        return 31;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java
index e88f0a0..b6c6d8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java
@@ -208,37 +208,37 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
         }
 
         switch (writer.state()) {
-            case 10:
-                if (!writer.writeMessage("clientRemapVer", clientRemapVer))
+            case 11:
+                if (!writer.writeAffinityTopologyVersion("clientRemapVer", clientRemapVer))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 12:
                 if (!writer.writeObjectArray("dhtVers", dhtVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeBooleanArray("filterRes", filterRes))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 14:
                 if (!writer.writeObjectArray("mappedVers", mappedVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 15:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 16:
                 if (!writer.writeCollection("pending", pending, MessageCollectionItemType.MSG))
                     return false;
 
@@ -260,15 +260,15 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
             return false;
 
         switch (reader.state()) {
-            case 10:
-                clientRemapVer = reader.readMessage("clientRemapVer");
+            case 11:
+                clientRemapVer = reader.readAffinityTopologyVersion("clientRemapVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 11:
+            case 12:
                 dhtVers = reader.readObjectArray("dhtVers", MessageCollectionItemType.MSG, GridCacheVersion.class);
 
                 if (!reader.isLastRead())
@@ -276,7 +276,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 filterRes = reader.readBooleanArray("filterRes");
 
                 if (!reader.isLastRead())
@@ -284,7 +284,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
 
                 reader.incrementState();
 
-            case 13:
+            case 14:
                 mappedVers = reader.readObjectArray("mappedVers", MessageCollectionItemType.MSG, GridCacheVersion.class);
 
                 if (!reader.isLastRead())
@@ -292,7 +292,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
 
                 reader.incrementState();
 
-            case 14:
+            case 15:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -300,7 +300,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 pending = reader.readCollection("pending", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -320,7 +320,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 16;
+        return 17;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java
index c87e521..3040e5c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java
@@ -311,7 +311,7 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 4:
                 accessTtl = reader.readLong("accessTtl");
 
                 if (!reader.isLastRead())
@@ -319,7 +319,7 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 createTtl = reader.readLong("createTtl");
 
                 if (!reader.isLastRead())
@@ -327,7 +327,7 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
@@ -335,7 +335,7 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -343,7 +343,7 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 7:
+            case 8:
                 key = reader.readMessage("key");
 
                 if (!reader.isLastRead())
@@ -351,7 +351,7 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 8:
+            case 9:
                 mvccSnapshot = reader.readMessage("mvccSnapshot");
 
                 if (!reader.isLastRead())
@@ -359,7 +359,7 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 9:
+            case 10:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -367,7 +367,7 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 10:
+            case 11:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -375,15 +375,15 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
 
                 reader.incrementState();
 
-            case 11:
-                topVer = reader.readMessage("topVer");
+            case 12:
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
-            case 12:
+            case 13:
                 txLbl = reader.readString("txLbl");
 
                 if (!reader.isLastRead())
@@ -411,61 +411,61 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
         }
 
         switch (writer.state()) {
-            case 3:
+            case 4:
                 if (!writer.writeLong("accessTtl", accessTtl))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeLong("createTtl", createTtl))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 8:
                 if (!writer.writeMessage("key", key))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 9:
                 if (!writer.writeMessage("mvccSnapshot", mvccSnapshot))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 10:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 11:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
-                if (!writer.writeMessage("topVer", topVer))
+            case 12:
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 13:
                 if (!writer.writeString("txLbl", txLbl))
                     return false;
 
@@ -488,7 +488,7 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 13;
+        return 14;
     }
 
     /** {@inheritDoc} */


[10/50] [abbrv] ignite git commit: IGNITE-10237 Fixed inspections

Posted by sb...@apache.org.
IGNITE-10237 Fixed inspections


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

Branch: refs/heads/ignite-10044
Commit: 6923bd8c9cbd8562459f84a3b86bc7760dd22aa7
Parents: 2311afd
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Nov 16 22:45:42 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Nov 16 22:45:42 2018 +0300

----------------------------------------------------------------------
 .../platform/client/cache/ClientCacheNodePartitionsRequest.java     | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6923bd8c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java
index 377d26f..5e33860 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.odbc.ClientConnectableNodePartitions;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;


[06/50] [abbrv] ignite git commit: merge

Posted by sb...@apache.org.
 merge


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

Branch: refs/heads/ignite-10044
Commit: fb8fd071b1d69f2cb054d3798320380eb3401362
Parents: 9a6d642 64b19a9
Author: sboikov <sb...@apache.org>
Authored: Fri Nov 16 16:37:04 2018 +0300
Committer: sboikov <sb...@apache.org>
Committed: Fri Nov 16 16:37:04 2018 +0300

----------------------------------------------------------------------
 .../internal/pagemem/PageIdAllocator.java       |   3 +
 .../processors/cache/CacheGroupContext.java     |   1 -
 .../cache/GridCacheConcurrentMapImpl.java       |   1 -
 .../processors/cache/GridCacheContext.java      |  34 +-
 .../cache/GridCacheEvictionManager.java         |   1 -
 .../cache/distributed/dht/GridDhtGetFuture.java |  40 +-
 .../distributed/dht/GridDhtGetSingleFuture.java |  41 +-
 .../dht/GridDhtTopologyFutureAdapter.java       | 233 ++++---
 .../dht/GridPartitionedGetFuture.java           |   2 +-
 .../dht/GridPartitionedSingleGetFuture.java     |   9 +-
 .../dht/preloader/GridDhtPartitionDemander.java |   1 -
 .../GridDhtPartitionsExchangeFuture.java        |   4 +-
 .../dht/topology/GridDhtLocalPartition.java     |   1 -
 .../distributed/near/GridNearGetFuture.java     |   2 +-
 .../persistence/metastorage/MetaStorage.java    |   7 +
 .../metastorage/MetastorageTree.java            |   8 +
 .../wal/reader/IgniteWalIteratorFactory.java    |   3 +-
 .../reader/StandaloneWalRecordsIterator.java    |   6 +-
 .../wal/reader/StrictBoundsCheckException.java  |  35 +
 .../odbc/odbc/OdbcRequestHandler.java           |  12 +-
 .../processors/odbc/odbc/OdbcUtils.java         |  22 +
 .../cache/ClientCacheNodePartitionsRequest.java |   3 +-
 .../processors/query/GridQueryIndexing.java     |  23 +-
 .../processors/query/GridQueryProcessor.java    | 275 ++++----
 .../processors/query/QueryKeyValueIterable.java |  53 ++
 .../processors/query/QueryKeyValueIterator.java |  68 ++
 .../visor/node/VisorNodeDataCollectorJob.java   |   3 +-
 ...IgniteClientCacheInitializationFailTest.java |  19 +-
 ...CacheResultIsNotNullOnPartitionLossTest.java |  23 +-
 .../IgniteCachePartitionLossPolicySelfTest.java | 650 +++++++------------
 .../metastorage/IgniteMetaStorageBasicTest.java |  77 +++
 .../processors/query/h2/IgniteH2Indexing.java   | 285 ++------
 ...CacheAbstractQueryDetailMetricsSelfTest.java |  15 +-
 .../CacheAbstractQueryMetricsSelfTest.java      |  13 -
 ...niteCacheLockPartitionOnAffinityRunTest.java |   2 +
 ...ndexingCachePartitionLossPolicySelfTest.java |   2 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 639 +-----------------
 ...H2ResultSetIteratorNullifyOnEndSelfTest.java | 186 ------
 .../cpp/odbc-test/src/meta_queries_test.cpp     |  43 ++
 .../cpp/odbc-test/src/sql_get_info_test.cpp     |   2 +-
 .../cpp/odbc/src/config/connection_info.cpp     |   2 +-
 .../thin-client-test/src/cache_client_test.cpp  |  27 +
 42 files changed, 1055 insertions(+), 1821 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fb8fd071/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------


[17/50] [abbrv] ignite git commit: IGNITE-8867: [ML] Bagging on learning sample

Posted by sb...@apache.org.
IGNITE-8867: [ML] Bagging on learning sample

this closes #5058


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

Branch: refs/heads/ignite-10044
Commit: 355ce6fe8839ea707bded79a6c21a2f74451366b
Parents: 28cb3a0
Author: Artem Malykh <am...@gmail.com>
Authored: Mon Nov 19 00:59:56 2018 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Mon Nov 19 00:59:56 2018 +0300

----------------------------------------------------------------------
 ...ggedLogisticRegressionSGDTrainerExample.java | 108 ++++++
 .../logistic/bagged/package-info.java           |  22 ++
 .../ml/composition/BaggingModelTrainer.java     | 200 ----------
 .../ignite/ml/dataset/DatasetBuilder.java       |  11 +
 .../ml/dataset/PartitionContextBuilder.java     |  21 ++
 .../ignite/ml/dataset/PartitionDataBuilder.java |  11 +-
 .../ignite/ml/dataset/UpstreamTransformer.java  |  42 +++
 .../ml/dataset/UpstreamTransformerChain.java    | 154 ++++++++
 .../dataset/impl/cache/CacheBasedDataset.java   |  15 +-
 .../impl/cache/CacheBasedDatasetBuilder.java    |  15 +-
 .../dataset/impl/cache/util/ComputeUtils.java   |  95 +++--
 .../ml/dataset/impl/local/LocalDataset.java     |   2 +-
 .../dataset/impl/local/LocalDatasetBuilder.java |  95 +++--
 .../environment/LearningEnvironmentBuilder.java |   2 +-
 .../binomial/LogisticRegressionSGDTrainer.java  |   9 +-
 .../ignite/ml/trainers/DatasetTrainer.java      |   1 +
 .../ignite/ml/trainers/TrainerTransformers.java | 376 +++++++++++++++++++
 .../BaggingUpstreamTransformer.java             |  58 +++
 .../ml/trainers/transformers/package-info.java  |  22 ++
 .../impurity/ImpurityHistogramsComputer.java    |   2 +-
 .../java/org/apache/ignite/ml/util/Utils.java   |  32 ++
 .../org/apache/ignite/ml/IgniteMLTestSuite.java |   4 +-
 .../impl/cache/util/ComputeUtilsTest.java       |   3 +
 .../apache/ignite/ml/trainers/BaggingTest.java  | 218 +++++++++++
 24 files changed, 1261 insertions(+), 257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/bagged/BaggedLogisticRegressionSGDTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/bagged/BaggedLogisticRegressionSGDTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/bagged/BaggedLogisticRegressionSGDTrainerExample.java
new file mode 100644
index 0000000..baf513a
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/bagged/BaggedLogisticRegressionSGDTrainerExample.java
@@ -0,0 +1,108 @@
+/*
+ * 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.examples.ml.regression.logistic.bagged;
+
+import java.io.FileNotFoundException;
+import java.util.Arrays;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.examples.ml.util.MLSandboxDatasets;
+import org.apache.ignite.examples.ml.util.SandboxMLCache;
+import org.apache.ignite.ml.composition.ModelsComposition;
+import org.apache.ignite.ml.composition.predictionsaggregator.OnMajorityPredictionsAggregator;
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+import org.apache.ignite.ml.nn.UpdatesStrategy;
+import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate;
+import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator;
+import org.apache.ignite.ml.regressions.logistic.binomial.LogisticRegressionSGDTrainer;
+import org.apache.ignite.ml.selection.cv.CrossValidation;
+import org.apache.ignite.ml.selection.scoring.metric.Accuracy;
+import org.apache.ignite.ml.trainers.DatasetTrainer;
+import org.apache.ignite.ml.trainers.TrainerTransformers;
+
+/**
+ * This example shows how bagging technique may be applied to arbitrary trainer.
+ * As an example (a bit synthetic) logistic regression is considered.
+ * <p>
+ * Code in this example launches Ignite grid and fills the cache with test data points (based on the
+ * <a href="https://en.wikipedia.org/wiki/Iris_flower_data_set"></a>Iris dataset</a>).</p>
+ * <p>
+ * After that it trains bootstrapped (or bagged) version of logistic regression trainer. Bootstrapping is done
+ * on both samples and features (<a href="https://en.wikipedia.org/wiki/Bootstrap_aggregating"></a>Samples bagging</a>,
+ * <a href="https://en.wikipedia.org/wiki/Random_subspace_method"></a>Features bagging</a>).</p>
+ * <p>
+ * Finally, this example applies cross-validation to resulted model and prints accuracy if each fold.</p>
+ */
+public class BaggedLogisticRegressionSGDTrainerExample {
+    /** Run example. */
+    public static void main(String[] args) throws FileNotFoundException {
+        System.out.println();
+        System.out.println(">>> Logistic regression model over partitioned dataset usage example started.");
+        // Start ignite grid.
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            System.out.println(">>> Ignite grid started.");
+
+            IgniteCache<Integer, Vector> dataCache = new SandboxMLCache(ignite)
+                .fillCacheWith(MLSandboxDatasets.TWO_CLASSED_IRIS);
+
+            System.out.println(">>> Create new logistic regression trainer object.");
+            LogisticRegressionSGDTrainer<?> trainer = new LogisticRegressionSGDTrainer<>()
+                .withUpdatesStgy(new UpdatesStrategy<>(
+                    new SimpleGDUpdateCalculator(0.2),
+                    SimpleGDParameterUpdate::sumLocal,
+                    SimpleGDParameterUpdate::avg
+                ))
+                .withMaxIterations(100000)
+                .withLocIterations(100)
+                .withBatchSize(10)
+                .withSeed(123L);
+
+            System.out.println(">>> Perform the training to get the model.");
+
+            DatasetTrainer< ModelsComposition, Double> baggedTrainer = TrainerTransformers.makeBagged(
+                trainer,
+                10,
+                0.6,
+                4,
+                3,
+                new OnMajorityPredictionsAggregator(),
+                123L);
+
+            System.out.println(">>> Perform evaluation of the model.");
+
+            double[] score = new CrossValidation<ModelsComposition, Double, Integer, Vector>().score(
+                baggedTrainer,
+                new Accuracy<>(),
+                ignite,
+                dataCache,
+                (k, v) -> v.copyOfRange(1, v.size()),
+                (k, v) -> v.get(0),
+                3
+            );
+
+            System.out.println(">>> ---------------------------------");
+
+            Arrays.stream(score).forEach(sc -> {
+                System.out.println("\n>>> Accuracy " + sc);
+            });
+
+            System.out.println(">>> Bagged logistic regression model over partitioned dataset usage example completed.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/bagged/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/bagged/package-info.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/bagged/package-info.java
new file mode 100644
index 0000000..ea0d19e
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/bagged/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * ML bagged logistic regression examples.
+ */
+package org.apache.ignite.examples.ml.regression.logistic.bagged;

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/composition/BaggingModelTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/BaggingModelTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/BaggingModelTrainer.java
deleted file mode 100644
index 493c1da..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/BaggingModelTrainer.java
+++ /dev/null
@@ -1,200 +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.ml.composition;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-import org.apache.ignite.ml.Model;
-import org.apache.ignite.ml.composition.predictionsaggregator.PredictionsAggregator;
-import org.apache.ignite.ml.dataset.DatasetBuilder;
-import org.apache.ignite.ml.environment.logging.MLLogger;
-import org.apache.ignite.ml.environment.parallelism.Promise;
-import org.apache.ignite.ml.math.functions.IgniteBiFunction;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.math.primitives.vector.Vector;
-import org.apache.ignite.ml.math.primitives.vector.VectorUtils;
-import org.apache.ignite.ml.selection.split.mapper.SHA256UniformMapper;
-import org.apache.ignite.ml.trainers.DatasetTrainer;
-import org.apache.ignite.ml.util.Utils;
-import org.jetbrains.annotations.NotNull;
-
-/**
- * Abstract trainer implementing bagging logic. In each learning iteration the algorithm trains one model on subset of
- * learning sample and subspace of features space. Each model is produced from same model-class [e.g. Decision Trees].
- */
-public abstract class BaggingModelTrainer extends DatasetTrainer<ModelsComposition, Double> {
-    /**
-     * Predictions aggregator.
-     */
-    private final PredictionsAggregator predictionsAggregator;
-    /**
-     * Number of features to draw from original features vector to train each model.
-     */
-    private final int maximumFeaturesCntPerMdl;
-    /**
-     * Ensemble size.
-     */
-    private final int ensembleSize;
-    /**
-     * Size of sample part in percent to train one model.
-     */
-    private final double samplePartSizePerMdl;
-    /**
-     * Feature vector size.
-     */
-    private final int featureVectorSize;
-
-    /**
-     * Constructs new instance of BaggingModelTrainer.
-     *
-     * @param predictionsAggregator Predictions aggregator.
-     * @param featureVectorSize Feature vector size.
-     * @param maximumFeaturesCntPerMdl Number of features to draw from original features vector to train each model.
-     * @param ensembleSize Ensemble size.
-     * @param samplePartSizePerMdl Size of sample part in percent to train one model.
-     */
-    public BaggingModelTrainer(PredictionsAggregator predictionsAggregator,
-        int featureVectorSize,
-        int maximumFeaturesCntPerMdl,
-        int ensembleSize,
-        double samplePartSizePerMdl) {
-
-        this.predictionsAggregator = predictionsAggregator;
-        this.maximumFeaturesCntPerMdl = maximumFeaturesCntPerMdl;
-        this.ensembleSize = ensembleSize;
-        this.samplePartSizePerMdl = samplePartSizePerMdl;
-        this.featureVectorSize = featureVectorSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> ModelsComposition fit(DatasetBuilder<K, V> datasetBuilder,
-        IgniteBiFunction<K, V, Vector> featureExtractor,
-        IgniteBiFunction<K, V, Double> lbExtractor) {
-
-        MLLogger log = environment.logger(getClass());
-        log.log(MLLogger.VerboseLevel.LOW, "Start learning");
-
-        Long startTs = System.currentTimeMillis();
-
-        List<IgniteSupplier<ModelOnFeaturesSubspace>> tasks = new ArrayList<>();
-        for(int i = 0; i < ensembleSize; i++)
-            tasks.add(() -> learnModel(datasetBuilder, featureExtractor, lbExtractor));
-
-        List<Model<Vector, Double>> models = environment.parallelismStrategy().submit(tasks)
-            .stream().map(Promise::unsafeGet)
-            .collect(Collectors.toList());
-
-        double learningTime = (double)(System.currentTimeMillis() - startTs) / 1000.0;
-        log.log(MLLogger.VerboseLevel.LOW, "The training time was %.2fs", learningTime);
-        log.log(MLLogger.VerboseLevel.LOW, "Learning finished");
-        return new ModelsComposition(models, predictionsAggregator);
-    }
-
-    /**
-     * Trains one model on part of sample and features subspace.
-     *
-     * @param datasetBuilder Dataset builder.
-     * @param featureExtractor Feature extractor.
-     * @param lbExtractor Label extractor.
-     */
-    @NotNull private <K, V> ModelOnFeaturesSubspace learnModel(
-        DatasetBuilder<K, V> datasetBuilder,
-        IgniteBiFunction<K, V, Vector> featureExtractor,
-        IgniteBiFunction<K, V, Double> lbExtractor) {
-
-        Random rnd = new Random();
-        SHA256UniformMapper<K, V> sampleFilter = new SHA256UniformMapper<>(rnd);
-        long featureExtractorSeed = rnd.nextLong();
-        Map<Integer, Integer> featuresMapping = createFeaturesMapping(featureExtractorSeed, featureVectorSize);
-
-        //TODO: IGNITE-8867 Need to implement bootstrapping algorithm
-        Long startTs = System.currentTimeMillis();
-        Model<Vector, Double> mdl = buildDatasetTrainerForModel().fit(
-            datasetBuilder.withFilter((features, answer) -> sampleFilter.map(features, answer) < samplePartSizePerMdl),
-            wrapFeatureExtractor(featureExtractor, featuresMapping),
-            lbExtractor);
-        double learningTime = (double)(System.currentTimeMillis() - startTs) / 1000.0;
-        environment.logger(getClass()).log(MLLogger.VerboseLevel.HIGH, "One model training time was %.2fs", learningTime);
-
-        return new ModelOnFeaturesSubspace(featuresMapping, mdl);
-    }
-
-    /**
-     * Constructs mapping from original feature vector to subspace.
-     *
-     * @param seed Seed.
-     * @param featuresVectorSize Features vector size.
-     */
-    private Map<Integer, Integer> createFeaturesMapping(long seed, int featuresVectorSize) {
-        int[] featureIdxs = Utils.selectKDistinct(featuresVectorSize, maximumFeaturesCntPerMdl, new Random(seed));
-        Map<Integer, Integer> locFeaturesMapping = new HashMap<>();
-
-        IntStream.range(0, maximumFeaturesCntPerMdl)
-            .forEach(localId -> locFeaturesMapping.put(localId, featureIdxs[localId]));
-
-        return locFeaturesMapping;
-    }
-
-    /**
-     * Creates trainer specific to ensemble.
-     */
-    protected abstract DatasetTrainer<? extends Model<Vector, Double>, Double> buildDatasetTrainerForModel();
-
-    /**
-     * Wraps the original feature extractor with features subspace mapping applying.
-     *
-     * @param featureExtractor Feature extractor.
-     * @param featureMapping Feature mapping.
-     */
-    private <K, V> IgniteBiFunction<K, V, Vector> wrapFeatureExtractor(
-        IgniteBiFunction<K, V, Vector> featureExtractor,
-        Map<Integer, Integer> featureMapping) {
-
-        return featureExtractor.andThen((IgniteFunction<Vector, Vector>)featureValues -> {
-            double[] newFeaturesValues = new double[featureMapping.size()];
-            featureMapping.forEach((localId, featureValueId) -> newFeaturesValues[localId] = featureValues.get(featureValueId));
-            return VectorUtils.of(newFeaturesValues);
-        });
-    }
-
-    /**
-     * Learn new models on dataset and create new Compositions over them and already learned models.
-     *
-     * @param mdl Learned model.
-     * @param datasetBuilder Dataset builder.
-     * @param featureExtractor Feature extractor.
-     * @param lbExtractor Label extractor.
-     * @param <K> Type of a key in {@code upstream} data.
-     * @param <V> Type of a value in {@code upstream} data.
-     * @return New models composition.
-     */
-    @Override public <K, V> ModelsComposition updateModel(ModelsComposition mdl, DatasetBuilder<K, V> datasetBuilder,
-        IgniteBiFunction<K, V, Vector> featureExtractor, IgniteBiFunction<K, V, Double> lbExtractor) {
-
-        ArrayList<Model<Vector, Double>> newModels = new ArrayList<>(mdl.getModels());
-        newModels.addAll(fit(datasetBuilder, featureExtractor, lbExtractor).getModels());
-
-        return new ModelsComposition(newModels, predictionsAggregator);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java
index 19bdde9..4dd0a96 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java
@@ -21,6 +21,7 @@ import java.io.Serializable;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
+import org.apache.ignite.ml.trainers.transformers.BaggingUpstreamTransformer;
 
 /**
  * A builder constructing instances of a {@link Dataset}. Implementations of this interface encapsulate logic of
@@ -48,6 +49,16 @@ public interface DatasetBuilder<K, V> {
     public <C extends Serializable, D extends AutoCloseable> Dataset<C, D> build(
         PartitionContextBuilder<K, V, C> partCtxBuilder, PartitionDataBuilder<K, V, C, D> partDataBuilder);
 
+    /**
+     * Get upstream transformers chain. This chain is applied to upstream data before it is passed
+     * to {@link PartitionDataBuilder} and {@link PartitionContextBuilder}. This is needed to allow
+     * transformation to upstream data which are agnostic of any changes that happen after.
+     * Such transformations may be used for deriving meta-algorithms such as bagging
+     * (see {@link BaggingUpstreamTransformer}).
+     *
+     * @return Upstream transformers chain.
+     */
+    public UpstreamTransformerChain<K, V> upstreamTransformersChain();
 
     /**
      * Returns new instance of DatasetBuilder using conjunction of internal filter and {@code filterToAdd}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java
index 027ec34..6e1fec3 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java
@@ -19,6 +19,7 @@ package org.apache.ignite.ml.dataset;
 
 import java.io.Serializable;
 import java.util.Iterator;
+import java.util.stream.Stream;
 import org.apache.ignite.ml.dataset.primitive.builder.context.EmptyContextBuilder;
 import org.apache.ignite.ml.math.functions.IgniteFunction;
 
@@ -37,6 +38,10 @@ import org.apache.ignite.ml.math.functions.IgniteFunction;
 public interface PartitionContextBuilder<K, V, C extends Serializable> extends Serializable {
     /**
      * Builds a new partition {@code context} from an {@code upstream} data.
+     * Important: there is no guarantee that there will be no more than one UpstreamEntry with given key,
+     * UpstreamEntry should be thought rather as a container saving all data from upstream, but omitting uniqueness
+     * constraint. This constraint is omitted to allow upstream data transformers in {@link DatasetBuilder} replicating
+     * entries. For example it can be useful for bootstrapping.
      *
      * @param upstreamData Partition {@code upstream} data.
      * @param upstreamDataSize Partition {@code upstream} data size.
@@ -44,6 +49,22 @@ public interface PartitionContextBuilder<K, V, C extends Serializable> extends S
      */
     public C build(Iterator<UpstreamEntry<K, V>> upstreamData, long upstreamDataSize);
 
+
+    /**
+     * Builds a new partition {@code context} from an {@code upstream} data.
+     * Important: there is no guarantee that there will be no more than one UpstreamEntry with given key,
+     * UpstreamEntry should be thought rather as a container saving all data from upstream, but omitting uniqueness
+     * constraint. This constraint is omitted to allow upstream data transformers in {@link DatasetBuilder} replicating
+     * entries. For example it can be useful for bootstrapping.
+     *
+     * @param upstreamData Partition {@code upstream} data.
+     * @param upstreamDataSize Partition {@code upstream} data size.
+     * @return Partition {@code context}.
+     */
+    public default C build(Stream<UpstreamEntry<K, V>> upstreamData, long upstreamDataSize) {
+        return build(upstreamData.iterator(), upstreamDataSize);
+    }
+
     /**
      * Makes a composed partition {@code context} builder that first builds a {@code context} and then applies the
      * specified function on the result.

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java
index c1391b1..54c7611 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java
@@ -19,6 +19,7 @@ package org.apache.ignite.ml.dataset;
 
 import java.io.Serializable;
 import java.util.Iterator;
+import java.util.stream.Stream;
 import org.apache.ignite.ml.dataset.primitive.builder.data.SimpleDatasetDataBuilder;
 import org.apache.ignite.ml.dataset.primitive.builder.data.SimpleLabeledDatasetDataBuilder;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
@@ -39,7 +40,11 @@ import org.apache.ignite.ml.math.functions.IgniteBiFunction;
 @FunctionalInterface
 public interface PartitionDataBuilder<K, V, C extends Serializable, D extends AutoCloseable> extends Serializable {
     /**
-     * Builds a new partition {@code data} from a partition {@code upstream} data and partition {@code context}
+     * Builds a new partition {@code data} from a partition {@code upstream} data and partition {@code context}.
+     * Important: there is no guarantee that there will be no more than one UpstreamEntry with given key,
+     * UpstreamEntry should be thought rather as a container saving all data from upstream, but omitting uniqueness
+     * constraint. This constraint is omitted to allow upstream data transformers in {@link DatasetBuilder} replicating
+     * entries. For example it can be useful for bootstrapping.
      *
      * @param upstreamData Partition {@code upstream} data.
      * @param upstreamDataSize Partition {@code upstream} data size.
@@ -48,6 +53,10 @@ public interface PartitionDataBuilder<K, V, C extends Serializable, D extends Au
      */
     public D build(Iterator<UpstreamEntry<K, V>> upstreamData, long upstreamDataSize, C ctx);
 
+    public default D build(Stream<UpstreamEntry<K, V>> upstreamData, long upstreamDataSize, C ctx) {
+        return build(upstreamData.iterator(), upstreamDataSize, ctx);
+    }
+
     /**
      * Makes a composed partition {@code data} builder that first builds a {@code data} and then applies the specified
      * function on the result.

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamTransformer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamTransformer.java
new file mode 100644
index 0000000..ba70e2e
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamTransformer.java
@@ -0,0 +1,42 @@
+/*
+ * 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.ml.dataset;
+
+import java.io.Serializable;
+import java.util.Random;
+import java.util.stream.Stream;
+
+/**
+ * Interface of transformer of upstream.
+ *
+ * @param <K> Type of keys in the upstream.
+ * @param <V> Type of values in the upstream.
+ */
+@FunctionalInterface
+public interface UpstreamTransformer<K, V> extends Serializable {
+    /**
+     * Perform transformation of upstream.
+     *
+     * @param rnd Random numbers generator.
+     * @param upstream Upstream.
+     * @return Transformed upstream.
+     */
+    // TODO: IGNITE-10296: Inject capabilities of randomization through learning environment.
+    // TODO: IGNITE-10297: Investigate possibility of API change.
+    public Stream<UpstreamEntry<K, V>> transform(Random rnd, Stream<UpstreamEntry<K, V>> upstream);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamTransformerChain.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamTransformerChain.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamTransformerChain.java
new file mode 100644
index 0000000..dc83926
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamTransformerChain.java
@@ -0,0 +1,154 @@
+/*
+ * 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.ml.dataset;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Stream;
+import org.apache.ignite.ml.math.functions.IgniteFunction;
+
+/**
+ * Class representing chain of transformers applied to upstream.
+ *
+ * @param <K> Type of upstream keys.
+ * @param <V> Type of upstream values.
+ */
+public class UpstreamTransformerChain<K, V> implements Serializable {
+    /** Seed used for transformations. */
+    private Long seed;
+
+    /** List of upstream transformations. */
+    private List<UpstreamTransformer<K, V>> list;
+
+    /**
+     * Creates empty upstream transformers chain (basically identity function).
+     *
+     * @param <K> Type of upstream keys.
+     * @param <V> Type of upstream values.
+     * @return Empty upstream transformers chain.
+     */
+    public static <K, V> UpstreamTransformerChain<K, V> empty() {
+        return new UpstreamTransformerChain<>();
+    }
+
+    /**
+     * Creates upstream transformers chain consisting of one specified transformer.
+     *
+     * @param <K> Type of upstream keys.
+     * @param <V> Type of upstream values.
+     * @return Upstream transformers chain consisting of one specified transformer.
+     */
+    public static <K, V> UpstreamTransformerChain<K, V> of(UpstreamTransformer<K, V> trans) {
+        UpstreamTransformerChain<K, V> res = new UpstreamTransformerChain<>();
+        return res.addUpstreamTransformer(trans);
+    }
+
+    /**
+     * Construct instance of this class.
+     */
+    private UpstreamTransformerChain() {
+        list = new ArrayList<>();
+        seed = new Random().nextLong();
+    }
+
+    /**
+     * Adds upstream transformer to this chain.
+     *
+     * @param next Transformer to add.
+     * @return This chain with added transformer.
+     */
+    public UpstreamTransformerChain<K, V> addUpstreamTransformer(UpstreamTransformer<K, V> next) {
+        list.add(next);
+
+        return this;
+    }
+
+    /**
+     * Add upstream transformer based on given lambda.
+     *
+     * @param transformer Transformer.
+     * @return This object.
+     */
+    public UpstreamTransformerChain<K, V> addUpstreamTransformer(IgniteFunction<Stream<UpstreamEntry<K, V>>,
+        Stream<UpstreamEntry<K, V>>> transformer) {
+        return addUpstreamTransformer((rnd, upstream) -> transformer.apply(upstream));
+    }
+
+    /**
+     * Performs stream transformation using RNG based on provided seed as pseudo-randomness source for all
+     * transformers in the chain.
+     *
+     * @param upstream Upstream.
+     * @return Transformed upstream.
+     */
+    public Stream<UpstreamEntry<K, V>> transform(Stream<UpstreamEntry<K, V>> upstream) {
+        Random rnd = new Random(seed);
+
+        Stream<UpstreamEntry<K, V>> res = upstream;
+
+        for (UpstreamTransformer<K, V> kvUpstreamTransformer : list) {
+            res = kvUpstreamTransformer.transform(rnd, res);
+        }
+
+        return res;
+    }
+
+    /**
+     * Checks if this chain is empty.
+     *
+     * @return Result of check if this chain is empty.
+     */
+    public boolean isEmpty() {
+        return list.isEmpty();
+    }
+
+    /**
+     * Set seed for transformations.
+     *
+     * @param seed Seed.
+     * @return This object.
+     */
+    public UpstreamTransformerChain<K, V> setSeed(long seed) {
+        this.seed = seed;
+
+        return this;
+    }
+
+    /**
+     * Modifies seed for transformations if it is present.
+     *
+     * @param f Modification function.
+     * @return This object.
+     */
+    public UpstreamTransformerChain<K, V> modifySeed(IgniteFunction<Long, Long> f) {
+        seed = f.apply(seed);
+
+        return this;
+    }
+
+    /**
+     * Get seed used for RNG in transformations.
+     *
+     * @return Seed used for RNG in transformations.
+     */
+    public Long seed() {
+        return seed;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java
index e5eb483..0736906 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java
@@ -26,7 +26,9 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.ml.dataset.Dataset;
+import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.dataset.PartitionDataBuilder;
+import org.apache.ignite.ml.dataset.UpstreamTransformerChain;
 import org.apache.ignite.ml.dataset.impl.cache.util.ComputeUtils;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
 import org.apache.ignite.ml.math.functions.IgniteBinaryOperator;
@@ -59,6 +61,9 @@ public class CacheBasedDataset<K, V, C extends Serializable, D extends AutoClose
     /** Filter for {@code upstream} data. */
     private final IgniteBiPredicate<K, V> filter;
 
+    /** Chain of transformers applied to upstream. */
+    private final UpstreamTransformerChain<K, V> upstreamTransformers;
+
     /** Ignite Cache with partition {@code context}. */
     private final IgniteCache<Integer, C> datasetCache;
 
@@ -75,16 +80,22 @@ public class CacheBasedDataset<K, V, C extends Serializable, D extends AutoClose
      * @param ignite Ignite instance.
      * @param upstreamCache Ignite Cache with {@code upstream} data.
      * @param filter Filter for {@code upstream} data.
+     * @param upstreamTransformers Transformers of upstream data (see description in {@link DatasetBuilder}).
      * @param datasetCache Ignite Cache with partition {@code context}.
      * @param partDataBuilder Partition {@code data} builder.
      * @param datasetId Dataset ID.
      */
-    public CacheBasedDataset(Ignite ignite, IgniteCache<K, V> upstreamCache, IgniteBiPredicate<K, V> filter,
+    public CacheBasedDataset(
+        Ignite ignite,
+        IgniteCache<K, V> upstreamCache,
+        IgniteBiPredicate<K, V> filter,
+        UpstreamTransformerChain<K, V> upstreamTransformers,
         IgniteCache<Integer, C> datasetCache, PartitionDataBuilder<K, V, C, D> partDataBuilder,
         UUID datasetId) {
         this.ignite = ignite;
         this.upstreamCache = upstreamCache;
         this.filter = filter;
+        this.upstreamTransformers = upstreamTransformers;
         this.datasetCache = datasetCache;
         this.partDataBuilder = partDataBuilder;
         this.datasetId = datasetId;
@@ -102,6 +113,7 @@ public class CacheBasedDataset<K, V, C extends Serializable, D extends AutoClose
                 Ignition.localIgnite(),
                 upstreamCacheName,
                 filter,
+                upstreamTransformers,
                 datasetCacheName,
                 datasetId,
                 part,
@@ -131,6 +143,7 @@ public class CacheBasedDataset<K, V, C extends Serializable, D extends AutoClose
                 Ignition.localIgnite(),
                 upstreamCacheName,
                 filter,
+                upstreamTransformers,
                 datasetCacheName,
                 datasetId,
                 part,

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java
index 335ce63..1d00875 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java
@@ -27,6 +27,7 @@ import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.dataset.PartitionContextBuilder;
 import org.apache.ignite.ml.dataset.PartitionDataBuilder;
+import org.apache.ignite.ml.dataset.UpstreamTransformerChain;
 import org.apache.ignite.ml.dataset.impl.cache.util.ComputeUtils;
 import org.apache.ignite.ml.dataset.impl.cache.util.DatasetAffinityFunctionWrapper;
 
@@ -56,6 +57,9 @@ public class CacheBasedDatasetBuilder<K, V> implements DatasetBuilder<K, V> {
     /** Filter for {@code upstream} data. */
     private final IgniteBiPredicate<K, V> filter;
 
+    /** Chain of upstream transformers. */
+    private final UpstreamTransformerChain<K, V> transformersChain;
+
     /**
      * Constructs a new instance of cache based dataset builder that makes {@link CacheBasedDataset} with default
      * predicate that passes all upstream entries to dataset.
@@ -78,6 +82,7 @@ public class CacheBasedDatasetBuilder<K, V> implements DatasetBuilder<K, V> {
         this.ignite = ignite;
         this.upstreamCache = upstreamCache;
         this.filter = filter;
+        transformersChain = UpstreamTransformerChain.empty();
     }
 
     /** {@inheritDoc} */
@@ -102,16 +107,24 @@ public class CacheBasedDatasetBuilder<K, V> implements DatasetBuilder<K, V> {
             ignite,
             upstreamCache.getName(),
             filter,
+            transformersChain,
             datasetCache.getName(),
             partCtxBuilder,
             RETRIES,
             RETRY_INTERVAL
         );
 
-        return new CacheBasedDataset<>(ignite, upstreamCache, filter, datasetCache, partDataBuilder, datasetId);
+        return new CacheBasedDataset<>(ignite, upstreamCache, filter, transformersChain, datasetCache, partDataBuilder, datasetId);
     }
 
     /** {@inheritDoc} */
+    @Override public UpstreamTransformerChain<K, V> upstreamTransformersChain() {
+        return transformersChain;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
     @Override public DatasetBuilder<K, V> withFilter(IgniteBiPredicate<K, V> filterToAdd) {
         return new CacheBasedDatasetBuilder<>(ignite, upstreamCache,
             (e1, e2) -> filter.apply(e1, e2) && filterToAdd.apply(e1, e2));

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java
index a5cdd3b..6646e89 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java
@@ -27,6 +27,7 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.locks.LockSupport;
+import java.util.stream.Stream;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
@@ -40,13 +41,17 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.ml.dataset.PartitionContextBuilder;
 import org.apache.ignite.ml.dataset.PartitionDataBuilder;
 import org.apache.ignite.ml.dataset.UpstreamEntry;
+import org.apache.ignite.ml.dataset.UpstreamTransformerChain;
 import org.apache.ignite.ml.math.functions.IgniteFunction;
+import org.apache.ignite.ml.util.Utils;
 
 /**
  * Util class that provides common methods to perform computations on top of the Ignite Compute Grid.
  */
 public class ComputeUtils {
-    /** Template of the key used to store partition {@code data} in local storage. */
+    /**
+     * Template of the key used to store partition {@code data} in local storage.
+     */
     private static final String DATA_STORAGE_KEY_TEMPLATE = "part_data_storage_%s";
 
     /**
@@ -136,6 +141,7 @@ public class ComputeUtils {
      * @param ignite Ignite instance.
      * @param upstreamCacheName Name of an {@code upstream} cache.
      * @param filter Filter for {@code upstream} data.
+     * @param transformersChain Upstream transformers.
      * @param datasetCacheName Name of a partition {@code context} cache.
      * @param datasetId Dataset ID.
      * @param part Partition index.
@@ -146,8 +152,13 @@ public class ComputeUtils {
      * @param <D> Type of a partition {@code data}.
      * @return Partition {@code data}.
      */
-    public static <K, V, C extends Serializable, D extends AutoCloseable> D getData(Ignite ignite,
-        String upstreamCacheName, IgniteBiPredicate<K, V> filter, String datasetCacheName, UUID datasetId, int part,
+    public static <K, V, C extends Serializable, D extends AutoCloseable> D getData(
+        Ignite ignite,
+        String upstreamCacheName, IgniteBiPredicate<K, V> filter,
+        UpstreamTransformerChain<K, V> transformersChain,
+        String datasetCacheName,
+        UUID datasetId,
+        int part,
         PartitionDataBuilder<K, V, C, D> partDataBuilder) {
 
         PartitionDataStorage dataStorage = (PartitionDataStorage)ignite
@@ -166,13 +177,22 @@ public class ComputeUtils {
             qry.setPartition(part);
             qry.setFilter(filter);
 
-            long cnt = computeCount(upstreamCache, qry);
+            UpstreamTransformerChain<K, V> chainCopy = Utils.copy(transformersChain);
+            chainCopy.modifySeed(s -> s + part);
+
+            long cnt = computeCount(upstreamCache, qry, chainCopy);
 
             if (cnt > 0) {
                 try (QueryCursor<UpstreamEntry<K, V>> cursor = upstreamCache.query(qry,
                     e -> new UpstreamEntry<>(e.getKey(), e.getValue()))) {
 
-                    Iterator<UpstreamEntry<K, V>> iter = new IteratorWithConcurrentModificationChecker<>(cursor.iterator(), cnt,
+                    Iterator<UpstreamEntry<K, V>> it = cursor.iterator();
+                    if (!chainCopy.isEmpty()) {
+                        Stream<UpstreamEntry<K, V>> transformedStream = chainCopy.transform(Utils.asStream(it, cnt));
+                        it = transformedStream.iterator();
+                    }
+
+                    Iterator<UpstreamEntry<K, V>> iter = new IteratorWithConcurrentModificationChecker<>(it, cnt,
                         "Cache expected to be not modified during dataset data building [partition=" + part + ']');
 
                     return partDataBuilder.build(iter, cnt, ctx);
@@ -193,21 +213,25 @@ public class ComputeUtils {
         ignite.cluster().nodeLocalMap().remove(String.format(DATA_STORAGE_KEY_TEMPLATE, datasetId));
     }
 
-
     /**
      * Initializes partition {@code context} by loading it from a partition {@code upstream}.
-     *
+     *  @param <K> Type of a key in {@code upstream} data.
+     * @param <V> Type of a value in {@code upstream} data.
+     * @param <C> Type of a partition {@code context}.
      * @param ignite Ignite instance.
      * @param upstreamCacheName Name of an {@code upstream} cache.
      * @param filter Filter for {@code upstream} data.
-     * @param datasetCacheName Name of a partition {@code context} cache.
+     * @param transformersChain Upstream data {@link Stream} transformers chain.
      * @param ctxBuilder Partition {@code context} builder.
-     * @param <K> Type of a key in {@code upstream} data.
-     * @param <V> Type of a value in {@code upstream} data.
-     * @param <C> Type of a partition {@code context}.
      */
-    public static <K, V, C extends Serializable> void initContext(Ignite ignite, String upstreamCacheName,
-        IgniteBiPredicate<K, V> filter, String datasetCacheName, PartitionContextBuilder<K, V, C> ctxBuilder, int retries,
+    public static <K, V, C extends Serializable> void initContext(
+        Ignite ignite,
+        String upstreamCacheName,
+        IgniteBiPredicate<K, V> filter,
+        UpstreamTransformerChain<K, V> transformersChain,
+        String datasetCacheName,
+        PartitionContextBuilder<K, V, C> ctxBuilder,
+        int retries,
         int interval) {
         affinityCallWithRetries(ignite, Arrays.asList(datasetCacheName, upstreamCacheName), part -> {
             Ignite locIgnite = Ignition.localIgnite();
@@ -219,13 +243,23 @@ public class ComputeUtils {
             qry.setPartition(part);
             qry.setFilter(filter);
 
-            long cnt = computeCount(locUpstreamCache, qry);
-
             C ctx;
+            UpstreamTransformerChain<K, V> chainCopy = Utils.copy(transformersChain);
+            chainCopy.modifySeed(s -> s + part);
+
+            long cnt = computeCount(locUpstreamCache, qry, transformersChain);
+
             try (QueryCursor<UpstreamEntry<K, V>> cursor = locUpstreamCache.query(qry,
                 e -> new UpstreamEntry<>(e.getKey(), e.getValue()))) {
 
-                Iterator<UpstreamEntry<K, V>> iter = new IteratorWithConcurrentModificationChecker<>(cursor.iterator(), cnt,
+                Iterator<UpstreamEntry<K, V>> it = cursor.iterator();
+                if (!chainCopy.isEmpty()) {
+                    Stream<UpstreamEntry<K, V>> transformedStream = chainCopy.transform(Utils.asStream(it, cnt));
+                    it = transformedStream.iterator();
+                }
+                Iterator<UpstreamEntry<K, V>> iter = new IteratorWithConcurrentModificationChecker<>(
+                    it,
+                    cnt,
                     "Cache expected to be not modified during dataset data building [partition=" + part + ']');
 
                 ctx = ctxBuilder.build(iter, cnt);
@@ -245,6 +279,7 @@ public class ComputeUtils {
      * @param ignite Ignite instance.
      * @param upstreamCacheName Name of an {@code upstream} cache.
      * @param filter Filter for {@code upstream} data.
+     * @param transformersChain Transformers of upstream data.
      * @param datasetCacheName Name of a partition {@code context} cache.
      * @param ctxBuilder Partition {@code context} builder.
      * @param retries Number of retries for the case when one of partitions not found on the node.
@@ -252,10 +287,15 @@ public class ComputeUtils {
      * @param <V> Type of a value in {@code upstream} data.
      * @param <C> Type of a partition {@code context}.
      */
-    public static <K, V, C extends Serializable> void initContext(Ignite ignite, String upstreamCacheName,
-        IgniteBiPredicate<K, V> filter, String datasetCacheName, PartitionContextBuilder<K, V, C> ctxBuilder,
+    public static <K, V, C extends Serializable> void initContext(
+        Ignite ignite,
+        String upstreamCacheName,
+        IgniteBiPredicate<K, V> filter,
+        UpstreamTransformerChain<K, V> transformersChain,
+        String datasetCacheName,
+        PartitionContextBuilder<K, V, C> ctxBuilder,
         int retries) {
-        initContext(ignite, upstreamCacheName, filter, datasetCacheName, ctxBuilder, retries, 0);
+        initContext(ignite, upstreamCacheName, filter, transformersChain, datasetCacheName, ctxBuilder, retries, 0);
     }
 
     /**
@@ -288,16 +328,25 @@ public class ComputeUtils {
     /**
      * Computes number of entries selected from the cache by the query.
      *
-     * @param cache Ignite cache with upstream data.
-     * @param qry Cache query.
      * @param <K> Type of a key in {@code upstream} data.
      * @param <V> Type of a value in {@code upstream} data.
+     * @param cache Ignite cache with upstream data.
+     * @param qry Cache query.
+     * @param transformersChain Transformers of stream of upstream data.
      * @return Number of entries supplied by the iterator.
      */
-    private static  <K, V> long computeCount(IgniteCache<K, V> cache, ScanQuery<K, V> qry) {
+    private static <K, V> long computeCount(
+        IgniteCache<K, V> cache,
+        ScanQuery<K, V> qry,
+        UpstreamTransformerChain<K, V> transformersChain) {
         try (QueryCursor<UpstreamEntry<K, V>> cursor = cache.query(qry,
             e -> new UpstreamEntry<>(e.getKey(), e.getValue()))) {
-            return computeCount(cursor.iterator());
+
+            // 'If' statement below is just for optimization, to avoid unnecessary iterator -> stream -> iterator
+            // operations.
+            return transformersChain.isEmpty() ?
+                computeCount(cursor.iterator()) :
+                computeCount(transformersChain.transform(Utils.asStream(cursor.iterator())).iterator());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java
index e312b20..975beda 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java
@@ -25,7 +25,7 @@ import org.apache.ignite.ml.math.functions.IgniteBinaryOperator;
 import org.apache.ignite.ml.math.functions.IgniteTriFunction;
 
 /**
- * An implementation of dataset based on local data structures such as {@code Map} and {@code List} and doesn't requires
+ * An implementation of dataset based on local data structures such as {@code Map} and {@code List} and doesn't require
  * Ignite environment. Introduces for testing purposes mostly, but can be used for simple local computations as well.
  *
  * @param <C> Type of a partition {@code context}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java
index 6e0df2f..ce909ff 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java
@@ -19,7 +19,6 @@ package org.apache.ignite.ml.dataset.impl.local;
 
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -28,7 +27,9 @@ import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.dataset.PartitionContextBuilder;
 import org.apache.ignite.ml.dataset.PartitionDataBuilder;
 import org.apache.ignite.ml.dataset.UpstreamEntry;
+import org.apache.ignite.ml.dataset.UpstreamTransformerChain;
 import org.apache.ignite.ml.math.functions.IgniteFunction;
+import org.apache.ignite.ml.util.Utils;
 
 /**
  * A dataset builder that makes {@link LocalDataset}. Encapsulate logic of building local dataset such as allocation
@@ -47,6 +48,9 @@ public class LocalDatasetBuilder<K, V> implements DatasetBuilder<K, V> {
     /** Filter for {@code upstream} data. */
     private final IgniteBiPredicate<K, V> filter;
 
+    /** Upstream transformers. */
+    private final UpstreamTransformerChain<K, V> upstreamTransformers;
+
     /**
      * Constructs a new instance of local dataset builder that makes {@link LocalDataset} with default predicate that
      * passes all upstream entries to dataset.
@@ -69,6 +73,7 @@ public class LocalDatasetBuilder<K, V> implements DatasetBuilder<K, V> {
         this.upstreamMap = upstreamMap;
         this.filter = filter;
         this.partitions = partitions;
+        this.upstreamTransformers = UpstreamTransformerChain.empty();
     }
 
     /** {@inheritDoc} */
@@ -77,28 +82,55 @@ public class LocalDatasetBuilder<K, V> implements DatasetBuilder<K, V> {
         List<C> ctxList = new ArrayList<>();
         List<D> dataList = new ArrayList<>();
 
-        Map<K, V> filteredMap = new HashMap<>();
-        upstreamMap.forEach((key, val) -> {
-            if (filter.apply(key, val))
-                filteredMap.put(key, val);
-        });
+        List<UpstreamEntry<K, V>> entriesList = new ArrayList<>();
+
+        upstreamMap
+            .entrySet()
+            .stream()
+            .filter(en -> filter.apply(en.getKey(), en.getValue()))
+            .map(en -> new UpstreamEntry<>(en.getKey(), en.getValue()))
+            .forEach(entriesList::add);
 
-        int partSize = Math.max(1, filteredMap.size() / partitions);
+        int partSize = Math.max(1, entriesList.size() / partitions);
 
-        Iterator<K> firstKeysIter = filteredMap.keySet().iterator();
-        Iterator<K> secondKeysIter = filteredMap.keySet().iterator();
+        Iterator<UpstreamEntry<K, V>> firstKeysIter = entriesList.iterator();
+        Iterator<UpstreamEntry<K, V>> secondKeysIter = entriesList.iterator();
+        Iterator<UpstreamEntry<K, V>> thirdKeysIter = entriesList.iterator();
 
         int ptr = 0;
-        for (int part = 0; part < partitions; part++) {
-            int cnt = part == partitions - 1 ? filteredMap.size() - ptr : Math.min(partSize, filteredMap.size() - ptr);
 
-            C ctx = cnt > 0 ? partCtxBuilder.build(
-                new IteratorWindow<>(firstKeysIter, k -> new UpstreamEntry<>(k, filteredMap.get(k)), cnt),
-                cnt
-            ) : null;
+        for (int part = 0; part < partitions; part++) {
+            int cnt = part == partitions - 1 ? entriesList.size() - ptr : Math.min(partSize, entriesList.size() - ptr);
+
+            int p = part;
+            upstreamTransformers.modifySeed(s -> s + p);
+
+            if (!upstreamTransformers.isEmpty()) {
+                cnt = (int)upstreamTransformers.transform(
+                    Utils.asStream(new IteratorWindow<>(thirdKeysIter, k -> k, cnt))).count();
+            }
+
+            Iterator<UpstreamEntry<K, V>> iter;
+            if (upstreamTransformers.isEmpty()) {
+                iter = new IteratorWindow<>(firstKeysIter, k -> k, cnt);
+            }
+            else {
+                iter = upstreamTransformers.transform(
+                    Utils.asStream(new IteratorWindow<>(firstKeysIter, k -> k, cnt))).iterator();
+            }
+            C ctx = cnt > 0 ? partCtxBuilder.build(iter, cnt) : null;
+
+            Iterator<UpstreamEntry<K, V>> iter1;
+            if (upstreamTransformers.isEmpty()) {
+                iter1 = upstreamTransformers.transform(
+                    Utils.asStream(new IteratorWindow<>(secondKeysIter, k -> k, cnt))).iterator();
+            }
+            else {
+                iter1 = new IteratorWindow<>(secondKeysIter, k -> k, cnt);
+            }
 
             D data = cnt > 0 ? partDataBuilder.build(
-                new IteratorWindow<>(secondKeysIter, k -> new UpstreamEntry<>(k, filteredMap.get(k)), cnt),
+                iter1,
                 cnt,
                 ctx
             ) : null;
@@ -113,6 +145,13 @@ public class LocalDatasetBuilder<K, V> implements DatasetBuilder<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public UpstreamTransformerChain<K, V> upstreamTransformersChain() {
+        return upstreamTransformers;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
     @Override public DatasetBuilder<K, V> withFilter(IgniteBiPredicate<K, V> filterToAdd) {
         return new LocalDatasetBuilder<>(upstreamMap,
             (e1, e2) -> filter.apply(e1, e2) && filterToAdd.apply(e1, e2), partitions);
@@ -126,16 +165,24 @@ public class LocalDatasetBuilder<K, V> implements DatasetBuilder<K, V> {
      * @param <T> Target type of entries.
      */
     private static class IteratorWindow<K, T> implements Iterator<T> {
-        /** Delegate iterator. */
+        /**
+         * Delegate iterator.
+         */
         private final Iterator<K> delegate;
 
-        /** Transformer that transforms entries from one type to another. */
+        /**
+         * Transformer that transforms entries from one type to another.
+         */
         private final IgniteFunction<K, T> map;
 
-        /** Count of entries to produce. */
+        /**
+         * Count of entries to produce.
+         */
         private final int cnt;
 
-        /** Number of already produced entries. */
+        /**
+         * Number of already produced entries.
+         */
         private int ptr;
 
         /**
@@ -151,12 +198,16 @@ public class LocalDatasetBuilder<K, V> implements DatasetBuilder<K, V> {
             this.cnt = cnt;
         }
 
-        /** {@inheritDoc} */
+        /**
+         * {@inheritDoc}
+         */
         @Override public boolean hasNext() {
             return delegate.hasNext() && ptr < cnt;
         }
 
-        /** {@inheritDoc} */
+        /**
+         * {@inheritDoc}
+         */
         @Override public T next() {
             ++ptr;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/environment/LearningEnvironmentBuilder.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/environment/LearningEnvironmentBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/environment/LearningEnvironmentBuilder.java
index 91e832d..98f584f 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/environment/LearningEnvironmentBuilder.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/environment/LearningEnvironmentBuilder.java
@@ -35,7 +35,7 @@ public class LearningEnvironmentBuilder {
     /**
      * Creates an instance of LearningEnvironmentBuilder.
      */
-    LearningEnvironmentBuilder() {
+    public LearningEnvironmentBuilder() {
         parallelismStgy = NoParallelismStrategy.INSTANCE;
         loggingFactory = NoOpLogger.factory();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java
index 74a296d..47fa59d 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java
@@ -74,16 +74,15 @@ public class LogisticRegressionSGDTrainer<P extends Serializable> extends Single
         IgniteBiFunction<K, V, Double> lbExtractor) {
 
         IgniteFunction<Dataset<EmptyContext, SimpleLabeledDatasetData>, MLPArchitecture> archSupplier = dataset -> {
-            int cols = dataset.compute(data -> {
+            Integer cols = dataset.compute(data -> {
                 if (data.getFeatures() == null)
                     return null;
                 return data.getFeatures().length / data.getRows();
             }, (a, b) -> {
+                // If both are null then zero will be propagated, no good.
                 if (a == null)
-                    return b == null ? 0 : b;
-                if (b == null)
-                    return a;
-                return b;
+                    return b;
+                return a;
             });
 
             MLPArchitecture architecture = new MLPArchitecture(cols);

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java
index 5c3913e..f321744 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java
@@ -310,4 +310,5 @@ public abstract class DatasetTrainer<M extends Model, L> {
             super("Cannot train model on empty dataset");
         }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/trainers/TrainerTransformers.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/TrainerTransformers.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/TrainerTransformers.java
new file mode 100644
index 0000000..4f11327
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/TrainerTransformers.java
@@ -0,0 +1,376 @@
+/*
+ * 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.ml.trainers;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.ml.Model;
+import org.apache.ignite.ml.composition.ModelsComposition;
+import org.apache.ignite.ml.composition.predictionsaggregator.PredictionsAggregator;
+import org.apache.ignite.ml.dataset.Dataset;
+import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.dataset.PartitionContextBuilder;
+import org.apache.ignite.ml.dataset.PartitionDataBuilder;
+import org.apache.ignite.ml.dataset.UpstreamTransformerChain;
+import org.apache.ignite.ml.environment.LearningEnvironment;
+import org.apache.ignite.ml.environment.logging.MLLogger;
+import org.apache.ignite.ml.environment.parallelism.Promise;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+import org.apache.ignite.ml.math.functions.IgniteFunction;
+import org.apache.ignite.ml.math.functions.IgniteSupplier;
+import org.apache.ignite.ml.math.functions.IgniteTriFunction;
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+import org.apache.ignite.ml.math.primitives.vector.VectorUtils;
+import org.apache.ignite.ml.trainers.transformers.BaggingUpstreamTransformer;
+import org.apache.ignite.ml.util.Utils;
+
+/**
+ * Class containing various trainer transformers.
+ */
+public class TrainerTransformers {
+    /**
+     * Add bagging logic to a given trainer.
+     *
+     * @param ensembleSize Size of ensemble.
+     * @param subsampleRatio Subsample ratio to whole dataset.
+     * @param aggregator Aggregator.
+     * @param <M> Type of one model in ensemble.
+     * @param <L> Type of labels.
+     * @return Bagged trainer.
+     */
+    public static <M extends Model<Vector, Double>, L> DatasetTrainer<ModelsComposition, L> makeBagged(
+        DatasetTrainer<M, L> trainer,
+        int ensembleSize,
+        double subsampleRatio,
+        PredictionsAggregator aggregator) {
+        return makeBagged(trainer, ensembleSize, subsampleRatio, -1, -1, aggregator, new Random().nextLong());
+    }
+
+    /**
+     * Add bagging logic to a given trainer.
+     *
+     * @param ensembleSize Size of ensemble.
+     * @param subsampleRatio Subsample ratio to whole dataset.
+     * @param aggregator Aggregator.
+     * @param featureVectorSize Feature vector dimensionality.
+     * @param featuresSubspaceDim Feature subspace dimensionality.
+     * @param transformationSeed Transformations seed.
+     * @param <M> Type of one model in ensemble.
+     * @param <L> Type of labels.
+     * @return Bagged trainer.
+     */
+    // TODO: IGNITE-10296: Inject capabilities of seeding through learning environment (remove).
+    public static <M extends Model<Vector, Double>, L> DatasetTrainer<ModelsComposition, L> makeBagged(
+        DatasetTrainer<M, L> trainer,
+        int ensembleSize,
+        double subsampleRatio,
+        int featureVectorSize,
+        int featuresSubspaceDim,
+        PredictionsAggregator aggregator,
+        Long transformationSeed) {
+        return new DatasetTrainer<ModelsComposition, L>() {
+            /** {@inheritDoc} */
+            @Override public <K, V> ModelsComposition fit(
+                DatasetBuilder<K, V> datasetBuilder,
+                IgniteBiFunction<K, V, Vector> featureExtractor,
+                IgniteBiFunction<K, V, L> lbExtractor) {
+                datasetBuilder.upstreamTransformersChain().setSeed(
+                    transformationSeed == null
+                        ? new Random().nextLong()
+                        : transformationSeed);
+
+                return runOnEnsemble(
+                    (db, i, fe) -> (() -> trainer.fit(db, fe, lbExtractor)),
+                    datasetBuilder,
+                    ensembleSize,
+                    subsampleRatio,
+                    featureVectorSize,
+                    featuresSubspaceDim,
+                    featureExtractor,
+                    aggregator,
+                    environment);
+            }
+
+            /** {@inheritDoc} */
+            @Override protected boolean checkState(ModelsComposition mdl) {
+                return mdl.getModels().stream().allMatch(m -> trainer.checkState((M)m));
+            }
+
+            /** {@inheritDoc} */
+            @Override protected <K, V> ModelsComposition updateModel(
+                ModelsComposition mdl,
+                DatasetBuilder<K, V> datasetBuilder,
+                IgniteBiFunction<K, V, Vector> featureExtractor,
+                IgniteBiFunction<K, V, L> lbExtractor) {
+                return runOnEnsemble(
+                    (db, i, fe) -> (() -> trainer.updateModel(
+                        ((ModelWithMapping<Vector, Double, M>)mdl.getModels().get(i)).model(),
+                        db,
+                        fe,
+                        lbExtractor)),
+                    datasetBuilder,
+                    ensembleSize,
+                    subsampleRatio,
+                    featureVectorSize,
+                    featuresSubspaceDim,
+                    featureExtractor,
+                    aggregator,
+                    environment);
+            }
+        };
+    }
+
+    /**
+     * This method accepts function which for given dataset builder and index of model in ensemble generates
+     * task of training this model.
+     *
+     * @param trainingTaskGenerator Training test generator.
+     * @param datasetBuilder Dataset builder.
+     * @param ensembleSize Size of ensemble.
+     * @param subsampleRatio Ratio (subsample size) / (initial dataset size).
+     * @param featuresVectorSize Dimensionality of feature vector.
+     * @param featureSubspaceDim Dimensionality of feature subspace.
+     * @param aggregator Aggregator of models.
+     * @param environment Environment.
+     * @param <K> Type of keys in dataset builder.
+     * @param <V> Type of values in dataset builder.
+     * @param <M> Type of model.
+     * @return Composition of models trained on bagged dataset.
+     */
+    private static <K, V, M extends Model<Vector, Double>> ModelsComposition runOnEnsemble(
+        IgniteTriFunction<DatasetBuilder<K, V>, Integer, IgniteBiFunction<K, V, Vector>, IgniteSupplier<M>> trainingTaskGenerator,
+        DatasetBuilder<K, V> datasetBuilder,
+        int ensembleSize,
+        double subsampleRatio,
+        int featuresVectorSize,
+        int featureSubspaceDim,
+        IgniteBiFunction<K, V, Vector> extractor,
+        PredictionsAggregator aggregator,
+        LearningEnvironment environment) {
+
+        MLLogger log = environment.logger(datasetBuilder.getClass());
+        log.log(MLLogger.VerboseLevel.LOW, "Start learning.");
+
+        List<int[]> mappings = null;
+        if (featuresVectorSize > 0) {
+            mappings = IntStream.range(0, ensembleSize).mapToObj(
+                modelIdx -> getMapping(
+                    featuresVectorSize,
+                    featureSubspaceDim,
+                    datasetBuilder.upstreamTransformersChain().seed() + modelIdx))
+                .collect(Collectors.toList());
+        }
+
+        Long startTs = System.currentTimeMillis();
+
+        datasetBuilder
+            .upstreamTransformersChain()
+            .addUpstreamTransformer(new BaggingUpstreamTransformer<>(subsampleRatio));
+
+        List<IgniteSupplier<M>> tasks = new ArrayList<>();
+        List<IgniteBiFunction<K, V, Vector>> extractors = new ArrayList<>();
+        if (mappings != null) {
+            for (int[] mapping : mappings) {
+                extractors.add(wrapExtractor(extractor, mapping));
+            }
+        }
+
+        for (int i = 0; i < ensembleSize; i++) {
+            UpstreamTransformerChain<K, V> newChain = Utils.copy(datasetBuilder.upstreamTransformersChain());
+            DatasetBuilder<K, V> newBuilder = withNewChain(datasetBuilder, newChain);
+            int j = i;
+            newChain.modifySeed(s -> s * s + j);
+            tasks.add(
+                trainingTaskGenerator.apply(newBuilder, i, mappings != null ? extractors.get(i) : extractor));
+        }
+
+        List<ModelWithMapping<Vector, Double, M>> models = environment.parallelismStrategy().submit(tasks)
+            .stream()
+            .map(Promise::unsafeGet)
+            .map(ModelWithMapping<Vector, Double, M>::new)
+            .collect(Collectors.toList());
+
+        // If we need to do projection, do it.
+        if (mappings != null) {
+            for (int i = 0; i < models.size(); i++) {
+                models.get(i).setMapping(getProjector(mappings.get(i)));
+            }
+        }
+
+        double learningTime = (double)(System.currentTimeMillis() - startTs) / 1000.0;
+        log.log(MLLogger.VerboseLevel.LOW, "The training time was %.2fs.", learningTime);
+        log.log(MLLogger.VerboseLevel.LOW, "Learning finished.");
+
+        return new ModelsComposition(models, aggregator);
+    }
+
+    /**
+     * Get mapping R^featuresVectorSize -> R^maximumFeaturesCntPerMdl.
+     *
+     * @param featuresVectorSize Features vector size (Dimension of initial space).
+     * @param maximumFeaturesCntPerMdl Dimension of target space.
+     * @param seed Seed.
+     * @return Mapping R^featuresVectorSize -> R^maximumFeaturesCntPerMdl.
+     */
+    public static int[] getMapping(int featuresVectorSize, int maximumFeaturesCntPerMdl, long seed) {
+        return Utils.selectKDistinct(featuresVectorSize, maximumFeaturesCntPerMdl, new Random(seed));
+    }
+
+    /**
+     * Get projector from index mapping.
+     *
+     * @param mapping Index mapping.
+     * @return Projector.
+     */
+    public static IgniteFunction<Vector, Vector> getProjector(int[] mapping) {
+        return v -> {
+            Vector res = VectorUtils.zeroes(mapping.length);
+            for (int i = 0; i < mapping.length; i++) {
+                res.set(i, v.get(mapping[i]));
+            }
+            return res;
+        };
+    }
+
+    /**
+     * Creates feature extractor which is a composition of given feature extractor and projection given by
+     * coordinate indexes mapping.
+     *
+     * @param featureExtractor Initial feature extractor.
+     * @param featureMapping Coordinate indexes mapping.
+     * @param <K> Type of keys.
+     * @param <V> Type of values.
+     * @return Composition of given feature extractor and projection given by coordinate indexes mapping.
+     */
+    private static <K, V> IgniteBiFunction<K, V, Vector> wrapExtractor(IgniteBiFunction<K, V, Vector> featureExtractor,
+        int[] featureMapping) {
+        return featureExtractor.andThen((IgniteFunction<Vector, Vector>)featureValues -> {
+            double[] newFeaturesValues = new double[featureMapping.length];
+            for (int i = 0; i < featureMapping.length; i++) {
+                newFeaturesValues[i] = featureValues.get(featureMapping[i]);
+            }
+            return VectorUtils.of(newFeaturesValues);
+        });
+    }
+
+    /**
+     * Model with mapping from X to X.
+     *
+     * @param <X> Input space.
+     * @param <Y> Output space.
+     * @param <M> Model.
+     */
+    private static class ModelWithMapping<X, Y, M extends Model<X, Y>> implements Model<X, Y> {
+        /** Model. */
+        private final M model;
+
+        /** Mapping. */
+        private IgniteFunction<X, X> mapping;
+
+        /**
+         * Create instance of this class from a given model.
+         * Identity mapping will be used as a mapping.
+         *
+         * @param model Model.
+         */
+        public ModelWithMapping(M model) {
+            this(model, x -> x);
+        }
+
+        /**
+         * Create instance of this class from given model and mapping.
+         *
+         * @param model Model.
+         * @param mapping Mapping.
+         */
+        public ModelWithMapping(M model, IgniteFunction<X, X> mapping) {
+            this.model = model;
+            this.mapping = mapping;
+        }
+
+        /**
+         * Sets mapping.
+         *
+         * @param mapping Mapping.
+         */
+        public void setMapping(IgniteFunction<X, X> mapping) {
+            this.mapping = mapping;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Y apply(X x) {
+            return model.apply(mapping.apply(x));
+        }
+
+        /**
+         * Gets model.
+         *
+         * @return Model.
+         */
+        public M model() {
+            return model;
+        }
+
+        /**
+         * Gets mapping.
+         *
+         * @return Mapping.
+         */
+        public IgniteFunction<X, X> mapping() {
+            return mapping;
+        }
+    }
+
+    /**
+     * Creates new dataset builder which is delegate of a given dataset builder in everything except
+     * new transformations chain.
+     *
+     * @param builder Initial builder.
+     * @param chain New chain.
+     * @param <K> Type of keys.
+     * @param <V> Type of values.
+     * @return new dataset builder which is delegate of a given dataset builder in everything except
+     * new transformations chain.
+     */
+    private static <K, V> DatasetBuilder<K, V> withNewChain(
+        DatasetBuilder<K, V> builder,
+        UpstreamTransformerChain<K, V> chain) {
+        return new DatasetBuilder<K, V>() {
+            /** {@inheritDoc} */
+            @Override public <C extends Serializable, D extends AutoCloseable> Dataset<C, D> build(
+                PartitionContextBuilder<K, V, C> partCtxBuilder, PartitionDataBuilder<K, V, C, D> partDataBuilder) {
+                return builder.build(partCtxBuilder, partDataBuilder);
+            }
+
+            /** {@inheritDoc} */
+            @Override public UpstreamTransformerChain<K, V> upstreamTransformersChain() {
+                return chain;
+            }
+
+            /** {@inheritDoc} */
+            @Override public DatasetBuilder<K, V> withFilter(IgniteBiPredicate<K, V> filterToAdd) {
+                return builder.withFilter(filterToAdd);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/trainers/transformers/BaggingUpstreamTransformer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/transformers/BaggingUpstreamTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/transformers/BaggingUpstreamTransformer.java
new file mode 100644
index 0000000..f935ebd
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/transformers/BaggingUpstreamTransformer.java
@@ -0,0 +1,58 @@
+/*
+ * 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.ml.trainers.transformers;
+
+import java.util.Random;
+import java.util.stream.Stream;
+import org.apache.commons.math3.distribution.PoissonDistribution;
+import org.apache.commons.math3.random.Well19937c;
+import org.apache.ignite.ml.dataset.UpstreamEntry;
+import org.apache.ignite.ml.dataset.UpstreamTransformer;
+
+/**
+ * This class encapsulates the logic needed to do bagging (bootstrap aggregating) by features.
+ * The action of this class on a given upstream is to replicate each entry in accordance to
+ * Poisson distribution.
+ *
+ * @param <K> Type of upstream keys.
+ * @param <V> Type of upstream values.
+ */
+public class BaggingUpstreamTransformer<K, V> implements UpstreamTransformer<K, V> {
+    /** Ratio of subsample to entire upstream size */
+    private double subsampleRatio;
+
+    /**
+     * Construct instance of this transformer with a given subsample ratio.
+     *
+     * @param subsampleRatio Subsample ratio.
+     */
+    public BaggingUpstreamTransformer(double subsampleRatio) {
+        this.subsampleRatio = subsampleRatio;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Stream<UpstreamEntry<K, V>> transform(Random rnd, Stream<UpstreamEntry<K, V>> upstream) {
+        PoissonDistribution poisson = new PoissonDistribution(
+            new Well19937c(rnd.nextLong()),
+            subsampleRatio,
+            PoissonDistribution.DEFAULT_EPSILON,
+            PoissonDistribution.DEFAULT_MAX_ITERATIONS);
+
+        return upstream.sequential().flatMap(en -> Stream.generate(() -> en).limit(poisson.sample()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/trainers/transformers/package-info.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/transformers/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/transformers/package-info.java
new file mode 100644
index 0000000..b698ead
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/transformers/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Various upstream transformers.
+ */
+package org.apache.ignite.ml.trainers.transformers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramsComputer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramsComputer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramsComputer.java
index 8320461..d202441 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramsComputer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramsComputer.java
@@ -45,7 +45,7 @@ public abstract class ImpurityHistogramsComputer<S extends ImpurityComputer<Boot
     private static final long serialVersionUID = -4984067145908187508L;
 
     /**
-     * Computes histograms for each features.
+     * Computes histograms for each feature.
      *
      * @param roots Random forest roots.
      * @param histMeta Histograms meta.

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/main/java/org/apache/ignite/ml/util/Utils.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/util/Utils.java b/modules/ml/src/main/java/org/apache/ignite/ml/util/Utils.java
index ed0ebd3..63a9f3c 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/util/Utils.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/util/Utils.java
@@ -22,7 +22,12 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
+import java.util.Iterator;
 import java.util.Random;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
 import org.apache.ignite.IgniteException;
 
 /**
@@ -98,4 +103,31 @@ public class Utils {
     public static int[] selectKDistinct(int n, int k) {
         return selectKDistinct(n, k, new Random());
     }
+
+    /**
+     * Convert given iterator to a stream with known count of entries.
+     *
+     * @param iter Iterator.
+     * @param cnt Count.
+     * @param <T> Type of entries.
+     * @return Stream constructed from iterator.
+     */
+    public static <T> Stream<T> asStream(Iterator<T> iter, long cnt) {
+        return StreamSupport.stream(
+                Spliterators.spliterator(iter, cnt, Spliterator.ORDERED),
+                false);
+    }
+
+    /**
+     * Convert given iterator to a stream.
+     *
+     * @param iter Iterator.
+     * @param <T> Iterator content type.
+     * @return Stream constructed from iterator.
+     */
+    public static <T> Stream<T> asStream(Iterator<T> iter) {
+        return StreamSupport.stream(
+                Spliterators.spliteratorUnknownSize(iter, Spliterator.ORDERED),
+                false);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
index 481e1fa..e26b5b8 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
@@ -32,6 +32,7 @@ import org.apache.ignite.ml.regressions.RegressionsTestSuite;
 import org.apache.ignite.ml.selection.SelectionTestSuite;
 import org.apache.ignite.ml.structures.StructuresTestSuite;
 import org.apache.ignite.ml.svm.SVMTestSuite;
+import org.apache.ignite.ml.trainers.BaggingTest;
 import org.apache.ignite.ml.tree.DecisionTreeTestSuite;
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
@@ -57,7 +58,8 @@ import org.junit.runners.Suite;
     CompositionTestSuite.class,
     EnvironmentTestSuite.class,
     StructuresTestSuite.class,
-    CommonTestSuite.class
+    CommonTestSuite.class,
+    BaggingTest.class
 })
 public class IgniteMLTestSuite {
     // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java
index 952fc43..cee8f4f 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.ml.dataset.UpstreamEntry;
+import org.apache.ignite.ml.dataset.UpstreamTransformerChain;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -178,6 +179,7 @@ public class ComputeUtilsTest extends GridCommonAbstractTest {
                     ignite,
                     upstreamCacheName,
                     (k, v) -> true,
+                    UpstreamTransformerChain.empty(),
                     datasetCacheName,
                     datasetId,
                     0,
@@ -227,6 +229,7 @@ public class ComputeUtilsTest extends GridCommonAbstractTest {
             ignite,
             upstreamCacheName,
             (k, v) -> true,
+            UpstreamTransformerChain.empty(),
             datasetCacheName,
             (upstream, upstreamSize) -> {
 


[47/50] [abbrv] ignite git commit: ignite-10044

Posted by sb...@apache.org.
ignite-10044


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

Branch: refs/heads/ignite-10044
Commit: 93b9967c410fa25c86024f1fe56f74443eaca27d
Parents: c634be2
Author: sboikov <sb...@apache.org>
Authored: Tue Nov 20 17:36:18 2018 +0300
Committer: sboikov <sb...@apache.org>
Committed: Tue Nov 20 17:36:18 2018 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        | 39 +++++++++---
 .../topology/GridClientPartitionTopology.java   | 66 ++++++++++++++++++--
 2 files changed, 91 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/93b9967c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 72f01f0..3ce5cf1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -1806,7 +1806,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             Set<String> caches = exchActions.cachesToResetLostPartitions();
 
             if (!F.isEmpty(caches))
-                resetLostPartitions(caches);
+                resetLostPartitions(caches, false);
         }
 
         if (cctx.kernalContext().clientNode() || (dynamicCacheStartExchange() && exchangeLocE != null)) {
@@ -2078,7 +2078,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             }
 
             if (serverNodeDiscoveryEvent() || localJoinExchange())
-                detectLostPartitions(res);
+                detectLostPartitions(res, false);
 
             Map<Integer, CacheGroupValidation> m = U.newHashMap(cctx.cache().cacheGroups().size());
 
@@ -2997,7 +2997,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      *
      * @param resTopVer Result topology version.
      */
-    private void detectLostPartitions(AffinityTopologyVersion resTopVer) {
+    private void detectLostPartitions(AffinityTopologyVersion resTopVer, boolean crd) {
         boolean detected = false;
 
         long time = System.currentTimeMillis();
@@ -3016,6 +3016,11 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     detected |= detectedOnGrp;
                 }
             }
+
+            if (crd) {
+                for (GridClientPartitionTopology top : cctx.exchange().clientTopologies())
+                    top.detectLostPartitions(resTopVer, null);
+            }
         }
 
         if (detected) {
@@ -3033,24 +3038,38 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     /**
      * @param cacheNames Cache names.
      */
-    private void resetLostPartitions(Collection<String> cacheNames) {
+    private void resetLostPartitions(Collection<String> cacheNames, boolean crd) {
         assert !exchCtx.mergeExchanges();
 
         synchronized (cctx.exchange().interruptLock()) {
             if (Thread.currentThread().isInterrupted())
                 return;
 
-            for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                if (grp.isLocal())
-                    continue;
+            for (String cacheName : cacheNames) {
+                boolean found = false;
+
+                for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
+                    if (grp.isLocal())
+                        continue;
 
-                for (String cacheName : cacheNames) {
                     if (grp.hasCache(cacheName)) {
                         grp.topology().resetLostPartitions(initialVersion());
 
+                        found = true;
+
                         break;
                     }
                 }
+
+                if (crd && !found) {
+                    DynamicCacheDescriptor cacheDesc = cctx.affinity().caches().get(CU.cacheId(cacheName));
+
+                    if (cacheDesc != null) {
+                        GridDhtPartitionTopology top = cctx.exchange().clientTopology(cacheDesc.groupId(), context().events().discoveryCache());
+
+                        top.resetLostPartitions(initialVersion());
+                    }
+                }
             }
         }
     }
@@ -3276,7 +3295,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         Set<String> caches = exchActions.cachesToResetLostPartitions();
 
                         if (!F.isEmpty(caches))
-                            resetLostPartitions(caches);
+                            resetLostPartitions(caches, true);
                     }
                 }
                 else if (discoveryCustomMessage instanceof SnapshotDiscoveryMessage
@@ -3288,7 +3307,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     assignPartitionsStates();
 
                 if (exchCtx.events().hasServerLeft())
-                    detectLostPartitions(resTopVer);
+                    detectLostPartitions(resTopVer, true);
             }
 
             // Recalculate new affinity based on partitions availability.

http://git-wip-us.apache.org/repos/asf/ignite/blob/93b9967c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
index 01db508..1a99e59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
@@ -26,13 +26,16 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
@@ -59,7 +62,9 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.EVICTED;
+import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.LOST;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING;
 
@@ -96,7 +101,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     private AffinityTopologyVersion lastExchangeVer;
 
     /** */
-    private AffinityTopologyVersion topVer = AffinityTopologyVersion.NONE;
+    private AffinityTopologyVersion topVer;
 
     /** */
     private volatile boolean stopping;
@@ -125,6 +130,9 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     /** */
     private volatile Map<Integer, Long> globalPartSizes;
 
+    /** */
+    private TreeSet<Integer> lostParts;
+
     /**
      * @param cctx Context.
      * @param discoCache Discovery data cache.
@@ -998,14 +1006,53 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @Override public boolean detectLostPartitions(AffinityTopologyVersion affVer, DiscoveryEvent discoEvt) {
-        assert false : "detectLostPartitions should never be called on client topology";
+        lock.writeLock().lock();
 
-        return false;
+
+        boolean changed = false;
+
+        try {
+            for (int part = 0; part < parts; part++) {
+                boolean lost = F.contains(lostParts, part);
+
+                if (!lost) {
+                    boolean hasOwner = false;
+
+                    for (GridDhtPartitionMap partMap : node2part.values()) {
+                        if (partMap.get(part) == OWNING) {
+                            hasOwner = true;
+                            break;
+                        }
+                    }
+
+                    if (!hasOwner) {
+                        if (lostParts == null)
+                            lostParts = new TreeSet<>();
+
+                        changed = true;
+
+                        lostParts.add(part);
+                    }
+                }
+            }
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+
+        return changed;
     }
 
     /** {@inheritDoc} */
     @Override public void resetLostPartitions(AffinityTopologyVersion affVer) {
-        assert false : "resetLostPartitions should never be called on client topology";
+        lock.writeLock().lock();
+
+        try {
+            lostParts = null;
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
     }
 
     /** {@inheritDoc} */
@@ -1208,6 +1255,17 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
                 }
             }
 
+            if (lostParts != null) {
+                for (Map.Entry<UUID, GridDhtPartitionMap> e : node2part.entrySet()) {
+                    for (Integer part : lostParts) {
+                        GridDhtPartitionState state = e.getValue().get(part);
+
+                        if (state != null && state.active())
+                            e.getValue().put(part, LOST);
+                    }
+                }
+            }
+
             for (Map.Entry<Integer, Set<UUID>> entry : ownersByUpdCounters.entrySet())
                 part2node.put(entry.getKey(), entry.getValue());
 


[35/50] [abbrv] ignite git commit: IGNITE-10159 Fixed muted IgniteCacheAbstractQuerySelfTest.testObjectQueryWithSwap - Fixes #5336.

Posted by sb...@apache.org.
IGNITE-10159 Fixed muted IgniteCacheAbstractQuerySelfTest.testObjectQueryWithSwap - Fixes #5336.

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/2c23d680
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/2c23d680
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/2c23d680

Branch: refs/heads/ignite-10044
Commit: 2c23d68071e6cd2a604c92bf5bb94c75c7da522b
Parents: 2a59d83
Author: Alexey Platonov <ap...@gmail.com>
Authored: Tue Nov 20 11:20:58 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 20 11:25:23 2018 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheAbstractQuerySelfTest.java | 37 +++++++++++---------
 .../IgniteCacheReplicatedQuerySelfTest.java     | 17 ++-------
 2 files changed, 22 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2c23d680/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index bf6d6da..1c1b879 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -266,7 +266,7 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
         super.afterTest();
 
         for(String cacheName : ignite().cacheNames())
-            ignite().cache(cacheName).removeAll();
+            ignite().cache(cacheName).destroy();
     }
 
     /** {@inheritDoc} */
@@ -621,11 +621,16 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
 
     /**
      * JUnit.
-     *
-     * @throws Exception In case of error.
      */
-    public void testSimpleCustomTableName() throws Exception {
-        final IgniteCache<Integer, Object> cache = ignite().cache(DEFAULT_CACHE_NAME);
+    public void testSimpleCustomTableName() {
+        CacheConfiguration<Integer, Object> cacheConf = new CacheConfiguration<Integer, Object>(cacheConfiguration())
+            .setName(DEFAULT_CACHE_NAME)
+            .setQueryEntities(Arrays.asList(
+                new QueryEntity(Integer.class, Type1.class),
+                new QueryEntity(Integer.class, Type2.class)
+            ));
+
+        final IgniteCache<Integer, Object> cache = ignite().getOrCreateCache(cacheConf);
 
         cache.put(10, new Type1(1, "Type1 record #1"));
         cache.put(20, new Type1(2, "Type1 record #2"));
@@ -937,13 +942,13 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
 
     /**
      * JUnit.
-     *
-     * @throws Exception In case of error.
      */
-    public void _testObjectQueryWithSwap() throws Exception {
-        fail("http://atlassian.gridgain.com/jira/browse/GG-11216");
+    public void testObjectQueryWithSwap() {
+        CacheConfiguration<Integer, ObjectValue> config = new CacheConfiguration<Integer, ObjectValue>(cacheConfiguration());
 
-        IgniteCache<Integer, ObjectValue> cache = jcache(Integer.class, ObjectValue.class);
+        config.setOnheapCacheEnabled(true);
+
+        IgniteCache<Integer, ObjectValue> cache = jcache(ignite(), config, Integer.class, ObjectValue.class);
 
         boolean partitioned = cache.getConfiguration(CacheConfiguration.class).getCacheMode() == PARTITIONED;
 
@@ -956,16 +961,14 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
             IgniteCache<Integer, ObjectValue> c = g.cache(cache.getName());
 
             for (int i = 0; i < cnt; i++) {
-                if (i % 2 == 0) {
-                    assertNotNull(c.localPeek(i, CachePeekMode.ONHEAP));
+                assertNotNull(c.localPeek(i, CachePeekMode.ONHEAP));
 
-                    c.localEvict(Collections.singleton(i)); // Swap.
+                c.localEvict(Collections.singleton(i)); // Swap.
 
-                    if (!partitioned || g.affinity(cache.getName()).mapKeyToNode(i).isLocal()) {
-                        ObjectValue peekVal = c.localPeek(i, CachePeekMode.ONHEAP);
+                if (!partitioned || g.affinity(cache.getName()).mapKeyToNode(i).isLocal()) {
+                    ObjectValue peekVal = c.localPeek(i, CachePeekMode.ONHEAP);
 
-                        assertNull("Non-null value for peek [key=" + i + ", val=" + peekVal + ']', peekVal);
-                    }
+                    assertNull("Non-null value for peek [key=" + i + ", val=" + peekVal + ']', peekVal);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/2c23d680/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
index b9af3b6..8635416 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
@@ -101,8 +101,8 @@ public class IgniteCacheReplicatedQuerySelfTest extends IgniteCacheAbstractQuery
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
 
         ignite1 = grid(0);
         ignite2 = grid(1);
@@ -113,19 +113,6 @@ public class IgniteCacheReplicatedQuerySelfTest extends IgniteCacheAbstractQuery
         cache3 = jcache(ignite3, CacheKey.class, CacheValue.class);
     }
 
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
-
-        ignite1 = null;
-        ignite2 = null;
-        ignite3 = null;
-
-        cache1 = null;
-        cache2 = null;
-        cache3 = null;
-    }
-
     /**
      * @throws Exception If failed.
      */


[46/50] [abbrv] ignite git commit: IGNITE-10043 Do not reset the list of LOST partitions when only one server node is left in the cluster - Fixes #5436

Posted by sb...@apache.org.
IGNITE-10043 Do not reset the list of LOST partitions when only one server node is left in the cluster - Fixes #5436


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

Branch: refs/heads/ignite-10044
Commit: 8c08de7beabf79dec28d2920707e7851b4c97823
Parents: b48a291
Author: Semyon Boikov <sb...@apache.org>
Authored: Tue Nov 20 17:05:01 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 20 17:05:01 2018 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        |   5 +-
 .../dht/topology/GridDhtPartitionTopology.java  |   5 +-
 .../topology/GridDhtPartitionTopologyImpl.java  | 115 ++++++++-----------
 .../IgniteCachePartitionLossPolicySelfTest.java |   6 +-
 4 files changed, 57 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8c08de7b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 2e792f1..86000cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -3013,7 +3013,10 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
             for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
                 if (!grp.isLocal()) {
-                    boolean detectedOnGrp = grp.topology().detectLostPartitions(resTopVer, events().lastEvent());
+                    // Do not trigger lost partition events on start.
+                    boolean event = !localJoinExchange() && !activateCluster();
+
+                    boolean detectedOnGrp = grp.topology().detectLostPartitions(resTopVer, event ? events().lastEvent() : null);
 
                     detected |= detectedOnGrp;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c08de7b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
index be8a789..2f5e57b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
@@ -25,6 +25,7 @@ import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
@@ -327,10 +328,10 @@ public interface GridDhtPartitionTopology {
      * This method should be called on topology coordinator after all partition messages are received.
      *
      * @param resTopVer Exchange result version.
-     * @param discoEvt Discovery event for which we detect lost partitions.
+     * @param discoEvt Discovery event for which we detect lost partitions if {@link EventType#EVT_CACHE_REBALANCE_PART_DATA_LOST} event should be fired.
      * @return {@code True} if partitions state got updated.
      */
-    public boolean detectLostPartitions(AffinityTopologyVersion resTopVer, DiscoveryEvent discoEvt);
+    public boolean detectLostPartitions(AffinityTopologyVersion resTopVer, @Nullable DiscoveryEvent discoEvt);
 
     /**
      * Resets the state of all LOST partitions to OWNING.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c08de7b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
index a127876..f64c1c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReferenceArray;
@@ -112,8 +113,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     /** Node to partition map. */
     private GridDhtPartitionFullMap node2part;
 
-    /** Partitions map for left nodes. */
-    private GridDhtPartitionFullMap leftNode2Part = new GridDhtPartitionFullMap();
+    /** */
+    private Set<Integer> lostParts;
 
     /** */
     private final Map<Integer, Set<UUID>> diffFromAffinity = new HashMap<>();
@@ -1493,13 +1494,6 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                                 log.trace("Removing left node from full map update [grp=" + grp.cacheOrGroupName() +
                                     ", nodeId=" + nodeId + ", partMap=" + partMap + ']');
 
-                            if (node2part.containsKey(nodeId)) {
-                                GridDhtPartitionMap map = partMap.get(nodeId);
-
-                                if (map != null)
-                                    leftNode2Part.put(nodeId, map);
-                            }
-
                             it.remove();
                         }
                     }
@@ -2029,7 +2023,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean detectLostPartitions(AffinityTopologyVersion resTopVer, DiscoveryEvent discoEvt) {
+    @Override public boolean detectLostPartitions(AffinityTopologyVersion resTopVer, @Nullable DiscoveryEvent discoEvt) {
         ctx.database().checkpointReadLock();
 
         try {
@@ -2039,48 +2033,55 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 if (node2part == null)
                     return false;
 
+                PartitionLossPolicy plc = grp.config().getPartitionLossPolicy();
+
+                assert plc != null;
+
                 int parts = grp.affinity().partitions();
 
-                Set<Integer> lost = new HashSet<>(parts);
+                Set<Integer> recentlyLost = null;
 
-                for (int p = 0; p < parts; p++)
-                    lost.add(p);
+                boolean changed = false;
 
-                for (GridDhtPartitionMap partMap : node2part.values()) {
-                    for (Map.Entry<Integer, GridDhtPartitionState> e : partMap.entrySet()) {
-                        if (e.getValue() == OWNING) {
-                            lost.remove(e.getKey());
+                for (int part = 0; part < parts; part++) {
+                    boolean lost = F.contains(lostParts, part);
 
-                            if (lost.isEmpty())
+                    if (!lost) {
+                        boolean hasOwner = false;
+
+                        for (GridDhtPartitionMap partMap : node2part.values()) {
+                            if (partMap.get(part) == OWNING) {
+                                hasOwner = true;
                                 break;
+                            }
                         }
-                    }
-                }
 
-                boolean changed = false;
+                        if (!hasOwner) {
+                            lost = true;
 
-                if (!F.isEmpty(lost)) {
-                    PartitionLossPolicy plc = grp.config().getPartitionLossPolicy();
+                            if (lostParts == null)
+                                lostParts = new TreeSet<>();
 
-                    assert plc != null;
+                            lostParts.add(part);
 
-                    Set<Integer> recentlyLost = new HashSet<>();
+                            if (discoEvt != null) {
+                                if (recentlyLost == null)
+                                    recentlyLost = new HashSet<>();
 
-                    for (Map.Entry<UUID, GridDhtPartitionMap> leftEntry : leftNode2Part.entrySet()) {
-                        for (Map.Entry<Integer, GridDhtPartitionState> entry : leftEntry.getValue().entrySet()) {
-                            if (entry.getValue() == OWNING)
-                                recentlyLost.add(entry.getKey());
-                        }
-                    }
+                                recentlyLost.add(part);
 
-                    if (!recentlyLost.isEmpty()) {
-                        U.warn(log, "Detected lost partitions [grp=" + grp.cacheOrGroupName()
-                            + ", parts=" + S.compact(recentlyLost)
-                            + ", plc=" + plc + "]");
+                                if (grp.eventRecordable(EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST)) {
+                                    grp.addRebalanceEvent(part,
+                                        EVT_CACHE_REBALANCE_PART_DATA_LOST,
+                                        discoEvt.eventNode(),
+                                        discoEvt.type(),
+                                        discoEvt.timestamp());
+                                }
+                            }
+                        }
                     }
 
-                    // Update partition state on all nodes.
-                    for (Integer part : lost) {
+                    if (lost) {
                         long updSeq = updateSeq.incrementAndGet();
 
                         GridDhtLocalPartition locPart = localPartition(part, resTopVer, false, true);
@@ -2106,21 +2107,17 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                                     e.getValue().put(part, LOST);
                             }
                         }
-
-                        if (recentlyLost.contains(part) && grp.eventRecordable(EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST)) {
-                            grp.addRebalanceEvent(part,
-                                EVT_CACHE_REBALANCE_PART_DATA_LOST,
-                                discoEvt.eventNode(),
-                                discoEvt.type(),
-                                discoEvt.timestamp());
-                        }
                     }
+                }
 
-                    if (plc != PartitionLossPolicy.IGNORE)
-                        grp.needsRecovery(true);
+                if (recentlyLost != null) {
+                    U.warn(log, "Detected lost partitions [grp=" + grp.cacheOrGroupName()
+                        + ", parts=" + S.compact(recentlyLost)
+                        + ", plc=" + plc + "]");
                 }
 
-                leftNode2Part.clear();
+                if (lostParts != null && plc != PartitionLossPolicy.IGNORE)
+                    grp.needsRecovery(true);
 
                 return changed;
             }
@@ -2170,6 +2167,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                 checkEvictions(updSeq, grp.affinity().readyAffinity(resTopVer));
 
+                lostParts = null;
+
                 grp.needsRecovery(false);
             }
             finally {
@@ -2189,22 +2188,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         lock.readLock().lock();
 
         try {
-            Set<Integer> res = null;
-
-            int parts = grp.affinity().partitions();
-
-            for (GridDhtPartitionMap partMap : node2part.values()) {
-                for (Map.Entry<Integer, GridDhtPartitionState> e : partMap.entrySet()) {
-                    if (e.getValue() == LOST) {
-                        if (res == null)
-                            res = new HashSet<>(parts);
-
-                        res.add(e.getKey());
-                    }
-                }
-            }
-
-            return res == null ? Collections.<Integer>emptySet() : res;
+            return lostParts == null ? Collections.<Integer>emptySet() : new HashSet<>(lostParts);
         }
         finally {
             lock.readLock().unlock();
@@ -2559,9 +2543,6 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
             GridDhtPartitionMap parts = node2part.remove(nodeId);
 
-            if (parts != null)
-                leftNode2Part.put(nodeId, parts);
-
             if (!grp.isReplicated()) {
                 if (parts != null) {
                     for (Integer p : parts.keySet()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c08de7b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
index cfe578d..226ae22 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
@@ -299,8 +299,6 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
      * @throws Exception if failed.
      */
     public void testReadWriteSafeWithBackupsAfterKillThreeNodesWithPersistence() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-10043");
-
         partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE;
 
         backups = 1;
@@ -924,12 +922,12 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
 
             for (Map<Integer, Semaphore> map : lostMap) {
                 for (Map.Entry<Integer, Semaphore> entry : map.entrySet())
-                    assertTrue("Failed to wait for partition LOST event for partition:" + entry.getKey(), entry.getValue().tryAcquire(1));
+                    assertTrue("Failed to wait for partition LOST event for partition: " + entry.getKey(), entry.getValue().tryAcquire(1));
             }
 
             for (Map<Integer, Semaphore> map : lostMap) {
                 for (Map.Entry<Integer, Semaphore> entry : map.entrySet())
-                    assertFalse("Partition LOST event raised twice for partition:" + entry.getKey(), entry.getValue().tryAcquire(1));
+                    assertFalse("Partition LOST event raised twice for partition: " + entry.getKey(), entry.getValue().tryAcquire(1));
             }
 
             return parts;


[44/50] [abbrv] ignite git commit: IGNITE-9999 Added verbose logging for node recovery - Fixes #5371

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java
index 425f9b9..93cc074 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java
@@ -21,21 +21,24 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.file.OpenOption;
+import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
 import com.google.common.collect.Lists;
+import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
@@ -45,12 +48,17 @@ import org.apache.ignite.failure.FailureHandler;
 import org.apache.ignite.failure.StopNodeFailureHandler;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.GridCacheUtils;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
-import org.apache.ignite.internal.processors.cache.persistence.wal.memtracker.PageMemoryTrackerConfiguration;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -94,7 +102,6 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
         );
 
         DataStorageConfiguration dsCfg = new DataStorageConfiguration()
-            .setAlwaysWriteFullPages(true)
             .setWalMode(WALMode.LOG_ONLY)
             .setCheckpointFrequency(1024 * 1024 * 1024) // Disable automatic checkpoints.
             .setDefaultDataRegionConfiguration(
@@ -110,7 +117,11 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
         if (ioFactory != null)
             dsCfg.setFileIOFactory(ioFactory);
 
-        cfg.setPluginConfigurations(new PageMemoryTrackerConfiguration().setEnabled(false).setCheckPagesOnCheckpoint(true));
+        TestRecordingCommunicationSpi spi = new TestRecordingCommunicationSpi();
+
+        spi.record(GridDhtPartitionDemandMessage.class);
+
+        cfg.setCommunicationSpi(spi);
 
         return cfg;
     }
@@ -130,16 +141,16 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
      * @param cacheMode Cache mode.
      * @param atomicityMode Atomicity mode.
      */
-    private CacheConfiguration<Object, Object> cacheConfiguration(String name, @Nullable String groupName, CacheMode cacheMode, CacheAtomicityMode atomicityMode) {
-        CacheConfiguration<Object, Object> cfg = new CacheConfiguration<>();
+    protected CacheConfiguration<Object, Object> cacheConfiguration(String name, @Nullable String groupName, CacheMode cacheMode, CacheAtomicityMode atomicityMode) {
+        CacheConfiguration<Object, Object> cfg = new CacheConfiguration<>(name)
+            .setGroupName(groupName)
+            .setCacheMode(cacheMode)
+            .setAtomicityMode(atomicityMode)
+            .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC)
+            .setBackups(2)
+            .setAffinity(new RendezvousAffinityFunction(false, 32));
 
-        cfg.setGroupName(groupName);
-        cfg.setName(name);
-        cfg.setCacheMode(cacheMode);
-        cfg.setAtomicityMode(atomicityMode);
-        cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cfg.setBackups(2);
-        cfg.setAffinity(new RendezvousAffinityFunction(false, 32));
+        cfg.setIndexedTypes(Integer.class, Integer.class);
 
         return cfg;
     }
@@ -172,25 +183,23 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
 
         IgniteEx node = grid(2);
 
-        AggregateCacheLoader aggCacheLoader = new AggregateCacheLoader(node);
-
-        aggCacheLoader.start();
+        AggregateCacheLoader cacheLoader = new AggregateCacheLoader(node);
 
-        U.sleep(3000);
+        cacheLoader.loadByTime(5_000).get();
 
         forceCheckpoint();
 
-        U.sleep(3000);
-
-        aggCacheLoader.stop();
+        cacheLoader.loadByTime(5_000).get();
 
         stopGrid(2, true);
 
-        startGrid(2);
+        node = startGrid(2);
 
         awaitPartitionMapExchange();
 
-        aggCacheLoader.consistencyCheck(grid(2));
+        cacheLoader.consistencyCheck(node);
+
+        checkNoRebalanceAfterRecovery();
     }
 
     /**
@@ -203,29 +212,27 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
 
         IgniteEx node = grid(2);
 
-        AggregateCacheLoader aggCacheLoader = new AggregateCacheLoader(node);
-
-        aggCacheLoader.start();
+        AggregateCacheLoader cacheLoader = new AggregateCacheLoader(node);
 
-        U.sleep(3000);
+        cacheLoader.loadByTime(5_000).get();
 
         forceCheckpoint();
 
-        U.sleep(3000);
-
-        aggCacheLoader.stop();
+        cacheLoader.loadByTime(5_000).get();
 
         stopGrid(2, true);
 
         crd.cluster().active(false);
 
-        startGrid(2);
+        node = startGrid(2);
 
         crd.cluster().active(true);
 
         awaitPartitionMapExchange();
 
-        aggCacheLoader.consistencyCheck(grid(2));
+        checkNoRebalanceAfterRecovery();
+
+        cacheLoader.consistencyCheck(node);
     }
 
     /**
@@ -268,17 +275,13 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
 
         node.getOrCreateCaches(dynamicCaches);
 
-        AggregateCacheLoader aggCacheLoader = new AggregateCacheLoader(node);
-
-        aggCacheLoader.start();
+        AggregateCacheLoader cacheLoader = new AggregateCacheLoader(node);
 
-        U.sleep(3000);
+        cacheLoader.loadByTime(5_000).get();
 
         forceCheckpoint();
 
-        U.sleep(3000);
-
-        aggCacheLoader.stop();
+        cacheLoader.loadByTime(5_000).get();
 
         stopGrid(2, true);
 
@@ -286,8 +289,10 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
 
         awaitPartitionMapExchange();
 
+        checkNoRebalanceAfterRecovery();
+
         for (int idx = 0; idx < 3; idx++)
-            aggCacheLoader.consistencyCheck(grid(idx));
+            cacheLoader.consistencyCheck(grid(idx));
     }
 
     /**
@@ -300,17 +305,13 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
 
         IgniteEx node = grid(2);
 
-        AggregateCacheLoader aggCacheLoader = new AggregateCacheLoader(node);
+        AggregateCacheLoader cacheLoader = new AggregateCacheLoader(node);
 
-        aggCacheLoader.start();
-
-        U.sleep(3000);
+        cacheLoader.loadByTime(5_000).get();
 
         forceCheckpoint();
 
-        U.sleep(3000);
-
-        aggCacheLoader.stop();
+        cacheLoader.loadByTime(5_000).get();
 
         stopGrid(2, true);
 
@@ -323,34 +324,26 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
         awaitPartitionMapExchange();
 
         for (int idx = 0; idx < 3; idx++)
-            aggCacheLoader.consistencyCheck(grid(idx));
+            cacheLoader.consistencyCheck(grid(idx));
     }
 
     /**
      *
      */
     public void testRecoveryOnCrushDuringCheckpointOnNodeStart() throws Exception {
-        // Crash recovery fails because of the bug in pages recycling.
-        // Test passes if don't perform removes in cache loader.
-        fail("https://issues.apache.org/jira/browse/IGNITE-9303");
-
         IgniteEx crd = (IgniteEx) startGridsMultiThreaded(3, false);
 
         crd.cluster().active(true);
 
         IgniteEx node = grid(2);
 
-        AggregateCacheLoader aggCacheLoader = new AggregateCacheLoader(node);
-
-        aggCacheLoader.start();
+        AggregateCacheLoader cacheLoader = new AggregateCacheLoader(node);
 
-        U.sleep(3000);
+        cacheLoader.loadByTime(5_000).get();
 
         forceCheckpoint();
 
-        U.sleep(3000);
-
-        aggCacheLoader.stop();
+        cacheLoader.loadByTime(5_000).get();
 
         stopGrid(2, false);
 
@@ -363,6 +356,18 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
         }
         catch (Exception expected) { }
 
+        // Wait until node will leave cluster.
+        GridTestUtils.waitForCondition(() -> {
+            try {
+                grid(2);
+            }
+            catch (IgniteIllegalStateException e) {
+                return true;
+            }
+
+            return false;
+        }, getTestTimeout());
+
         ioFactory = null;
 
         // Start node again and check recovery.
@@ -370,8 +375,10 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
 
         awaitPartitionMapExchange();
 
+        checkNoRebalanceAfterRecovery();
+
         for (int idx = 0; idx < 3; idx++)
-            aggCacheLoader.consistencyCheck(grid(idx));
+            cacheLoader.consistencyCheck(grid(idx));
     }
 
     /** {@inheritDoc} */
@@ -381,7 +388,30 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
-        return 600 * 1000;
+        return 120 * 1000;
+    }
+
+    /**
+     * Method checks that there were no rebalance for all caches (excluding sys cache).
+     */
+    private void checkNoRebalanceAfterRecovery() {
+        int sysCacheGroupId = CU.cacheId(GridCacheUtils.UTILITY_CACHE_NAME);
+
+        List<Ignite> nodes = G.allGrids();
+
+        for (Ignite node : nodes) {
+            TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(node);
+
+            List<Integer> rebalancedGroups = spi.recordedMessages(true).stream()
+                .map(msg -> (GridDhtPartitionDemandMessage) msg)
+                .map(msg -> msg.groupId())
+                .filter(grpId -> grpId != sysCacheGroupId)
+                .distinct()
+                .collect(Collectors.toList());
+
+            Assert.assertTrue("There was unexpected rebalance for some groups" +
+                    " [node=" + node.name() + ", groups=" + rebalancedGroups + ']', rebalancedGroups.isEmpty());
+        }
     }
 
     /**
@@ -389,59 +419,50 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
      */
     private static class AggregateCacheLoader {
         /** Ignite. */
-        IgniteEx ignite;
-
-        /** Stop flag. */
-        AtomicBoolean stopFlag;
+        final IgniteEx ignite;
 
         /** Cache loaders. */
-        Map<CacheLoader, IgniteInternalFuture> cacheLoaders;
+        final List<CacheLoader> cacheLoaders;
 
         /**
          * @param ignite Ignite.
          */
         public AggregateCacheLoader(IgniteEx ignite) {
             this.ignite = ignite;
+
+            List<CacheLoader> cacheLoaders = new ArrayList<>();
+
+            for (String cacheName : ignite.cacheNames())
+                cacheLoaders.add(new CacheLoader(ignite, cacheName));
+
+            this.cacheLoaders = cacheLoaders;
         }
 
         /**
-         *
+         * @param timeMillis Loading time in milliseconds.
          */
-        public void start() {
-            if (stopFlag != null && !stopFlag.get())
-                throw new IllegalStateException("Cache loaders must be stopped before start again");
-
-            stopFlag = new AtomicBoolean();
-            cacheLoaders = new HashMap<>();
+        public IgniteInternalFuture<?> loadByTime(int timeMillis) {
+            GridCompoundFuture<?, ?> loadFut = new GridCompoundFuture();
 
-            for (String cacheName : ignite.cacheNames()) {
-                CacheLoader loader = new CacheLoader(ignite, stopFlag, cacheName);
+            for (CacheLoader cacheLoader : cacheLoaders) {
+                long endTime = U.currentTimeMillis() + timeMillis;
 
-                IgniteInternalFuture loadFuture = GridTestUtils.runAsync(loader);
+                cacheLoader.stopPredicate = it -> U.currentTimeMillis() >= endTime;
 
-                cacheLoaders.put(loader, loadFuture);
+                loadFut.add(GridTestUtils.runAsync(cacheLoader));
             }
-        }
 
-        /**
-         *
-         */
-        public void stop() throws IgniteCheckedException {
-            if (stopFlag != null)
-                stopFlag.set(true);
+            loadFut.markInitialized();
 
-            if (cacheLoaders != null)
-                for (IgniteInternalFuture loadFuture : cacheLoaders.values())
-                    loadFuture.get();
+            return loadFut;
         }
 
         /**
-         * @param ignite Ignite.
+         * @param ignite Ignite node to check consistency from.
          */
         public void consistencyCheck(IgniteEx ignite) {
-            if (cacheLoaders != null)
-                for (CacheLoader cacheLoader : cacheLoaders.keySet())
-                    cacheLoader.consistencyCheck(ignite);
+            for (CacheLoader cacheLoader : cacheLoaders)
+                cacheLoader.consistencyCheck(ignite);
         }
     }
 
@@ -455,38 +476,38 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
         /** Ignite. */
         final IgniteEx ignite;
 
-        /** Stop flag. */
-        final AtomicBoolean stopFlag;
+        /** Stop predicate. */
+        volatile Predicate<IgniteEx> stopPredicate;
 
         /** Cache name. */
         final String cacheName;
 
         /** Local cache. */
-        final Map<Object, Object> locCache = new ConcurrentHashMap<>();
+        final Map<Integer, TestValue> locCache = new ConcurrentHashMap<>();
 
         /**
          * @param ignite Ignite.
-         * @param stopFlag Stop flag.
          * @param cacheName Cache name.
          */
-        public CacheLoader(IgniteEx ignite, AtomicBoolean stopFlag, String cacheName) {
+        public CacheLoader(IgniteEx ignite, String cacheName) {
             this.ignite = ignite;
-            this.stopFlag = stopFlag;
             this.cacheName = cacheName;
         }
 
         /** {@inheritDoc} */
         @Override public void run() {
-            while (!stopFlag.get()) {
+            final Predicate<IgniteEx> predicate = stopPredicate;
+
+            while (!predicate.test(ignite)) {
                 ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
                 int key = rnd.nextInt(KEYS_SPACE);
 
                 boolean remove = rnd.nextInt(100) <= 20;
 
-                IgniteCache<Object, Object> cache = ignite.getOrCreateCache(cacheName);
-
                 try {
+                    IgniteCache<Object, Object> cache = ignite.getOrCreateCache(cacheName);
+
                     if (remove) {
                         cache.remove(key);
 
@@ -496,10 +517,15 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
                         int[] payload = new int[KEYS_SPACE];
                         Arrays.fill(payload, key);
 
-                        cache.put(key, payload);
+                        TestValue val = new TestValue(key, payload);
 
-                        locCache.put(key, payload);
+                        cache.put(key, val);
+
+                        locCache.put(key, val);
                     }
+
+                    // Throttle against GC.
+                    U.sleep(1);
                 }
                 catch (Exception ignored) { }
             }
@@ -509,14 +535,14 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
          *
          */
         public void consistencyCheck(IgniteEx ignite) {
-            IgniteCache<Object, Object> cache = ignite.getOrCreateCache(cacheName);
+            IgniteCache<Integer, TestValue> cache = ignite.getOrCreateCache(cacheName);
 
             for (int key = 0; key < KEYS_SPACE; key++) {
-                int[] expectedValue = (int[]) locCache.get(key);
-                int[] actualValue = (int[]) cache.get(key);
+                TestValue expectedVal = locCache.get(key);
+                TestValue actualVal = cache.get(key);
 
                 Assert.assertEquals("Consistency check failed for: " + cache.getName() + ", key=" + key,
-                    arrayToString(expectedValue), arrayToString(actualValue));
+                    expectedVal, actualVal);
             }
         }
 
@@ -566,21 +592,45 @@ public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @param arr Array.
+     * Test payload with indexed field.
      */
-    static String arrayToString(int[] arr) {
-        if (arr == null)
-            return "null";
+    static class TestValue {
+        /** Indexed field. */
+        @QuerySqlField(index = true)
+        private final int indexedField;
 
-        StringBuilder sb = new StringBuilder();
+        /** Payload. */
+        private final int[] payload;
 
-        sb.append('[');
+        /**
+         * @param indexedField Indexed field.
+         * @param payload Payload.
+         */
+        public TestValue(int indexedField, int[] payload) {
+            this.indexedField = indexedField;
+            this.payload = payload;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestValue testValue = (TestValue) o;
+
+            return indexedField == testValue.indexedField &&
+                Arrays.equals(payload, testValue.payload);
+        }
 
-        for (int i = 0; i < Math.min(arr.length, 10); i++)
-            sb.append(i).append(",");
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int result = Objects.hash(indexedField);
 
-        sb.append(']');
+            result = 31 * result + Arrays.hashCode(payload);
 
-        return sb.toString();
+            return result;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java
index 93c4047..7e17271 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java
@@ -160,64 +160,6 @@ public class IgnitePdsDiskErrorsRecoveringTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Test node stopping & recovering on start marker writing fail during activation.
-     *
-     * @throws Exception If test failed.
-     */
-    public void testRecoveringOnNodeStartMarkerWriteFail() throws Exception {
-        // Fail to write node start marker tmp file at the second checkpoint. Pass only initial checkpoint.
-        ioFactory = new FilteringFileIOFactory("started.bin" + FilePageStoreManager.TMP_SUFFIX, new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), 20));
-
-        IgniteEx grid = startGrid(0);
-        grid.cluster().active(true);
-
-        for (int i = 0; i < 1000; i++) {
-            byte payload = (byte) i;
-            byte[] data = new byte[2048];
-            Arrays.fill(data, payload);
-
-            grid.cache(CACHE_NAME).put(i, data);
-        }
-
-        stopAllGrids();
-
-        boolean activationFailed = false;
-        try {
-            grid = startGrid(0);
-        }
-        catch (IgniteCheckedException e) {
-            boolean interrupted = Thread.interrupted();
-
-            if (interrupted)
-                log.warning("Ignore interrupted excpetion [" +
-                    "thread=" + Thread.currentThread().getName() + ']', e);
-
-            activationFailed = true;
-        }
-
-        Assert.assertTrue("Ignite instance startup must be failed", activationFailed);
-
-        // Grid should be automatically stopped after checkpoint fail.
-        awaitStop(grid);
-
-        // Grid should be successfully recovered after stopping.
-        ioFactory = null;
-
-        IgniteEx recoveredGrid = startGrid(0);
-        recoveredGrid.cluster().active(true);
-
-        for (int i = 0; i < 1000; i++) {
-            byte payload = (byte) i;
-            byte[] data = new byte[2048];
-            Arrays.fill(data, payload);
-
-            byte[] actualData = (byte[]) recoveredGrid.cache(CACHE_NAME).get(i);
-            Assert.assertArrayEquals(data, actualData);
-        }
-    }
-
-
-    /**
      * Test node stopping & recovering on checkpoint begin fail.
      *
      * @throws Exception If test failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java
index 906f191..1a1d449 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java
@@ -23,7 +23,6 @@ import java.nio.file.FileVisitResult;
 import java.nio.file.Path;
 import java.nio.file.SimpleFileVisitor;
 import java.nio.file.attribute.BasicFileAttributes;
-import java.util.HashMap;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
@@ -178,7 +177,7 @@ public class IgniteNodeStoppedDuringDisableWALTest extends GridCommonAbstractTes
         boolean fail = false;
 
         try (WALIterator it = sharedContext.wal().replay(null)) {
-            dbMgr.applyUpdatesOnRecovery(it, (tup) -> true, (entry) -> true, new HashMap<>());
+            dbMgr.applyUpdatesOnRecovery(it, (ptr, rec) -> true, (entry) -> true);
         }
         catch (IgniteCheckedException e) {
             if (nodeStopPoint.needCleanUp)

http://git-wip-us.apache.org/repos/asf/ignite/blob/b48a291e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
index 3374860..5bf7e7f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
@@ -387,13 +387,12 @@ public class WalCompactionTest extends GridCommonAbstractTest {
 
         File[] cpMarkers = cpMarkersDir.listFiles(new FilenameFilter() {
             @Override public boolean accept(File dir, String name) {
-                return !(
-                    name.equals(cpMarkersToSave[0].getName()) ||
-                    name.equals(cpMarkersToSave[1].getName()) ||
-                    name.equals(cpMarkersToSave[2].getName()) ||
-                    name.equals(cpMarkersToSave[3].getName()) ||
-                    name.equals(cpMarkersToSave[4].getName())
-                );
+                for (File cpMarker : cpMarkersToSave) {
+                    if (cpMarker.getName().equals(name))
+                        return false;
+                }
+
+                return true;
             }
         });
 


[19/50] [abbrv] ignite git commit: IGNITE-10320 Web Console: Workaround for memory leak in chart component.

Posted by sb...@apache.org.
IGNITE-10320 Web Console: Workaround for memory leak in chart component.


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

Branch: refs/heads/ignite-10044
Commit: 166e87ed19929b9e064732752d5a5ce1069b5076
Parents: 96e2fdd
Author: Alexander Kalinin <ve...@yandex.ru>
Authored: Mon Nov 19 16:47:12 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Mon Nov 19 16:47:12 2018 +0700

----------------------------------------------------------------------
 .../app/components/ignite-chart/controller.js   | 48 +++++++++++++-------
 1 file changed, 31 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/166e87ed/modules/web-console/frontend/app/components/ignite-chart/controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/ignite-chart/controller.js b/modules/web-console/frontend/app/components/ignite-chart/controller.js
index 79156fc..81ed273 100644
--- a/modules/web-console/frontend/app/components/ignite-chart/controller.js
+++ b/modules/web-console/frontend/app/components/ignite-chart/controller.js
@@ -21,22 +21,13 @@ import _ from 'lodash';
  * @typedef {{x: number, y: {[key: string]: number}}} IgniteChartDataPoint
  */
 
-const RANGE_RATE_PRESET = [{
-    label: '1 min',
-    value: 1
-}, {
-    label: '5 min',
-    value: 5
-}, {
-    label: '10 min',
-    value: 10
-}, {
-    label: '15 min',
-    value: 15
-}, {
-    label: '30 min',
-    value: 30
-}];
+const RANGE_RATE_PRESET = [
+    {label: '1 min', value: 1},
+    {label: '5 min', value: 5},
+    {label: '10 min', value: 10},
+    {label: '15 min', value: 15},
+    {label: '30 min', value: 30}
+];
 
 export class IgniteChartController {
     /** @type {import('chart.js').ChartConfiguration} */
@@ -227,7 +218,8 @@ export class IgniteChartController {
                         duration: this.currentRange.value * 1000 * 60,
                         frameRate: 1000 / this.refreshRate || 1 / 3,
                         refresh: this.refreshRate || 3000,
-                        ttl: this.maxRangeInMilliseconds,
+                        // Temporary workaround before https://github.com/nagix/chartjs-plugin-streaming/issues/53 resolved.
+                        // ttl: this.maxRangeInMilliseconds,
                         onRefresh: () => {
                             this.onRefresh();
                         }
@@ -308,6 +300,24 @@ export class IgniteChartController {
                 this.config.data.datasets[datasetIndex].fill = false;
             }
         });
+
+        // Temporary workaround before https://github.com/nagix/chartjs-plugin-streaming/issues/53 resolved.
+        this.pruneHistory();
+    }
+
+    // Temporary workaround before https://github.com/nagix/chartjs-plugin-streaming/issues/53 resolved.
+    pruneHistory() {
+        if (!this.xRangeUpdateInProgress) {
+            const currenTime = Date.now();
+
+            while (currenTime - this.localHistory[0].x > this.maxRangeInMilliseconds)
+                this.localHistory.shift();
+
+            this.config.data.datasets.forEach((dataset) => {
+                while (currenTime - dataset.data[0].x > this.maxRangeInMilliseconds)
+                    dataset.data.shift();
+            });
+        }
     }
 
     /**
@@ -346,6 +356,8 @@ export class IgniteChartController {
 
     changeXRange(range) {
         if (this.chart) {
+            this.xRangeUpdateInProgress = true;
+
             this.chart.config.options.plugins.streaming.duration = range.value * 60 * 1000;
 
             this.clearDatasets();
@@ -353,6 +365,8 @@ export class IgniteChartController {
 
             this.onRefresh();
             this.rerenderChart();
+
+            this.xRangeUpdateInProgress = false;
         }
     }
 


[22/50] [abbrv] ignite git commit: IGNITE-9558 Avoid blocking transactions on client connect when possible - Fixes #4933.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
index 88005d0..b05c275 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
@@ -411,7 +411,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 writer.incrementState();
 
             case 8:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -500,7 +500,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 reader.incrementState();
 
             case 8:
-                topVer = reader.readMessage("topVer");
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 93a101b..d715362 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -688,7 +688,12 @@ public class IgniteTxHandler {
     private boolean needRemap(AffinityTopologyVersion expVer,
         AffinityTopologyVersion curVer,
         GridNearTxPrepareRequest req) {
-        if (expVer.equals(curVer))
+        if (curVer.equals(expVer))
+            return false;
+
+        AffinityTopologyVersion lastAffChangedTopVer = ctx.exchange().lastAffinityChangedTopologyVersion(expVer);
+
+        if (curVer.compareTo(expVer) <= 0 && curVer.compareTo(lastAffChangedTopVer) >= 0)
             return false;
 
         // TODO IGNITE-6754 check mvcc crd for mvcc enabled txs.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
index efcb48b..2c3892f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
@@ -191,4 +191,4 @@ public class IgniteTxKey implements Message {
     @Override public String toString() {
         return S.toString(IgniteTxKey.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksRequest.java
index 94fe005..86109c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksRequest.java
@@ -149,13 +149,13 @@ public class TxLocksRequest extends GridCacheMessage {
         }
 
         switch (writer.state()) {
-            case 2:
+            case 3:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 3:
+            case 4:
                 if (!writer.writeObjectArray("txKeysArr", txKeysArr, MessageCollectionItemType.MSG))
                     return false;
 
@@ -177,7 +177,7 @@ public class TxLocksRequest extends GridCacheMessage {
             return false;
 
         switch (reader.state()) {
-            case 2:
+            case 3:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -185,7 +185,7 @@ public class TxLocksRequest extends GridCacheMessage {
 
                 reader.incrementState();
 
-            case 3:
+            case 4:
                 txKeysArr = reader.readObjectArray("txKeysArr", MessageCollectionItemType.MSG, IgniteTxKey.class);
 
                 if (!reader.isLastRead())
@@ -205,7 +205,7 @@ public class TxLocksRequest extends GridCacheMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 4;
+        return 5;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksResponse.java
index a5c8f09..df5caa9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksResponse.java
@@ -239,25 +239,25 @@ public class TxLocksResponse extends GridCacheMessage {
         }
 
         switch (writer.state()) {
-            case 2:
+            case 3:
                 if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 3:
+            case 4:
                 if (!writer.writeObjectArray("locksArr", locksArr, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 5:
                 if (!writer.writeObjectArray("nearTxKeysArr", nearTxKeysArr, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 6:
                 if (!writer.writeObjectArray("txKeysArr", txKeysArr, MessageCollectionItemType.MSG))
                     return false;
 
@@ -279,7 +279,7 @@ public class TxLocksResponse extends GridCacheMessage {
             return false;
 
         switch (reader.state()) {
-            case 2:
+            case 3:
                 futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
@@ -287,7 +287,7 @@ public class TxLocksResponse extends GridCacheMessage {
 
                 reader.incrementState();
 
-            case 3:
+            case 4:
                 locksArr = reader.readObjectArray("locksArr", MessageCollectionItemType.MSG, TxLockList.class);
 
                 if (!reader.isLastRead())
@@ -295,7 +295,7 @@ public class TxLocksResponse extends GridCacheMessage {
 
                 reader.incrementState();
 
-            case 4:
+            case 5:
                 nearTxKeysArr = reader.readObjectArray("nearTxKeysArr", MessageCollectionItemType.MSG, IgniteTxKey.class);
 
                 if (!reader.isLastRead())
@@ -303,7 +303,7 @@ public class TxLocksResponse extends GridCacheMessage {
 
                 reader.incrementState();
 
-            case 5:
+            case 6:
                 txKeysArr = reader.readObjectArray("txKeysArr", MessageCollectionItemType.MSG, IgniteTxKey.class);
 
                 if (!reader.isLastRead())
@@ -323,7 +323,7 @@ public class TxLocksResponse extends GridCacheMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 6;
+        return 7;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheRawVersionedEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheRawVersionedEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheRawVersionedEntry.java
index 586a043..ceea0a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheRawVersionedEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheRawVersionedEntry.java
@@ -24,6 +24,7 @@ import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
@@ -39,6 +40,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Raw versioned entry.
  */
+@IgniteCodeGeneratingFail
 public class GridCacheRawVersionedEntry<K, V> extends DataStreamerEntry implements
     GridCacheVersionedEntry<K, V>, GridCacheVersionable, Externalizable {
     /** */
@@ -381,4 +383,4 @@ public class GridCacheRawVersionedEntry<K, V> extends DataStreamerEntry implemen
             "valBytesLen", valBytes != null ? valBytes.length : "n/a",
             "super", super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
index ec04ac3..5e7cc9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
@@ -274,4 +274,4 @@ public class GridContinuousMessage implements Message {
     @Override public String toString() {
         return S.toString(GridContinuousMessage.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
index f70ee9c..87313de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
@@ -369,7 +369,7 @@ public class DataStreamerRequest implements Message {
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -508,7 +508,7 @@ public class DataStreamerRequest implements Message {
                 reader.incrementState();
 
             case 13:
-                topVer = reader.readMessage("topVer");
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerResponse.java
index 4cb46e1..56f37c2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerResponse.java
@@ -169,4 +169,4 @@ public class DataStreamerResponse implements Message {
     @Override public byte fieldsCount() {
         return 3;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
index f6bddb6..e990cb3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.hadoop.shuffle;
 
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -35,6 +36,7 @@ import java.nio.ByteBuffer;
 /**
  * Direct shuffle message.
  */
+@IgniteCodeGeneratingFail
 public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
     /** */
     private static final long serialVersionUID = 0L;
@@ -268,4 +270,4 @@ public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
     @Override public String toString() {
         return S.toString(HadoopDirectShuffleMessage.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java
index 07b8c2f..2660e28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java
@@ -23,6 +23,7 @@ import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage;
 import org.apache.ignite.internal.util.GridUnsafe;
@@ -36,6 +37,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 /**
  * Shuffle message.
  */
+@IgniteCodeGeneratingFail
 public class HadoopShuffleMessage implements Message, HadoopMessage {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java
index 5ae3fed..76a08f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java
@@ -195,4 +195,4 @@ public class IgfsAckMessage extends IgfsCommunicationMessage {
     @Override public byte fieldsCount() {
         return 3;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockKey.java
index 736525d..98bf306 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockKey.java
@@ -297,4 +297,4 @@ public final class IgfsBlockKey implements IgfsBaseBlockKey, Message, Externaliz
     @Override public String toString() {
         return S.toString(IgfsBlockKey.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlocksMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlocksMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlocksMessage.java
index 2ec54b2..5448c50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlocksMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlocksMessage.java
@@ -176,4 +176,4 @@ public class IgfsBlocksMessage extends IgfsCommunicationMessage {
     @Override public byte fieldsCount() {
         return 3;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCommunicationMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCommunicationMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCommunicationMessage.java
index 412c45b..7f5daa6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCommunicationMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCommunicationMessage.java
@@ -77,4 +77,4 @@ public abstract class IgfsCommunicationMessage implements Message {
     @Override public byte fieldsCount() {
         return 0;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java
index b5e9666..2c2c98c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java
@@ -190,4 +190,4 @@ public class IgfsDeleteMessage extends IgfsCommunicationMessage {
     @Override public String toString() {
         return S.toString(IgfsDeleteMessage.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerRequest.java
index bb605b8..c64b627 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerRequest.java
@@ -157,4 +157,4 @@ public class IgfsFragmentizerRequest extends IgfsCommunicationMessage {
     @Override public byte fieldsCount() {
         return 2;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerResponse.java
index 76793bf..f7857dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerResponse.java
@@ -118,4 +118,4 @@ public class IgfsFragmentizerResponse extends IgfsCommunicationMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSyncMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSyncMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSyncMessage.java
index 2b32084..2707de5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSyncMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSyncMessage.java
@@ -149,4 +149,4 @@ public class IgfsSyncMessage extends IgfsCommunicationMessage {
     @Override public byte fieldsCount() {
         return 2;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
index 6b976c2..7fdd5d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
@@ -220,7 +220,7 @@ public class GridQueryNextPageResponse implements Message {
                 writer.incrementState();
 
             case 6:
-                if (!writer.writeMessage("retry", retry))
+                if (!writer.writeAffinityTopologyVersion("retry", retry))
                     return false;
 
                 writer.incrementState();
@@ -310,7 +310,7 @@ public class GridQueryNextPageResponse implements Message {
                 reader.incrementState();
 
             case 6:
-                retry = reader.readMessage("retry");
+                retry = reader.readAffinityTopologyVersion("retry");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaOperationStatusMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaOperationStatusMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaOperationStatusMessage.java
index 5f75e60..9e81d5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaOperationStatusMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaOperationStatusMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.schema.message;
 
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -30,6 +31,7 @@ import java.util.UUID;
 /**
  * Schema operation status message.
  */
+@IgniteCodeGeneratingFail
 public class SchemaOperationStatusMessage implements Message {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java
index b8b4edb..f3ec0e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java
@@ -171,4 +171,4 @@ public class GridTaskResultRequest implements Message {
     @Override public byte fieldsCount() {
         return 2;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java
index b9bb27c..88b42f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java
@@ -217,4 +217,4 @@ public class GridTaskResultResponse implements Message {
     @Override public byte fieldsCount() {
         return 4;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java
index 0200d77..e9c7d01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java
@@ -491,4 +491,4 @@ public class GridByteArrayList implements Message, Externalizable {
     @Override public String toString() {
         return S.toString(GridByteArrayList.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java
index 53767d6..c03cf58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java
@@ -599,4 +599,4 @@ public class GridIntList implements Message, Externalizable {
             }
         };
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java
index 8d09539..1c022b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java
@@ -26,6 +26,7 @@ import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.NoSuchElementException;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
@@ -38,6 +39,7 @@ import org.jetbrains.annotations.Nullable;
  * Minimal list API to work with primitive longs. This list exists
  * to avoid boxing/unboxing when using standard list from Java.
  */
+@IgniteCodeGeneratingFail
 public class GridLongList implements Message, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionItemType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionItemType.java b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionItemType.java
index 2134912..2340dd9 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionItemType.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionItemType.java
@@ -84,7 +84,10 @@ public enum MessageCollectionItemType {
     IGNITE_UUID,
 
     /** Message. */
-    MSG;
+    MSG,
+
+    /** Topology version. */
+    AFFINITY_TOPOLOGY_VERSION;
 
     /** Enum values. */
     private static final MessageCollectionItemType[] VALS = values();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java
index 0502042..6feee1a 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.UUID;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.lang.IgniteUuid;
 
 /**
@@ -230,6 +231,14 @@ public interface MessageReader {
     public IgniteUuid readIgniteUuid(String name);
 
     /**
+     * Reads {@link AffinityTopologyVersion}.
+     *
+     * @param name Field name.
+     * @return {@link AffinityTopologyVersion}.
+     */
+    public AffinityTopologyVersion readAffinityTopologyVersion(String name);
+
+    /**
      * Reads nested message.
      *
      * @param name Field name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageWriter.java b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageWriter.java
index 692955f..14d4417 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageWriter.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageWriter.java
@@ -22,6 +22,7 @@ import java.util.BitSet;
 import java.util.Collection;
 import java.util.Map;
 import java.util.UUID;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.lang.IgniteUuid;
 
 /**
@@ -255,6 +256,15 @@ public interface MessageWriter {
     public boolean writeIgniteUuid(String name, IgniteUuid val);
 
     /**
+     * Writes {@link AffinityTopologyVersion}.
+     *
+     * @param name Field name.
+     * @param val {@link AffinityTopologyVersion}.
+     * @return Whether value was fully written.
+     */
+    public boolean writeAffinityTopologyVersion(String name, AffinityTopologyVersion val);
+
+    /**
      * Writes nested message.
      *
      * @param name Field name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java
index 6ecb145..5f02804 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java
@@ -118,4 +118,4 @@ public class JobStealingRequest implements Message {
     @Override public String toString() {
         return S.toString(JobStealingRequest.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/HandshakeMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/HandshakeMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/HandshakeMessage.java
index e3be9c9..f845b0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/HandshakeMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/HandshakeMessage.java
@@ -19,6 +19,7 @@ package org.apache.ignite.spi.communication.tcp.messages;
 
 import java.nio.ByteBuffer;
 import java.util.UUID;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -29,6 +30,7 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 /**
  * Handshake message.
  */
+@IgniteCodeGeneratingFail
 public class HandshakeMessage implements Message {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/HandshakeMessage2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/HandshakeMessage2.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/HandshakeMessage2.java
index f27a825..2207813 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/HandshakeMessage2.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/HandshakeMessage2.java
@@ -19,6 +19,7 @@ package org.apache.ignite.spi.communication.tcp.messages;
 
 import java.nio.ByteBuffer;
 import java.util.UUID;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -26,6 +27,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 /**
  * Updated handshake message.
  */
+@IgniteCodeGeneratingFail
 public class HandshakeMessage2 extends HandshakeMessage {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/NodeIdMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/NodeIdMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/NodeIdMessage.java
index 2c6aa30..12e2522 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/NodeIdMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/NodeIdMessage.java
@@ -19,6 +19,7 @@ package org.apache.ignite.spi.communication.tcp.messages;
 
 import java.nio.ByteBuffer;
 import java.util.UUID;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -29,6 +30,7 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 /**
  * Node ID message.
  */
+@IgniteCodeGeneratingFail
 public class NodeIdMessage implements Message {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/RecoveryLastReceivedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/RecoveryLastReceivedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/RecoveryLastReceivedMessage.java
index eef2655..95d43f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/RecoveryLastReceivedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/messages/RecoveryLastReceivedMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.spi.communication.tcp.messages;
 
 import java.nio.ByteBuffer;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -27,6 +28,7 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 /**
  * Recovery acknowledgment message.
  */
+@IgniteCodeGeneratingFail
 public class RecoveryLastReceivedMessage implements Message {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNoAffinityExchangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNoAffinityExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNoAffinityExchangeTest.java
new file mode 100644
index 0000000..45c6f25
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNoAffinityExchangeTest.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.locks.Lock;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessage;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class CacheNoAffinityExchangeTest extends GridCommonAbstractTest {
+    /** */
+    private volatile boolean startClient;
+
+    /** */
+    private final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder().setShared(true);
+
+    /** */
+    private final TcpDiscoveryIpFinder CLIENT_IP_FINDER = new TcpDiscoveryVmIpFinder()
+        .setAddresses(Collections.singleton("127.0.0.1:47500"));
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDiscoverySpi(new TestDiscoverySpi().setIpFinder(IP_FINDER));
+
+        if (startClient) {
+            cfg.setClientMode(true);
+
+            // It is necessary to ensure that client always connects to grid(0).
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(CLIENT_IP_FINDER);
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoAffinityChangeOnClientJoin() throws Exception {
+        Ignite ig = startGrids(4);
+
+        ig.cluster().active(true);
+
+        IgniteCache<Integer, Integer> atomicCache = ig.createCache(new CacheConfiguration<Integer, Integer>()
+                .setName("atomic").setAtomicityMode(CacheAtomicityMode.ATOMIC));
+
+        IgniteCache<Integer, Integer> txCache = ig.createCache(new CacheConfiguration<Integer, Integer>()
+                .setName("tx").setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL));
+
+        assertTrue(GridTestUtils.waitForCondition(() ->
+                new AffinityTopologyVersion(4, 3).equals(grid(3).context().discovery().topologyVersionEx()),
+            5_000));
+
+        TestDiscoverySpi discoSpi = (TestDiscoverySpi) grid(2).context().discovery().getInjectedDiscoverySpi();
+
+        CountDownLatch latch = new CountDownLatch(1);
+
+        discoSpi.latch = latch;
+
+        startClient = true;
+
+        startGrid(4);
+
+        assertTrue(GridTestUtils.waitForCondition(() ->
+                new AffinityTopologyVersion(5, 0).equals(grid(0).context().discovery().topologyVersionEx()) &&
+                    new AffinityTopologyVersion(5, 0).equals(grid(1).context().discovery().topologyVersionEx()) &&
+                    new AffinityTopologyVersion(4, 3).equals(grid(2).context().discovery().topologyVersionEx()) &&
+                    new AffinityTopologyVersion(4, 3).equals(grid(3).context().discovery().topologyVersionEx()),
+            10_000));
+
+        for (int k = 0; k < 100; k++) {
+            atomicCache.put(k, k);
+            txCache.put(k, k);
+
+            Lock lock = txCache.lock(k);
+            lock.lock();
+            lock.unlock();
+        }
+
+        for (int k = 0; k < 100; k++) {
+            assertEquals(Integer.valueOf(k), atomicCache.get(k));
+            assertEquals(Integer.valueOf(k), txCache.get(k));
+        }
+
+        assertEquals(new AffinityTopologyVersion(5, 0), grid(0).context().discovery().topologyVersionEx());
+        assertEquals(new AffinityTopologyVersion(5, 0), grid(1).context().discovery().topologyVersionEx());
+        assertEquals(new AffinityTopologyVersion(4, 3), grid(2).context().discovery().topologyVersionEx());
+        assertEquals(new AffinityTopologyVersion(4, 3), grid(3).context().discovery().topologyVersionEx());
+
+        latch.countDown();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoAffinityChangeOnClientLeft() throws Exception {
+        Ignite ig = startGrids(4);
+
+        ig.cluster().active(true);
+
+        IgniteCache<Integer, Integer> atomicCache = ig.createCache(new CacheConfiguration<Integer, Integer>()
+            .setName("atomic").setAtomicityMode(CacheAtomicityMode.ATOMIC));
+
+        IgniteCache<Integer, Integer> txCache = ig.createCache(new CacheConfiguration<Integer, Integer>()
+            .setName("tx").setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL));
+
+        assertTrue(GridTestUtils.waitForCondition(() ->
+            new AffinityTopologyVersion(4, 3).equals(grid(3).context().discovery().topologyVersionEx()),
+            5_000));
+
+        startClient = true;
+
+        startGrid(4);
+
+        TestDiscoverySpi discoSpi = (TestDiscoverySpi)grid(2).context().discovery().getInjectedDiscoverySpi();
+
+        CountDownLatch latch = new CountDownLatch(1);
+
+        discoSpi.latch = latch;
+
+        stopGrid(4);
+
+        assertTrue(GridTestUtils.waitForCondition(() ->
+                new AffinityTopologyVersion(6, 0).equals(grid(0).context().discovery().topologyVersionEx()) &&
+                    new AffinityTopologyVersion(6, 0).equals(grid(1).context().discovery().topologyVersionEx()) &&
+                    new AffinityTopologyVersion(5, 0).equals(grid(2).context().discovery().topologyVersionEx()) &&
+                    new AffinityTopologyVersion(5, 0).equals(grid(3).context().discovery().topologyVersionEx()),
+            10_000));
+
+        for (int k = 0; k < 100; k++) {
+            atomicCache.put(k, k);
+            txCache.put(k, k);
+
+            Lock lock = txCache.lock(k);
+            lock.lock();
+            lock.unlock();
+        }
+
+        for (int k = 0; k < 100; k++) {
+            assertEquals(Integer.valueOf(k), atomicCache.get(k));
+            assertEquals(Integer.valueOf(k), txCache.get(k));
+        }
+
+        assertEquals(new AffinityTopologyVersion(6, 0), grid(0).context().discovery().topologyVersionEx());
+        assertEquals(new AffinityTopologyVersion(6, 0), grid(1).context().discovery().topologyVersionEx());
+        assertEquals(new AffinityTopologyVersion(5, 0), grid(2).context().discovery().topologyVersionEx());
+        assertEquals(new AffinityTopologyVersion(5, 0), grid(3).context().discovery().topologyVersionEx());
+
+        latch.countDown();
+    }
+
+    /**
+     *
+     */
+    public static class TestDiscoverySpi extends TcpDiscoverySpi {
+        /** */
+        private volatile CountDownLatch latch;
+
+        /** {@inheritDoc} */
+        @Override protected void startMessageProcess(TcpDiscoveryAbstractMessage msg) {
+            if (msg instanceof TcpDiscoveryNodeAddFinishedMessage || msg instanceof TcpDiscoveryNodeLeftMessage || msg instanceof TcpDiscoveryNodeFailedMessage) {
+                CountDownLatch latch0 = latch;
+
+                if (latch0 != null)
+                    try {
+                        latch0.await();
+                    }
+                    catch (InterruptedException ex) {
+                        throw new IgniteException(ex);
+                    }
+            }
+
+            super.startMessageProcess(msg);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
index 90d4e4e..36d8b41 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
@@ -845,8 +845,7 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
         checkCaches1(SRVS + CLIENTS);
 
         // Wait for late affinity assignment to finish.
-        grid(0).context().cache().context().exchange().affinityReadyFuture(
-            new AffinityTopologyVersion(SRVS + CLIENTS, 1)).get();
+        awaitPartitionMapExchange();
 
         final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(SRVS + CLIENTS + 1, 1);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java
index 2e89922..93c1066 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java
@@ -457,6 +457,8 @@ public class CacheExchangeMergeTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testMergeServerAndClientJoin1() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10186");
+
         final IgniteEx srv0 = startGrid(0);
 
         mergeExchangeWaitVersion(srv0, 3);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
index e9d9663..c5384b9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.CacheNoAffinityExchangeTest;
 import org.apache.ignite.internal.processors.cache.PartitionedAtomicCacheGetsDistributionTest;
 import org.apache.ignite.internal.processors.cache.PartitionedTransactionalOptimisticCacheGetsDistributionTest;
 import org.apache.ignite.internal.processors.cache.PartitionedTransactionalPessimisticCacheGetsDistributionTest;
@@ -112,6 +113,8 @@ public class IgniteCacheTestSuite6 extends TestSuite {
 
         suite.addTestSuite(CacheParallelStartTest.class);
 
+        suite.addTestSuite(CacheNoAffinityExchangeTest.class);
+
         //suite.addTestSuite(CacheClientsConcurrentStartTest.class);
         //suite.addTestSuite(GridCacheRebalancingOrderingTest.class);
         //suite.addTestSuite(IgniteCacheClientMultiNodeUpdateTopologyLockTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryTable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryTable.java
index 54f5f03..ca6343c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryTable.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryTable.java
@@ -93,6 +93,7 @@ public class QueryTable implements Message {
                     return false;
 
                 writer.incrementState();
+
         }
 
         return true;
@@ -121,6 +122,7 @@ public class QueryTable implements Message {
                     return false;
 
                 reader.incrementState();
+
         }
 
         return reader.afterMessageRead(QueryTable.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
index 7024884..186104a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
@@ -30,6 +31,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 /**
  * Range request.
  */
+@IgniteCodeGeneratingFail
 public class GridH2IndexRangeRequest implements Message {
     /** */
     private UUID originNodeId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
index 4fe660c..18814bb 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
@@ -30,6 +31,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 /**
  * Range response message.
  */
+@IgniteCodeGeneratingFail
 public class GridH2IndexRangeResponse implements Message {
     /** */
     public static final byte STATUS_OK = 0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
index 0bec66e..cca366a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
@@ -50,6 +51,7 @@ import static org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuer
 /**
  * Query request.
  */
+@IgniteCodeGeneratingFail
 public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
     /** */
     private static final long serialVersionUID = 0L;
@@ -518,7 +520,7 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
                 writer.incrementState();
 
             case 9:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -633,7 +635,7 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
                 reader.incrementState();
 
             case 9:
-                topVer = reader.readMessage("topVer");
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
index 18f8880..fd5b524 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.query.h2.twostep.msg;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -28,6 +29,7 @@ import org.h2.value.Value;
 /**
  * Abstract message wrapper for H2 values.
  */
+@IgniteCodeGeneratingFail
 public abstract class GridH2ValueMessage implements Message {
     /**
      * Gets H2 value.


[43/50] [abbrv] ignite git commit: IGNITE-9828 MVCC: Notify Continuous Query manager about missed updates during PME. This closes #5189.

Posted by sb...@apache.org.
IGNITE-9828 MVCC: Notify Continuous Query manager about missed updates during PME. This closes #5189.


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

Branch: refs/heads/ignite-10044
Commit: 74f312e0c863824f1cefdd589a7a0d7da140a276
Parents: eb88885
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Tue Nov 20 15:26:31 2018 +0300
Committer: Igor Seliverstov <gv...@gmail.com>
Committed: Tue Nov 20 15:26:31 2018 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |  12 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../cache/IgniteCacheOffheapManager.java        |   4 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |   4 +-
 .../cache/PartitionUpdateCounter.java           |  28 +-
 .../GridDistributedTxRemoteAdapter.java         |   2 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |  53 ++-
 .../dht/GridDhtTxAbstractEnlistFuture.java      |  10 +-
 .../GridDhtPartitionsExchangeFuture.java        |  49 +++
 .../topology/GridClientPartitionTopology.java   |   8 +-
 .../dht/topology/GridDhtLocalPartition.java     |   7 +-
 .../dht/topology/GridDhtPartitionTopology.java  |   9 +-
 .../topology/GridDhtPartitionTopologyImpl.java  |  51 ++-
 .../cache/mvcc/MvccCachingManager.java          |  30 +-
 .../persistence/GridCacheOffheapManager.java    |   5 +-
 .../continuous/CacheContinuousQueryManager.java |  33 ++
 .../cache/transactions/IgniteTxAdapter.java     |   3 +
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../continuous/GridContinuousProcessor.java     |   6 +-
 ...tinuousQueryAsyncFailoverMvccTxSelfTest.java |  53 +++
 ...ContinuousQueryFailoverAbstractSelfTest.java |  90 ++++-
 ...usQueryFailoverMvccTxReplicatedSelfTest.java |  31 ++
 ...heContinuousQueryFailoverMvccTxSelfTest.java |  48 +++
 .../mvcc/CacheMvccBasicContinuousQueryTest.java | 363 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite4.java         |   6 +
 25 files changed, 821 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 9124884..96f4dcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -6733,12 +6733,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
         ensureFreeSpace();
 
+        CacheObject val = null;
+        CacheObject oldVal = null;
+
         lockEntry();
 
         try {
             checkObsolete();
 
-            CacheObject val = null;
 
             for (int i = 0; i < entries.size(); i++) {
                 GridCacheMvccEntryInfo info = (GridCacheMvccEntryInfo)entries.get(i);
@@ -6759,7 +6761,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     info.newMvccTxState());
             }
 
-            CacheObject oldVal = null;
 
             // Assume list contains  only previous committed row and rows changed by the current tx.
             if (!entries.isEmpty()) {
@@ -6806,7 +6807,12 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             }
         }
 
-        return new GridCacheUpdateTxResult(true, logPtr);
+        GridCacheUpdateTxResult res = new GridCacheUpdateTxResult(true, logPtr);
+
+        res.newValue(val);
+        res.oldValue(oldVal);
+
+        return res;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 55c5d6c..fa63b7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -1475,7 +1475,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     grp.affinity().similarAffinityKey());
 
                 if (sndCounters) {
-                    CachePartitionPartialCountersMap cntrsMap = grp.topology().localUpdateCounters(true, true);
+                    CachePartitionPartialCountersMap cntrsMap = grp.topology().localUpdateCounters(true);
 
                     m.addPartitionUpdateCounters(grp.groupId(),
                         newCntrMap ? cntrsMap : CachePartitionPartialCountersMap.toCountersMap(cntrsMap));
@@ -1499,7 +1499,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 top.similarAffinityKey());
 
             if (sndCounters) {
-                CachePartitionPartialCountersMap cntrsMap = top.localUpdateCounters(true, true);
+                CachePartitionPartialCountersMap cntrsMap = top.localUpdateCounters(true);
 
                 m.addPartitionUpdateCounters(top.groupId(),
                     newCntrMap ? cntrsMap : CachePartitionPartialCountersMap.toCountersMap(cntrsMap));

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
index d311708..2cf302f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
@@ -1117,8 +1117,10 @@ public interface IgniteCacheOffheapManager {
 
         /**
          * Flushes pending update counters closing all possible gaps.
+         *
+         * @return Even-length array of pairs [start, end] for each gap.
          */
-        public void finalizeUpdateCountres();
+        GridLongList finalizeUpdateCounters();
 
         /**
          * Preload a store into page memory.

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 5f467b3..08ce978 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -1580,8 +1580,8 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
         }
 
         /** {@inheritDoc} */
-        @Override public void finalizeUpdateCountres() {
-            pCntr.finalizeUpdateCountres();
+        @Override public GridLongList finalizeUpdateCounters() {
+            return pCntr.finalizeUpdateCounters();
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java
index fe44708..39d8d5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridLongList;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -180,14 +181,31 @@ public class PartitionUpdateCounter {
 
     /**
      * Flushes pending update counters closing all possible gaps.
+     *
+     * @return Even-length array of pairs [start, end] for each gap.
      */
-    public synchronized void finalizeUpdateCountres() {
-        Item last = queue.pollLast();
+    public synchronized GridLongList finalizeUpdateCounters() {
+        Item item = poll();
+
+        GridLongList gaps = null;
+
+        while (item != null) {
+            if (gaps == null)
+                gaps = new GridLongList((queue.size() + 1) * 2);
+
+            long start = cntr.get() + 1;
+            long end = item.start;
 
-        if (last != null)
-            update(last.start + last.delta);
+            gaps.add(start);
+            gaps.add(end);
+
+            // Close pending ranges.
+            update(item.start + item.delta);
+
+            item = poll();
+        }
 
-        queue.clear();
+        return gaps;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index c93e771..a915478 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -935,6 +935,8 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
         catch (IgniteCheckedException | RuntimeException | Error e) {
             state(UNKNOWN);
 
+            U.error(log, "Error during tx rollback.", e);
+
             if (e instanceof IgniteCheckedException)
                 throw new IgniteException(e);
             else if (e instanceof RuntimeException)

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 60463b4..4693232 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -1960,23 +1960,24 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                 req.subjectId(),
                 req.taskNameHash());
         }
-        catch (IgniteCheckedException | IgniteException ex) {
+        catch (Throwable e) {
             GridNearTxQueryResultsEnlistResponse res = new GridNearTxQueryResultsEnlistResponse(req.cacheId(),
                 req.futureId(),
                 req.miniId(),
                 req.version(),
-                ex);
+                e);
 
             try {
                 ctx.io().send(nearNode, res, ctx.ioPolicy());
             }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to send near enlist response [" +
-                    "txId=" + req.version() +
-                    ", node=" + nodeId +
-                    ", res=" + res + ']', e);
+            catch (IgniteCheckedException ioEx) {
+                U.error(log, "Failed to send near enlist response " +
+                    "[txId=" + req.version() + ", node=" + nodeId + ", res=" + res + ']', ioEx);
             }
 
+            if (e instanceof Error)
+                throw (Error) e;
+
             return;
         }
 
@@ -2231,26 +2232,6 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
 
     /**
      * @param primary Primary node.
-     * @param req Request.
-     * @param e Error.
-     */
-    private void onError(UUID primary, GridDhtTxQueryEnlistRequest req, Throwable e) {
-        GridDhtTxQueryEnlistResponse res = new GridDhtTxQueryEnlistResponse(ctx.cacheId(),
-            req.dhtFutureId(),
-            req.batchId(),
-            e);
-
-        try {
-            ctx.io().send(primary, res, ctx.ioPolicy());
-        }
-        catch (IgniteCheckedException ioEx) {
-            U.error(log, "Failed to send DHT enlist reply to primary node [node: " + primary + ", req=" + req +
-                ']', ioEx);
-        }
-    }
-
-    /**
-     * @param primary Primary node.
      * @param req Message.
      * @param first Flag if this is a first request in current operation.
      */
@@ -2321,8 +2302,22 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                     req + ']', ioEx);
             }
         }
-        catch (IgniteCheckedException e) {
-            onError(primary, req, e);
+        catch (Throwable e) {
+            GridDhtTxQueryEnlistResponse res = new GridDhtTxQueryEnlistResponse(ctx.cacheId(),
+                req.dhtFutureId(),
+                req.batchId(),
+                e);
+
+            try {
+                ctx.io().send(primary, res, ctx.ioPolicy());
+            }
+            catch (IgniteCheckedException ioEx) {
+                U.error(log, "Failed to send DHT enlist reply to primary node " +
+                    "[node: " + primary + ", req=" + req + ']', ioEx);
+            }
+
+            if (e instanceof Error)
+                throw (Error) e;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
index cfa8eb7..9949930 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
@@ -629,7 +629,7 @@ public abstract class GridDhtTxAbstractEnlistFuture<T> extends GridCacheFutureAd
                 updRes.oldValue(), tx.local(), tx.topologyVersion(), mvccSnapshot, cctx.cacheId(), tx, null, -1);
 
         if (op != EnlistOperation.LOCK)
-            addToBatch(entry.key(), val, updRes.mvccHistory(), entry.context().cacheId());
+            addToBatch(entry.key(), val, updRes.mvccHistory(), entry.context().cacheId(), op);
     }
 
     /**
@@ -642,7 +642,7 @@ public abstract class GridDhtTxAbstractEnlistFuture<T> extends GridCacheFutureAd
      * @param cacheId Cache Id.
      */
     private void addToBatch(KeyCacheObject key, Message val, List<MvccLinkAwareSearchRow> hist,
-        int cacheId) throws IgniteCheckedException {
+        int cacheId, EnlistOperation op) throws IgniteCheckedException {
         List<ClusterNode> backups = backupNodes(key);
 
         int part = cctx.affinity().partition(key);
@@ -680,13 +680,15 @@ public abstract class GridDhtTxAbstractEnlistFuture<T> extends GridCacheFutureAd
             if (batch == null)
                 batches.put(node.id(), batch = new Batch(node));
 
-            if (moving && hist0 == null) {
+            if (moving && hist0 == null && !op.isInvoke()) {
                 assert !F.isEmpty(hist);
 
                 hist0 = fetchHistoryInfo(key, hist);
             }
 
-            batch.add(key, moving ? hist0 : val);
+            Message m = moving && !op.isInvoke() ? hist0 : val;
+
+            batch.add(key, m);
 
             if (batch.size() == BATCH_SIZE) {
                 assert batches != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index a79c95f..44fc266 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -1466,6 +1466,9 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
         changeWalModeIfNeeded();
 
+        if (events().hasServerLeft())
+            finalizePartitionCounters();
+
         cctx.exchange().exchangerBlockingSectionBegin();
 
         try {
@@ -3465,6 +3468,17 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
+     * Collects non local cache groups.
+     *
+     * @return Collection of non local cache groups.
+     */
+    private List<CacheGroupContext> nonLocalCacheGroups() {
+        return cctx.cache().cacheGroups().stream()
+            .filter(grp -> !grp.isLocal() && !cacheGroupStopping(grp.groupId()))
+            .collect(Collectors.toList());
+    }
+
+    /**
      * Validates that partition update counters and cache sizes for all caches are consistent.
      */
     private void validatePartitionsState() {
@@ -3548,6 +3562,41 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
+     * Removes gaps in the local update counters. Gaps in update counters are possible on backup node when primary
+     * failed to send update counter deltas to backup.
+     */
+    private void finalizePartitionCounters() {
+        long time = System.currentTimeMillis();
+
+        try {
+            int parallelismLvl = cctx.kernalContext().config().getSystemThreadPoolSize();
+
+            // Reserve at least 2 threads for system operations.
+            parallelismLvl = Math.max(1, parallelismLvl - 2);
+
+            if (parallelismLvl > 1) {
+                U.doInParallel(parallelismLvl,
+                    cctx.kernalContext().getSystemExecutorService(),
+                    nonLocalCacheGroups(),
+                    grp -> {
+                        grp.topology().finalizeUpdateCounters();
+
+                        return null;
+                    }
+                );
+            }
+            else
+                nonLocalCacheGroups().forEach(grp -> grp.topology().finalizeUpdateCounters());
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to finalize partition counters", e);
+        }
+
+        if (log.isInfoEnabled())
+            log.info("Partition counters finalization performed in " + (System.currentTimeMillis() - time) + " ms.");
+    }
+
+    /**
      * @param finishState State.
      * @param msg Request.
      * @param nodeId Node ID.

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
index 01db508..b1d1048 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
@@ -1232,12 +1232,16 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Override public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros,
-        boolean finalizeCntrsBeforeCollecting) {
+    @Override public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros) {
         return CachePartitionPartialCountersMap.EMPTY;
     }
 
     /** {@inheritDoc} */
+    @Override public void finalizeUpdateCounters() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public Map<Integer, Long> partitionSizes() {
         return Collections.emptyMap();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
index 8ee7a9d..f3516a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.cache.extras.GridCacheObsoleteEntry
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner;
+import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridIterator;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -1362,9 +1363,11 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements
 
     /**
      * Flushes pending update counters closing all possible gaps.
+     *
+     * @return Even-length array of pairs [start, end] for each gap.
      */
-    public void finalizeUpdateCountres() {
-        store.finalizeUpdateCountres();
+    public GridLongList finalizeUpdateCounters() {
+        return store.finalizeUpdateCounters();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
index 25b284e..be8a789 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
@@ -345,17 +345,20 @@ public interface GridDhtPartitionTopology {
     public Collection<Integer> lostPartitions();
 
     /**
+     * Pre-processes partition update counters before exchange.
+     */
+    void finalizeUpdateCounters();
+
+    /**
      * @return Partition update counters.
      */
     public CachePartitionFullCountersMap fullUpdateCounters();
 
     /**
      * @param skipZeros {@code True} for adding zero counter to map.
-     * @param finalizeCntrsBeforeCollecting {@code True} indicates that partition counters should be finalized.
      * @return Partition update counters.
      */
-    public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros,
-        boolean finalizeCntrsBeforeCollecting);
+    public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros);
 
     /**
      * @return Partition cache sizes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
index 6418dc7..a127876 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
@@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.ExchangeDiscoveryEvents;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
@@ -57,6 +58,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.GridAtomicLong;
+import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.GridPartitionStateMap;
 import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -2670,6 +2672,49 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         }
     }
 
+    /**
+     * Pre-processes partition update counters before exchange.
+     */
+    @Override public void finalizeUpdateCounters() {
+        if (!grp.mvccEnabled())
+            return;
+
+        // It is need to acquire checkpoint lock before topology lock acquiring.
+        ctx.database().checkpointReadLock();
+
+        try {
+            lock.readLock().lock();
+
+            try {
+                for (int i = 0; i < locParts.length(); i++) {
+                    GridDhtLocalPartition part = locParts.get(i);
+
+                    if (part != null && part.state().active()) {
+                        // We need to close all gaps in partition update counters sequence. We assume this finalizing is
+                        // happened on exchange and hence all txs are completed. Therefore each gap in update counters
+                        // sequence is a result of undelivered DhtTxFinishMessage on backup (sequences on primary nodes
+                        // do not have gaps). Here we close these gaps and asynchronously notify continuous query engine
+                        // about the skipped events.
+                        AffinityTopologyVersion topVer = ctx.exchange().readyAffinityVersion();
+
+                        GridLongList gaps = part.finalizeUpdateCounters();
+
+                        if (gaps != null) {
+                            for (GridCacheContext ctx0 : grp.caches())
+                                ctx0.continuousQueries().closeBackupUpdateCountersGaps(ctx0, part.id(), topVer, gaps);
+                        }
+                    }
+                }
+            }
+            finally {
+                lock.readLock().unlock();
+            }
+        }
+        finally {
+            ctx.database().checkpointReadUnlock();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public CachePartitionFullCountersMap fullUpdateCounters() {
         lock.readLock().lock();
@@ -2683,8 +2728,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Override public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros,
-        boolean finalizeCntrsBeforeCollecting) {
+    @Override public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros) {
         lock.readLock().lock();
 
         try {
@@ -2705,9 +2749,6 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 if (part == null)
                     continue;
 
-                if (finalizeCntrsBeforeCollecting)
-                    part.finalizeUpdateCountres();
-
                 long updCntr = part.updateCounter();
                 long initCntr = part.initialUpdateCounter();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCachingManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCachingManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCachingManager.java
index 259f69b..8f83b6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCachingManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCachingManager.java
@@ -40,8 +40,10 @@ import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinu
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.TxCounters;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
@@ -146,16 +148,23 @@ public class MvccCachingManager extends GridCacheSharedManagerAdapter {
 
         Map<KeyCacheObject, MvccTxEntry> cached = buf.getCached();
 
-        if (F.isEmpty(cached) || !commit)
+        if (F.isEmpty(cached))
             return;
 
         TxCounters txCntrs = tx.txCounters(false);
 
-        assert txCntrs != null;
+        assert txCntrs != null || !commit;
 
-        Collection<PartitionUpdateCountersMessage> cntrsColl =  txCntrs.updateCounters();
+        if (txCntrs == null)
+            return;
+
+        Collection<PartitionUpdateCountersMessage> cntrsColl = txCntrs.updateCounters();
 
-        assert  !F.isEmpty(cntrsColl) : cntrsColl;
+        if (F.isEmpty(cntrsColl)) {
+            assert !commit;
+
+            return;
+        }
 
         // cacheId -> partId -> initCntr -> cntr + delta.
         Map<Integer, Map<Integer, T2<AtomicLong, Long>>> cntrsMap = new HashMap<>();
@@ -219,8 +228,8 @@ public class MvccCachingManager extends GridCacheSharedManagerAdapter {
                         contQryMgr.onEntryUpdated(
                             lsnrCol,
                             e.key(),
-                            e.value(),
-                            e.oldValue(),
+                            commit ? e.value() : null, // Force skip update counter if rolled back.
+                            commit ? e.oldValue() : null, // Force skip update counter if rolled back.
                             false,
                             e.key().partition(),
                             tx.local(),
@@ -253,7 +262,7 @@ public class MvccCachingManager extends GridCacheSharedManagerAdapter {
      * @return Map of listeners to be notified by this update.
      */
     public Map<UUID, CacheContinuousQueryListener> continuousQueryListeners(GridCacheContext ctx0, @Nullable IgniteInternalTx tx, KeyCacheObject key) {
-        boolean internal = key.internal() || !ctx0.userCache();
+        boolean internal = key != null && key.internal() || !ctx0.userCache();
 
         return ctx0.continuousQueries().notifyContinuousQueries(tx) ?
             ctx0.continuousQueries().updateListeners(internal, false) : null;
@@ -268,9 +277,11 @@ public class MvccCachingManager extends GridCacheSharedManagerAdapter {
         private IgniteUuid lastFutId;
 
         /** Main buffer for entries. */
+        @GridToStringInclude
         private Map<KeyCacheObject, MvccTxEntry> cached = new LinkedHashMap<>();
 
         /** Pending entries. */
+        @GridToStringInclude
         private SortedMap<Integer, Map<KeyCacheObject, MvccTxEntry>> pending;
 
         /**
@@ -337,5 +348,10 @@ public class MvccCachingManager extends GridCacheSharedManagerAdapter {
 
             pending.clear();
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(EnlistBuffer.class, this);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index 48e86e4..f24900f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -1722,12 +1722,11 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         }
 
         /** {@inheritDoc} */
-        @Override public void finalizeUpdateCountres() {
+        @Override public GridLongList finalizeUpdateCounters() {
             try {
                 CacheDataStore delegate0 = init0(true);
 
-                if (delegate0 != null)
-                    delegate0.finalizeUpdateCountres();
+                return delegate0 != null ? delegate0.finalizeUpdateCounters() : null;
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index 6bd3fc2..4c399bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -58,6 +58,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheManagerAdapter;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
@@ -67,6 +68,7 @@ import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.continuous.GridContinuousHandler;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI2;
@@ -275,6 +277,37 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * For cache updates in shared cache group need notify others caches CQ listeners
+     * that generated counter should be skipped.
+     *
+     * @param cctx Cache context.
+     * @param part Partition.
+     * @param topVer Topology version.
+     * @param gaps Even-length array of pairs [start, end] for each gap.
+     */
+    @Nullable public void closeBackupUpdateCountersGaps(GridCacheContext cctx,
+        int part,
+        AffinityTopologyVersion topVer,
+        GridLongList gaps) {
+        assert gaps != null && gaps.size() % 2 == 0;
+
+        for (int i = 0; i < gaps.size() / 2; i++) {
+            long gapStart = gaps.get(i * 2);
+            long gapStop = gaps.get(i * 2 + 1);
+
+            /*
+             * No user listeners should be called by this invocation. In the common case of partitioned cache or
+             * replicated cache with non-local-only listener gaps (dummy filtered CQ events) will be added to the
+             * backup queue without passing it to any listener. In the special case of local-only listener on
+             * replicated cache there is no backup queues used at all and therefore no gaps occur - all unfiltered
+             * events are passed to listeners upon arrive.
+             */
+            for (long cntr = gapStart; cntr <= gapStop; cntr++)
+                skipUpdateEvent(lsnrs, null, part, cntr, false, topVer);
+        }
+    }
+
+    /**
      * @param internal Internal entry flag (internal key or not user cache).
      * @param preload Whether update happened during preloading.
      * @return Registered listeners.

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 8b24e01..52659e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -273,6 +273,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
     protected ConsistentIdMapper consistentIdMapper;
 
     /** Mvcc tx update snapshot. */
+    @GridToStringInclude
     protected volatile MvccSnapshot mvccSnapshot;
 
     /** Rollback finish future. */
@@ -280,6 +281,8 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
     private volatile IgniteInternalFuture rollbackFut;
 
     /** */
+    @SuppressWarnings("unused")
+    @GridToStringExclude
     private volatile TxCounters txCounters;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index d715362..3685f7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -1895,7 +1895,7 @@ public class IgniteTxHandler {
                     invokeArgs = invokeVal.invokeArgs();
                 }
 
-                assert entryProc != null || !op.isInvoke();
+                assert entryProc != null || !op.isInvoke() : "entryProc=" + entryProc + ", op=" + op;
 
                 GridDhtCacheEntry entry = dht.entryExx(key, tx.topologyVersion());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index fbfd99b..4569f65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -1423,7 +1423,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
                 if (cache != null && !cache.isLocal() && cache.context().userCache())
                     req.addUpdateCounters(ctx.localNodeId(),
-                        toCountersMap(cache.context().topology().localUpdateCounters(false, false)));
+                        toCountersMap(cache.context().topology().localUpdateCounters(false)));
             }
         }
 
@@ -1564,7 +1564,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
                                 if (cache != null && !cache.isLocal() && cache.context().userCache()) {
                                     CachePartitionPartialCountersMap cntrsMap =
-                                        cache.context().topology().localUpdateCounters(false, false);
+                                        cache.context().topology().localUpdateCounters(false);
 
                                     cntrs = U.marshal(marsh, cntrsMap);
                                 }
@@ -2504,7 +2504,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
                         if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode())
                             cntrsPerNode.put(ctx.localNodeId(),
-                                toCountersMap(cctx.topology().localUpdateCounters(false, false)));
+                                toCountersMap(cctx.topology().localUpdateCounters(false)));
 
                         routine.handler().updateCounters(topVer, cntrsPerNode, cntrs);
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverMvccTxSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverMvccTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverMvccTxSelfTest.java
new file mode 100644
index 0000000..334d219
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverMvccTxSelfTest.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ *
+ */
+public class CacheContinuousQueryAsyncFailoverMvccTxSelfTest  extends CacheContinuousQueryFailoverAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL_SNAPSHOT;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean asyncCallback() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testBackupQueueEvict() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7311");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testLeftPrimaryAndBackupNodes() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10047");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
index 0fef7b2..e7c2261 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
@@ -74,8 +74,8 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.continuous.GridContinuousHandler;
 import org.apache.ignite.internal.processors.continuous.GridContinuousMessage;
 import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
@@ -87,6 +87,7 @@ import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.internal.util.typedef.PAX;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteAsyncCallback;
@@ -104,6 +105,7 @@ import org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionRollbackException;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.MINUTES;
@@ -111,6 +113,8 @@ import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  *
@@ -521,7 +525,7 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
             Affinity<Object> aff = grid(i).affinity(DEFAULT_CACHE_NAME);
 
             CachePartitionPartialCountersMap act = grid(i).cachex(DEFAULT_CACHE_NAME).context().topology()
-                .localUpdateCounters(false, false);
+                .localUpdateCounters(false);
 
             for (Map.Entry<Integer, Long> e : updCntrs.entrySet()) {
                 if (aff.mapPartitionToPrimaryAndBackups(e.getKey()).contains(grid(i).localNode())) {
@@ -760,6 +764,8 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
             }
         }, 5000L);
 
+        awaitPartitionMapExchange();
+
         for (; keyIter < keys.size(); keyIter++) {
             int key = keys.get(keyIter);
 
@@ -784,7 +790,18 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
                     expEvts.add(new T3<>((Object)key, (Object)val, (Object)key));
             }
 
-            clnCache.put(key, val);
+            boolean updated = false;
+
+            while (!updated) {
+                try {
+                    clnCache.put(key, val);
+
+                    updated = true;
+                }
+                catch (Exception ignore) {
+                    assertEquals(CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT, atomicityMode());
+                }
+            }
 
             filtered = !filtered;
         }
@@ -977,8 +994,8 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
                 T2<Object, Object> t = updates.get(key);
 
                 if (updateFromClient) {
-                    if (atomicityMode() == CacheAtomicityMode.TRANSACTIONAL) {
-                        try (Transaction tx = qryClient.transactions().txStart()) {
+                    if (atomicityMode() != CacheAtomicityMode.ATOMIC) {
+                        try (Transaction tx = qryClient.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
                             qryClientCache.put(key, key);
 
                             tx.commit();
@@ -993,8 +1010,8 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
                         qryClientCache.put(key, key);
                 }
                 else {
-                    if (atomicityMode() == CacheAtomicityMode.TRANSACTIONAL) {
-                        try (Transaction tx = ignite.transactions().txStart()) {
+                    if (atomicityMode() != CacheAtomicityMode.ATOMIC) {
+                        try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
                             cache.put(key, key);
 
                             tx.commit();
@@ -1755,18 +1772,30 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
                 if (filtered)
                     val = -val;
 
-                if (processorPut && prevVal != null) {
-                    qryClnCache.invoke(key, new CacheEntryProcessor<Object, Object, Void>() {
-                        @Override public Void process(MutableEntry<Object, Object> entry,
-                            Object... arguments) throws EntryProcessorException {
-                            entry.setValue(arguments[0]);
+                boolean updated = false;
+
+                while (!updated) {
+                    try {
+                        if (processorPut && prevVal != null) {
+                            qryClnCache.invoke(key, new CacheEntryProcessor<Object, Object, Void>() {
+                                @Override public Void process(MutableEntry<Object, Object> entry,
+                                    Object... arguments) throws EntryProcessorException {
+                                    entry.setValue(arguments[0]);
 
-                            return null;
+                                    return null;
+                                }
+                            }, val);
                         }
-                    }, val);
+                        else
+                            qryClnCache.put(key, val);
+
+                        updated = true;
+                    }
+                    catch (CacheException e) {
+                        assertTrue(X.hasCause(e, TransactionRollbackException.class));
+                        assertSame(atomicityMode(), CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT);
+                    }
                 }
-                else
-                    qryClnCache.put(key, val);
 
                 processorPut = !processorPut;
 
@@ -2020,7 +2049,20 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
 
                         Integer val = valCntr.incrementAndGet();
 
-                        Integer prevVal = (Integer)qryClnCache.getAndPut(key, val);
+                        Integer prevVal = null;
+
+                        boolean updated = false;
+
+                        while (!updated) {
+                            try {
+                                prevVal = (Integer)qryClnCache.getAndPut(key, val);
+
+                                updated = true;
+                            }
+                            catch (CacheException e) {
+                                assertSame(atomicityMode(), CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT);
+                            }
+                        }
 
                         expEvts.get(threadId).add(new T3<>((Object)key, (Object)val, (Object)prevVal));
 
@@ -2114,7 +2156,19 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
                 @Override public Object call() throws Exception {
                     Integer val0 = val.getAndIncrement();
 
-                    cache.put(key, val0);
+                    boolean updated = false;
+
+                    while (!updated) {
+                        try {
+                            cache.put(key, val0);
+
+                            updated = true;
+                        }
+                        catch (CacheException e) {
+                            assertTrue(X.hasCause(e, TransactionRollbackException.class));
+                            assertSame(atomicityMode(), CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT);
+                        }
+                    }
 
                     return null;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxReplicatedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxReplicatedSelfTest.java
new file mode 100644
index 0000000..2576d23
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxReplicatedSelfTest.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import org.apache.ignite.cache.CacheMode;
+
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ *
+ */
+public class CacheContinuousQueryFailoverMvccTxReplicatedSelfTest extends CacheContinuousQueryFailoverMvccTxSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return REPLICATED;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxSelfTest.java
new file mode 100644
index 0000000..76d1689
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxSelfTest.java
@@ -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.
+ */
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ *
+ */
+public class CacheContinuousQueryFailoverMvccTxSelfTest extends CacheContinuousQueryFailoverAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL_SNAPSHOT;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testBackupQueueEvict() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7311");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testLeftPrimaryAndBackupNodes() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10047");
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBasicContinuousQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBasicContinuousQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBasicContinuousQueryTest.java
index ed97b1b..e08341a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBasicContinuousQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBasicContinuousQueryTest.java
@@ -17,11 +17,14 @@
 package org.apache.ignite.internal.processors.cache.mvcc;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
 import javax.cache.Cache;
 import javax.cache.CacheException;
 import javax.cache.event.CacheEntryEvent;
@@ -29,23 +32,35 @@ import javax.cache.event.CacheEntryUpdatedListener;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.query.ContinuousQuery;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
+import org.apache.ignite.internal.processors.continuous.GridContinuousMessage;
 import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.transactions.Transaction;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccCachingManager.TX_SIZE_THRESHOLD;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.transactions.TransactionState.PREPARING;
+import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK;
 
 /**
  * Basic continuous queries test with enabled mvcc.
@@ -227,4 +242,352 @@ public class CacheMvccBasicContinuousQueryTest extends CacheMvccAbstractTest  {
             }
         },  CacheException.class, "Failed to run update. Transaction is too large. Consider reducing transaction size");
     }
+
+    /**
+     * @throws Exception  If failed.
+     */
+    public void testUpdateCountersGapClosedSimplePartitioned() throws Exception {
+        checkUpdateCountersGapIsProcessedSimple(CacheMode.PARTITIONED);
+    }
+
+    /**
+     * @throws Exception  If failed.
+     */
+    public void testUpdateCountersGapClosedSimpleReplicated() throws Exception {
+        checkUpdateCountersGapIsProcessedSimple(CacheMode.REPLICATED);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    private void checkUpdateCountersGapIsProcessedSimple(CacheMode cacheMode) throws Exception {
+        testSpi = true;
+
+        int srvCnt = 4;
+
+        startGridsMultiThreaded(srvCnt);
+
+        client = true;
+
+        IgniteEx nearNode = startGrid(srvCnt);
+
+        IgniteCache<Object, Object> cache = nearNode.createCache(
+            cacheConfiguration(cacheMode, FULL_SYNC, srvCnt - 1, srvCnt)
+                .setIndexedTypes(Integer.class, Integer.class));
+
+        IgniteEx primary = grid(0);
+
+        List<Integer> keys = primaryKeys(primary.cache(DEFAULT_CACHE_NAME), 3);
+
+        ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
+
+        List<CacheEntryEvent> arrivedEvts = new ArrayList<>();
+
+        CountDownLatch latch = new CountDownLatch(2);
+
+        qry.setLocalListener(new CacheEntryUpdatedListener<Integer, Integer>() {
+            @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+                for (CacheEntryEvent e : evts) {
+                    arrivedEvts.add(e);
+
+                    latch.countDown();
+                }
+            }
+        });
+
+        QueryCursor<Cache.Entry<Integer, Integer>> cur = nearNode.cache(DEFAULT_CACHE_NAME).query(qry);
+
+        // Initial value.
+        cache.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, 42)").setArgs(keys.get(0))).getAll();
+
+        Transaction txA = nearNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+        // prevent first transaction prepare on backups
+        TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(primary);
+
+        spi.blockMessages(new IgniteBiPredicate<ClusterNode, Message>() {
+                private final AtomicInteger limiter = new AtomicInteger();
+
+                @Override public boolean apply(ClusterNode node, Message msg) {
+                    if (msg instanceof GridDhtTxPrepareRequest)
+                        return limiter.getAndIncrement() < srvCnt - 1;
+
+                    if (msg instanceof GridContinuousMessage)
+                        return true;
+
+                    return false;
+                }
+            });
+
+        cache.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, 42)").setArgs(keys.get(1))).getAll();
+
+        txA.commitAsync();
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return nearNode.context().cache().context().tm().activeTransactions().stream().allMatch(tx -> tx.state() == PREPARING);
+            }
+        }, 3_000);
+
+        GridTestUtils.runAsync(() -> {
+            try (Transaction txB = nearNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, 42)").setArgs(keys.get(2)));
+
+                txB.commit();
+            }
+        }).get();
+
+        long primaryUpdCntr = getUpdateCounter(primary, keys.get(0));
+
+        assertEquals(3, primaryUpdCntr); // There were three updates.
+
+        // drop primary
+        stopGrid(primary.name());
+
+        // Wait all txs are rolled back.
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                boolean allRolledBack = true;
+
+                for (int i = 1; i < srvCnt; i++) {
+                    boolean rolledBack = grid(i).context().cache().context().tm().activeTransactions().stream().allMatch(tx -> tx.state() == ROLLED_BACK);
+
+                    allRolledBack &= rolledBack;
+                }
+
+                return allRolledBack;
+            }
+        }, 3_000);
+
+        for (int i = 1; i < srvCnt; i++) {
+            IgniteCache backupCache = grid(i).cache(DEFAULT_CACHE_NAME);
+
+            int size = backupCache.query(new SqlFieldsQuery("select * from Integer")).getAll().size();
+
+            long backupCntr = getUpdateCounter(grid(i), keys.get(0));
+
+            assertEquals(2, size);
+            assertEquals(primaryUpdCntr, backupCntr);
+        }
+
+        assertTrue(latch.await(3, SECONDS));
+
+        assertEquals(2, arrivedEvts.size());
+        assertEquals(keys.get(0), arrivedEvts.get(0).getKey());
+        assertEquals(keys.get(2), arrivedEvts.get(1).getKey());
+
+        cur.close();
+        nearNode.close();
+    }
+
+    /**
+     * @throws Exception  If failed.
+     */
+    public void testUpdateCountersGapClosedPartitioned() throws Exception {
+        checkUpdateCountersGapsClosed(CacheMode.PARTITIONED);
+    }
+
+    /**
+     * @throws Exception  If failed.
+     */
+    public void testUpdateCountersGapClosedReplicated() throws Exception {
+        checkUpdateCountersGapsClosed(CacheMode.REPLICATED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkUpdateCountersGapsClosed(CacheMode cacheMode) throws Exception {
+        testSpi = true;
+
+        int srvCnt = 4;
+
+        startGridsMultiThreaded(srvCnt);
+
+        IgniteEx nearNode = grid(srvCnt - 1);
+
+        IgniteCache<Object, Object> cache = nearNode.createCache(
+            cacheConfiguration(cacheMode, FULL_SYNC, srvCnt - 1, srvCnt)
+                .setIndexedTypes(Integer.class, Integer.class));
+
+        IgniteEx primary = grid(0);
+
+        Affinity<Object> aff = nearNode.affinity(cache.getName());
+
+        int[] nearBackupParts = aff.backupPartitions(nearNode.localNode());
+
+        int[] primaryParts = aff.primaryPartitions(primary.localNode());
+
+        Collection<Integer> nearSet = new HashSet<>();
+
+        for (int part : nearBackupParts)
+            nearSet.add(part);
+
+        Collection<Integer> primarySet = new HashSet<>();
+
+        for (int part : primaryParts)
+            primarySet.add(part);
+
+        // We need backup partitions on the near node.
+        nearSet.retainAll(primarySet);
+
+        List<Integer> keys = singlePartKeys(primary.cache(DEFAULT_CACHE_NAME), 20, nearSet.iterator().next());
+
+        int range = 3;
+
+        ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
+
+        List<CacheEntryEvent> arrivedEvts = new ArrayList<>();
+
+        CountDownLatch latch = new CountDownLatch(range * 2);
+
+        qry.setLocalListener(new CacheEntryUpdatedListener<Integer, Integer>() {
+            @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+                for (CacheEntryEvent e : evts) {
+                    arrivedEvts.add(e);
+
+                    latch.countDown();
+                }
+            }
+        });
+
+        QueryCursor<Cache.Entry<Integer, Integer>> cur = nearNode.cache(DEFAULT_CACHE_NAME).query(qry);
+
+        // prevent first transaction prepare on backups
+        TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(primary);
+
+        spi.blockMessages(new IgniteBiPredicate<ClusterNode, Message>() {
+            private final AtomicInteger limiter = new AtomicInteger();
+
+            @Override public boolean apply(ClusterNode node, Message msg) {
+                if (msg instanceof GridDhtTxPrepareRequest)
+                    return limiter.getAndIncrement() < srvCnt - 1;
+
+                return false;
+            }
+        });
+
+        Transaction txA = primary.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+        for (int i = 0; i < range; i++)
+            primary.cache(DEFAULT_CACHE_NAME).put(keys.get(i), 2);
+
+        txA.commitAsync();
+
+        GridTestUtils.runAsync(() -> {
+            try (Transaction tx = primary.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                for (int i = range; i < range * 2; i++)
+                    primary.cache(DEFAULT_CACHE_NAME).put(keys.get(i), 1);
+
+                tx.commit();
+            }
+        }).get();
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return primary.context().cache().context().tm().activeTransactions().stream().allMatch(tx -> tx.state() == PREPARING);
+            }
+        }, 3_000);
+
+        GridTestUtils.runAsync(() -> {
+            try (Transaction txB = primary.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                for (int i = range * 2; i < range * 3; i++)
+                    primary.cache(DEFAULT_CACHE_NAME).put(keys.get(i), 3);
+
+                txB.commit();
+            }
+        }).get();
+
+        long primaryUpdCntr = getUpdateCounter(primary, keys.get(0));
+
+        assertEquals(range * 3, primaryUpdCntr);
+
+        // drop primary
+        stopGrid(primary.name());
+
+        // Wait all txs are rolled back.
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                boolean allRolledBack = true;
+
+                for (int i = 1; i < srvCnt; i++) {
+                    boolean rolledBack = grid(i).context().cache().context().tm().activeTransactions().stream().allMatch(tx -> tx.state() == ROLLED_BACK);
+
+                    allRolledBack &= rolledBack;
+                }
+
+                return allRolledBack;
+            }
+        }, 3_000);
+
+        for (int i = 1; i < srvCnt; i++) {
+            IgniteCache backupCache = grid(i).cache(DEFAULT_CACHE_NAME);
+
+            int size = backupCache.query(new SqlFieldsQuery("select * from Integer")).getAll().size();
+
+            long backupCntr = getUpdateCounter(grid(i), keys.get(0));
+
+            assertEquals(range * 2, size);
+            assertEquals(primaryUpdCntr, backupCntr);
+        }
+
+        assertTrue(latch.await(5, SECONDS));
+
+        assertEquals(range * 2, arrivedEvts.size());
+
+        cur.close();
+        nearNode.close();
+    }
+
+    /**
+     * @param primaryCache Cache.
+     * @param size Number of keys.
+     * @return Keys belong to a given part.
+     * @throws Exception If failed.
+     */
+    private List<Integer> singlePartKeys(IgniteCache<Object, Object> primaryCache, int size, int part) throws Exception {
+        Ignite ignite = primaryCache.unwrap(Ignite.class);
+
+        List<Integer> res = new ArrayList<>();
+
+        final Affinity<Object> aff = ignite.affinity(primaryCache.getName());
+
+        final ClusterNode node = ignite.cluster().localNode();
+
+        assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return aff.primaryPartitions(node).length > 0;
+            }
+        }, 5000));
+
+        int cnt = 0;
+
+        for (int key = 0; key < aff.partitions() * size * 10; key++) {
+            if (aff.partition(key) == part) {
+                res.add(key);
+
+                if (++cnt == size)
+                    break;
+            }
+        }
+
+        assertEquals(size, res.size());
+
+        return res;
+    }
+
+    /**
+     * @param node Node.
+     * @param key Key.
+     * @return Extracts update counter of partition which key belongs to.
+     */
+    private long getUpdateCounter(IgniteEx node, Integer key) {
+        int partId = node.cachex(DEFAULT_CACHE_NAME).context().affinity().partition(key);
+
+        GridDhtLocalPartition part = node.cachex(DEFAULT_CACHE_NAME).context().dht().topology().localPartition(partId);
+
+        assert part != null;
+
+        return part.updateCounter();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f312e0/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite4.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite4.java
index 2aa3419..147eb47 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite4.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite4.java
@@ -19,10 +19,13 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryAsyncFailoverAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryAsyncFailoverMvccTxSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryAsyncFailoverTxReplicatedSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryAsyncFailoverTxSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicReplicatedSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverMvccTxReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverMvccTxSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxReplicatedSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxSelfTest;
 
@@ -42,10 +45,13 @@ public class IgniteCacheQuerySelfTestSuite4 extends TestSuite {
         suite.addTestSuite(CacheContinuousQueryFailoverAtomicReplicatedSelfTest.class);
         suite.addTestSuite(CacheContinuousQueryFailoverTxSelfTest.class);
         suite.addTestSuite(CacheContinuousQueryFailoverTxReplicatedSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverMvccTxSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverMvccTxReplicatedSelfTest.class);
 
         suite.addTestSuite(CacheContinuousQueryAsyncFailoverAtomicSelfTest.class);
         suite.addTestSuite(CacheContinuousQueryAsyncFailoverTxReplicatedSelfTest.class);
         suite.addTestSuite(CacheContinuousQueryAsyncFailoverTxSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryAsyncFailoverMvccTxSelfTest.class);
 
         return suite;
     }


[39/50] [abbrv] ignite git commit: IGNITE-10328 Allow to destroy cache in code which were created via SQL - Fixes #5431.

Posted by sb...@apache.org.
IGNITE-10328 Allow to destroy cache in code which were created via SQL - Fixes #5431.

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/c8fd687f
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c8fd687f
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c8fd687f

Branch: refs/heads/ignite-10044
Commit: c8fd687fca7562d0560c8afc734b56f73edc76cf
Parents: 070819a
Author: EdShangGG <es...@gridgain.com>
Authored: Tue Nov 20 11:39:19 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 20 14:29:33 2018 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/ClusterCachesInfo.java     |  8 --------
 .../processors/cache/index/H2DynamicTableSelfTest.java   | 11 ++---------
 2 files changed, 2 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c8fd687f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 6ab4e67..6718cf3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -671,14 +671,6 @@ class ClusterCachesInfo {
                         continue;
                     }
 
-                    if (!req.sql() && desc.sql()) {
-                        ctx.cache().completeCacheStartFuture(req, false,
-                            new IgniteCheckedException("Only cache created with cache API may be removed with " +
-                                "direct call to destroyCache [cacheName=" + req.cacheName() + ']'));
-
-                        continue;
-                    }
-
                     DynamicCacheDescriptor old = registeredCaches.remove(req.cacheName());
 
                     if (req.restart())

http://git-wip-us.apache.org/repos/asf/ignite/blob/c8fd687f/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 0e06e35..07c0ede 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
@@ -808,7 +808,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     }
 
     /**
-     * Test that attempting to destroy via cache API a cache created via SQL yields an error.
+     * Test that attempting to destroy via cache API a cache created via SQL finishes successfully.
      * @throws Exception if failed.
      */
     public void testDestroyDynamicSqlCache() throws Exception {
@@ -816,14 +816,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
             " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
             "\"template=cache\"");
 
-        GridTestUtils.assertThrows(null, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                client().destroyCache(cacheName("Person"));
-
-                return null;
-            }
-        }, CacheException.class,
-        "Only cache created with cache API may be removed with direct call to destroyCache");
+        client().destroyCache(cacheName("Person"));
     }
 
     /**


[27/50] [abbrv] ignite git commit: IGNITE-9558 Avoid blocking transactions on client connect when possible - Fixes #4933.

Posted by sb...@apache.org.
IGNITE-9558 Avoid blocking transactions on client connect when possible - Fixes #4933.


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

Branch: refs/heads/ignite-10044
Commit: a63a81a51418a8ae1dadff019d9e5eace38e1631
Parents: 9e3bd7d
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Mon Nov 19 15:07:49 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Nov 19 15:07:49 2018 +0300

----------------------------------------------------------------------
 .../ignite/codegen/MessageCodeGenerator.java    |  20 +-
 .../ignite/internal/GridJobCancelRequest.java   |   2 +-
 .../ignite/internal/GridJobExecuteRequest.java  |   4 +-
 .../ignite/internal/GridJobExecuteResponse.java |   4 +-
 .../ignite/internal/GridJobSiblingsRequest.java |   2 +-
 .../internal/GridJobSiblingsResponse.java       |   2 +-
 .../ignite/internal/GridTaskCancelRequest.java  |   2 +-
 .../ignite/internal/GridTaskSessionRequest.java |   2 +-
 .../internal/direct/DirectMessageReader.java    |  30 ++
 .../internal/direct/DirectMessageWriter.java    |  26 ++
 .../direct/stream/DirectByteBufferStream.java   |  11 +
 .../stream/v1/DirectByteBufferStreamImplV1.java |  13 +
 .../stream/v2/DirectByteBufferStreamImplV2.java |  19 +-
 .../stream/v3/DirectByteBufferStreamImplV3.java | 298 +++++++++++++++++++
 .../checkpoint/GridCheckpointRequest.java       |   2 +-
 .../managers/communication/GridIoManager.java   |   2 +-
 .../communication/GridIoUserMessage.java        |   2 +-
 .../communication/IgniteIoTestMessage.java      |   2 +
 .../deployment/GridDeploymentInfoBean.java      |   2 +-
 .../deployment/GridDeploymentRequest.java       |   2 +-
 .../deployment/GridDeploymentResponse.java      |   2 +-
 .../eventstorage/GridEventStorageMessage.java   |   2 +-
 .../affinity/AffinityTopologyVersion.java       |   2 +-
 .../affinity/GridAffinityAssignmentCache.java   |  37 ++-
 .../UserAuthenticateResponseMessage.java        |   2 +-
 .../cache/CacheEntryInfoCollection.java         |   2 +-
 .../processors/cache/CacheEntryPredicate.java   |   2 +-
 .../cache/CacheEntryPredicateAdapter.java       |   2 +-
 .../cache/CacheEntryPredicateContainsValue.java |   2 +-
 .../cache/CacheEntryPredicateHasValue.java      |   2 +-
 .../cache/CacheEntryPredicateNoValue.java       |   2 +-
 .../cache/CacheEntrySerializablePredicate.java  |   2 +-
 .../processors/cache/CacheEvictionEntry.java    |   2 +-
 .../cache/CacheInvokeDirectResult.java          |   2 +-
 .../internal/processors/cache/CacheObject.java  |   2 +-
 .../processors/cache/CacheObjectAdapter.java    |   2 +-
 .../cache/CacheObjectByteArrayImpl.java         |   2 +-
 .../processors/cache/CacheObjectImpl.java       |   2 +-
 .../cache/GridCacheAffinityManager.java         |  14 +-
 .../cache/GridCacheGroupIdMessage.java          |   6 +-
 .../processors/cache/GridCacheIdMessage.java    |   6 +-
 .../processors/cache/GridCacheIoManager.java    |  17 +-
 .../processors/cache/GridCacheMessage.java      |  41 ++-
 .../GridCachePartitionExchangeManager.java      | 202 ++++++++++---
 .../GridChangeGlobalStateMessageResponse.java   |  10 +-
 .../processors/cache/KeyCacheObject.java        |   2 +-
 .../processors/cache/WalStateAckMessage.java    |   2 +
 .../distributed/GridCacheTtlUpdateRequest.java  |  30 +-
 .../distributed/GridCacheTxRecoveryRequest.java |  26 +-
 .../GridCacheTxRecoveryResponse.java            |  14 +-
 .../distributed/GridDistributedBaseMessage.java |  16 +-
 .../distributed/GridDistributedLockRequest.java |  54 ++--
 .../GridDistributedLockResponse.java            |  14 +-
 .../GridDistributedTxFinishRequest.java         |  62 ++--
 .../GridDistributedTxFinishResponse.java        |  18 +-
 .../GridDistributedTxPrepareRequest.java        |  54 ++--
 .../GridDistributedTxPrepareResponse.java       |  14 +-
 .../GridDistributedUnlockRequest.java           |   6 +-
 .../dht/ClientCacheDhtTopologyFuture.java       |   5 +
 .../dht/GridDhtAffinityAssignmentRequest.java   |  18 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |  26 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  10 +-
 .../distributed/dht/GridDhtLockRequest.java     |  50 ++--
 .../distributed/dht/GridDhtLockResponse.java    |  18 +-
 .../distributed/dht/GridDhtTopologyFuture.java  |   6 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |   7 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |  54 ++--
 .../dht/GridDhtTxFinishResponse.java            |  14 +-
 .../dht/GridDhtTxOnePhaseCommitAckRequest.java  |   6 +-
 .../dht/GridDhtTxPrepareRequest.java            |  44 +--
 .../dht/GridDhtTxPrepareResponse.java           |  22 +-
 .../dht/GridDhtTxQueryEnlistRequest.java        |  30 +-
 .../dht/GridDhtTxQueryEnlistResponse.java       |  14 +-
 .../dht/GridDhtTxQueryFirstEnlistRequest.java   |  42 +--
 .../distributed/dht/GridDhtUnlockRequest.java   |   6 +-
 .../dht/PartitionUpdateCountersMessage.java     |   2 +
 .../GridDhtAtomicAbstractUpdateRequest.java     |  42 +--
 .../dht/atomic/GridDhtAtomicCache.java          |   2 +-
 .../GridDhtAtomicDeferredUpdateResponse.java    |   6 +-
 .../dht/atomic/GridDhtAtomicNearResponse.java   |  22 +-
 .../GridDhtAtomicSingleUpdateRequest.java       |  18 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  66 ++--
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |  18 +-
 .../GridNearAtomicAbstractUpdateRequest.java    |  34 +--
 .../GridNearAtomicCheckUpdateRequest.java       |  10 +-
 .../atomic/GridNearAtomicFullUpdateRequest.java |  38 +--
 ...GridNearAtomicSingleUpdateFilterRequest.java |   6 +-
 ...GridNearAtomicSingleUpdateInvokeRequest.java |  10 +-
 .../GridNearAtomicSingleUpdateRequest.java      |  10 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |  34 +--
 .../dht/preloader/GridDhtForceKeysRequest.java  |  22 +-
 .../dht/preloader/GridDhtForceKeysResponse.java |  22 +-
 .../GridDhtPartitionDemandLegacyMessage.java    |  40 +--
 .../GridDhtPartitionDemandMessage.java          |  32 +-
 .../preloader/GridDhtPartitionExchangeId.java   |   6 +-
 .../GridDhtPartitionSupplyMessage.java          |  44 +--
 .../GridDhtPartitionSupplyMessageV2.java        |   6 +-
 .../GridDhtPartitionsAbstractMessage.java       |  14 +-
 .../GridDhtPartitionsExchangeFuture.java        |  12 +-
 .../preloader/GridDhtPartitionsFullMessage.java |  58 ++--
 .../GridDhtPartitionsSingleMessage.java         |  64 ++--
 .../GridDhtPartitionsSingleRequest.java         |   6 +-
 .../dht/preloader/latch/LatchAckMessage.java    |   6 +-
 .../topology/GridDhtPartitionTopologyImpl.java  | 112 ++++---
 .../distributed/near/CacheVersionedValue.java   |   2 +-
 .../distributed/near/GridNearGetRequest.java    |  58 ++--
 .../distributed/near/GridNearGetResponse.java   |  34 +--
 .../distributed/near/GridNearLockRequest.java   |  46 +--
 .../distributed/near/GridNearLockResponse.java  |  30 +-
 .../near/GridNearSingleGetRequest.java          |  46 +--
 .../near/GridNearSingleGetResponse.java         |  26 +-
 .../near/GridNearTxEnlistRequest.java           |  70 ++---
 .../near/GridNearTxEnlistResponse.java          |  34 +--
 .../near/GridNearTxFinishRequest.java           |  10 +-
 .../near/GridNearTxFinishResponse.java          |  14 +-
 .../near/GridNearTxPrepareRequest.java          |  34 +--
 .../near/GridNearTxPrepareResponse.java         |  70 ++---
 .../near/GridNearTxQueryEnlistRequest.java      |  78 ++---
 .../near/GridNearTxQueryEnlistResponse.java     |  44 +--
 .../GridNearTxQueryResultsEnlistRequest.java    |  62 ++--
 .../GridNearTxQueryResultsEnlistResponse.java   |  12 +-
 .../distributed/near/GridNearUnlockRequest.java |   2 +-
 .../mvcc/msg/MvccAckRequestTxAndQueryId.java    |   1 +
 .../cache/mvcc/msg/MvccSnapshotResponse.java    |   2 +
 .../cache/mvcc/msg/MvccTxSnapshotRequest.java   |   2 +
 .../PartitionCountersNeighborcastRequest.java   |  10 +-
 .../PartitionCountersNeighborcastResponse.java  |   6 +-
 .../cache/query/GridCacheQueryRequest.java      | 112 +++----
 .../cache/query/GridCacheQueryResponse.java     |  26 +-
 .../CacheContinuousQueryBatchAck.java           |  16 +-
 .../continuous/CacheContinuousQueryEntry.java   |   4 +-
 .../cache/transactions/IgniteTxHandler.java     |   7 +-
 .../cache/transactions/IgniteTxKey.java         |   2 +-
 .../cache/transactions/TxLocksRequest.java      |  10 +-
 .../cache/transactions/TxLocksResponse.java     |  18 +-
 .../version/GridCacheRawVersionedEntry.java     |   4 +-
 .../continuous/GridContinuousMessage.java       |   2 +-
 .../datastreamer/DataStreamerRequest.java       |   4 +-
 .../datastreamer/DataStreamerResponse.java      |   2 +-
 .../shuffle/HadoopDirectShuffleMessage.java     |   4 +-
 .../hadoop/shuffle/HadoopShuffleMessage.java    |   2 +
 .../processors/igfs/IgfsAckMessage.java         |   2 +-
 .../internal/processors/igfs/IgfsBlockKey.java  |   2 +-
 .../processors/igfs/IgfsBlocksMessage.java      |   2 +-
 .../igfs/IgfsCommunicationMessage.java          |   2 +-
 .../processors/igfs/IgfsDeleteMessage.java      |   2 +-
 .../igfs/IgfsFragmentizerRequest.java           |   2 +-
 .../igfs/IgfsFragmentizerResponse.java          |   2 +-
 .../processors/igfs/IgfsSyncMessage.java        |   2 +-
 .../messages/GridQueryNextPageResponse.java     |   4 +-
 .../message/SchemaOperationStatusMessage.java   |   2 +
 .../handlers/task/GridTaskResultRequest.java    |   2 +-
 .../handlers/task/GridTaskResultResponse.java   |   2 +-
 .../ignite/internal/util/GridByteArrayList.java |   2 +-
 .../ignite/internal/util/GridIntList.java       |   2 +-
 .../ignite/internal/util/GridLongList.java      |   2 +
 .../MessageCollectionItemType.java              |   5 +-
 .../extensions/communication/MessageReader.java |   9 +
 .../extensions/communication/MessageWriter.java |  10 +
 .../jobstealing/JobStealingRequest.java         |   2 +-
 .../tcp/messages/HandshakeMessage.java          |   2 +
 .../tcp/messages/HandshakeMessage2.java         |   2 +
 .../tcp/messages/NodeIdMessage.java             |   2 +
 .../messages/RecoveryLastReceivedMessage.java   |   2 +
 .../cache/CacheNoAffinityExchangeTest.java      | 218 ++++++++++++++
 .../IgniteClusterActivateDeactivateTest.java    |   3 +-
 .../distributed/CacheExchangeMergeTest.java     |   2 +
 .../testsuites/IgniteCacheTestSuite6.java       |   3 +
 .../processors/cache/query/QueryTable.java      |   2 +
 .../h2/twostep/msg/GridH2IndexRangeRequest.java |   2 +
 .../twostep/msg/GridH2IndexRangeResponse.java   |   2 +
 .../h2/twostep/msg/GridH2QueryRequest.java      |   6 +-
 .../h2/twostep/msg/GridH2ValueMessage.java      |   2 +
 173 files changed, 2200 insertions(+), 1272 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
index 615ffca..ca2dcdc 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
@@ -43,12 +43,7 @@ import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.IgniteCodeGeneratingFail;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockRequest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetRequest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
-import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccRecoveryFinishedMessage;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -121,6 +116,7 @@ public class MessageCodeGenerator {
         TYPES.put(BitSet.class, MessageCollectionItemType.BIT_SET);
         TYPES.put(UUID.class, MessageCollectionItemType.UUID);
         TYPES.put(IgniteUuid.class, MessageCollectionItemType.IGNITE_UUID);
+        TYPES.put(AffinityTopologyVersion.class, MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION);
     }
 
     /**
@@ -174,13 +170,7 @@ public class MessageCodeGenerator {
 
 //        gen.generateAll(true);
 
-        gen.generateAndWrite(GridNearGetRequest.class);
-        gen.generateAndWrite(GridNearSingleGetRequest.class);
-        gen.generateAndWrite(GridNearTxPrepareRequest.class);
-        gen.generateAndWrite(GridDhtTxPrepareRequest.class);
-        gen.generateAndWrite(GridDhtLockRequest.class);
-
-//        gen.generateAndWrite(GridNearAtomicUpdateRequest.class);
+//        gen.generateAndWrite(GridCacheMessage.class);
 
 //        gen.generateAndWrite(GridMessageCollection.class);
 //        gen.generateAndWrite(DataStreamerEntry.class);
@@ -672,6 +662,8 @@ public class MessageCodeGenerator {
             returnFalseIfFailed(write, "writer.writeUuid", field, getExpr);
         else if (type == IgniteUuid.class)
             returnFalseIfFailed(write, "writer.writeIgniteUuid", field, getExpr);
+        else if (type == AffinityTopologyVersion.class)
+            returnFalseIfFailed(write, "writer.writeAffinityTopologyVersion", field, getExpr);
         else if (type.isEnum()) {
             String arg = getExpr + " != null ? (byte)" + getExpr + ".ordinal() : -1";
 
@@ -754,6 +746,8 @@ public class MessageCodeGenerator {
             returnFalseIfReadFailed(name, "reader.readUuid", setExpr, field);
         else if (type == IgniteUuid.class)
             returnFalseIfReadFailed(name, "reader.readIgniteUuid", setExpr, field);
+        else if (type == AffinityTopologyVersion.class)
+            returnFalseIfReadFailed(name, "reader.readAffinityTopologyVersion", setExpr, field);
         else if (type.isEnum()) {
             String loc = name + "Ord";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java
index aaa69ea..ac3a873 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java
@@ -201,4 +201,4 @@ public class GridJobCancelRequest implements Message {
     @Override public String toString() {
         return S.toString(GridJobCancelRequest.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java
index 4357d1d..ebfeb01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java
@@ -664,7 +664,7 @@ public class GridJobExecuteRequest implements ExecutorAwareMessage {
                 writer.incrementState();
 
             case 24:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeAffinityTopologyVersion("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -885,7 +885,7 @@ public class GridJobExecuteRequest implements ExecutorAwareMessage {
                 reader.incrementState();
 
             case 24:
-                topVer = reader.readMessage("topVer");
+                topVer = reader.readAffinityTopologyVersion("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java
index 312435e..f052edf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java
@@ -282,7 +282,7 @@ public class GridJobExecuteResponse implements Message {
                 writer.incrementState();
 
             case 6:
-                if (!writer.writeMessage("retry", retry))
+                if (!writer.writeAffinityTopologyVersion("retry", retry))
                     return false;
 
                 writer.incrementState();
@@ -355,7 +355,7 @@ public class GridJobExecuteResponse implements Message {
                 reader.incrementState();
 
             case 6:
-                retry = reader.readMessage("retry");
+                retry = reader.readAffinityTopologyVersion("retry");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java
index 8a11cef..d743a35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java
@@ -161,4 +161,4 @@ public class GridJobSiblingsRequest implements Message {
     @Override public String toString() {
         return S.toString(GridJobSiblingsRequest.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java
index 3911446..dc59ab5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java
@@ -141,4 +141,4 @@ public class GridJobSiblingsResponse implements Message {
     @Override public String toString() {
         return S.toString(GridJobSiblingsResponse.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java
index 273d0a7..71c318b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java
@@ -124,4 +124,4 @@ public class GridTaskCancelRequest implements Message {
     @Override public String toString() {
         return S.toString(GridTaskCancelRequest.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java
index dbac893..576392e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java
@@ -189,4 +189,4 @@ public class GridTaskSessionRequest implements Message {
     @Override public String toString() {
         return S.toString(GridTaskSessionRequest.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java
index 47d7877..b820801 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java
@@ -27,6 +27,9 @@ import org.apache.ignite.internal.direct.state.DirectMessageStateItem;
 import org.apache.ignite.internal.direct.stream.DirectByteBufferStream;
 import org.apache.ignite.internal.direct.stream.v1.DirectByteBufferStreamImplV1;
 import org.apache.ignite.internal.direct.stream.v2.DirectByteBufferStreamImplV2;
+import org.apache.ignite.internal.direct.stream.v3.DirectByteBufferStreamImplV3;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteOutClosure;
 import org.apache.ignite.lang.IgniteUuid;
@@ -41,8 +44,13 @@ import org.jetbrains.annotations.Nullable;
  */
 public class DirectMessageReader implements MessageReader {
     /** State. */
+    @GridToStringInclude
     private final DirectMessageState<StateItem> state;
 
+    /** Protocol version. */
+    @GridToStringInclude
+    private final byte protoVer;
+
     /** Whether last field was fully read. */
     private boolean lastRead;
 
@@ -56,6 +64,8 @@ public class DirectMessageReader implements MessageReader {
                 return new StateItem(msgFactory, protoVer);
             }
         });
+
+        this.protoVer = protoVer;
     }
 
     /** {@inheritDoc} */
@@ -305,6 +315,21 @@ public class DirectMessageReader implements MessageReader {
     }
 
     /** {@inheritDoc} */
+    @Override public AffinityTopologyVersion readAffinityTopologyVersion(String name) {
+        if (protoVer >= 3) {
+            DirectByteBufferStream stream = state.item().stream;
+
+            AffinityTopologyVersion val = stream.readAffinityTopologyVersion();
+
+            lastRead = stream.lastFinished();
+
+            return val;
+        }
+
+        return readMessage(name);
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public <T extends Message> T readMessage(String name) {
         DirectByteBufferStream stream = state.item().stream;
 
@@ -409,6 +434,11 @@ public class DirectMessageReader implements MessageReader {
 
                     break;
 
+                case 3:
+                    stream = new DirectByteBufferStreamImplV3(msgFactory);
+
+                    break;
+
                 default:
                     throw new IllegalStateException("Invalid protocol version: " + protoVer);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageWriter.java
index 51cea17..bb88ffc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageWriter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageWriter.java
@@ -27,6 +27,8 @@ import org.apache.ignite.internal.direct.state.DirectMessageStateItem;
 import org.apache.ignite.internal.direct.stream.DirectByteBufferStream;
 import org.apache.ignite.internal.direct.stream.v1.DirectByteBufferStreamImplV1;
 import org.apache.ignite.internal.direct.stream.v2.DirectByteBufferStreamImplV2;
+import org.apache.ignite.internal.direct.stream.v3.DirectByteBufferStreamImplV3;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteOutClosure;
@@ -44,6 +46,10 @@ public class DirectMessageWriter implements MessageWriter {
     @GridToStringInclude
     private final DirectMessageState<StateItem> state;
 
+    /** Protocol version. */
+    @GridToStringInclude
+    private final byte protoVer;
+
     /**
      * @param protoVer Protocol version.
      */
@@ -53,6 +59,8 @@ public class DirectMessageWriter implements MessageWriter {
                 return new StateItem(protoVer);
             }
         });
+
+        this.protoVer = protoVer;
     }
 
     /** {@inheritDoc} */
@@ -273,6 +281,19 @@ public class DirectMessageWriter implements MessageWriter {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean writeAffinityTopologyVersion(String name, AffinityTopologyVersion val) {
+        if (protoVer >= 3) {
+            DirectByteBufferStream stream = state.item().stream;
+
+            stream.writeAffinityTopologyVersion(val);
+
+            return stream.lastFinished();
+        }
+
+        return writeMessage(name, val);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeMessage(String name, @Nullable Message msg) {
         DirectByteBufferStream stream = state.item().stream;
 
@@ -376,6 +397,11 @@ public class DirectMessageWriter implements MessageWriter {
 
                     break;
 
+                case 3:
+                    stream = new DirectByteBufferStreamImplV3(null);
+
+                    break;
+
                 default:
                     throw new IllegalStateException("Invalid protocol version: " + protoVer);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/DirectByteBufferStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/DirectByteBufferStream.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/DirectByteBufferStream.java
index 204e6b0..ae5502e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/DirectByteBufferStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/DirectByteBufferStream.java
@@ -22,6 +22,7 @@ import java.util.BitSet;
 import java.util.Collection;
 import java.util.Map;
 import java.util.UUID;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
@@ -161,6 +162,11 @@ public interface DirectByteBufferStream {
     public void writeIgniteUuid(IgniteUuid val);
 
     /**
+     * @param val Value.
+     */
+    public void writeAffinityTopologyVersion(AffinityTopologyVersion val);
+
+    /**
      * @param msg Message.
      * @param writer Writer.
      */
@@ -290,6 +296,11 @@ public interface DirectByteBufferStream {
     public IgniteUuid readIgniteUuid();
 
     /**
+     * @return Value.
+     */
+    public AffinityTopologyVersion readAffinityTopologyVersion();
+
+    /**
      * @param reader Reader.
      * @return Message.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v1/DirectByteBufferStreamImplV1.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v1/DirectByteBufferStreamImplV1.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v1/DirectByteBufferStreamImplV1.java
index c78c479..118e1f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v1/DirectByteBufferStreamImplV1.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v1/DirectByteBufferStreamImplV1.java
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.UUID;
 import org.apache.ignite.internal.direct.stream.DirectByteBufferStream;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -495,6 +496,11 @@ public class DirectByteBufferStreamImplV1 implements DirectByteBufferStream {
     }
 
     /** {@inheritDoc} */
+    @Override public void writeAffinityTopologyVersion(AffinityTopologyVersion val) {
+        throw new UnsupportedOperationException("Not implemented");
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeMessage(Message msg, MessageWriter writer) {
         if (msg != null) {
             if (buf.hasRemaining()) {
@@ -812,6 +818,11 @@ public class DirectByteBufferStreamImplV1 implements DirectByteBufferStream {
     }
 
     /** {@inheritDoc} */
+    @Override public AffinityTopologyVersion readAffinityTopologyVersion() {
+        throw new UnsupportedOperationException("Not implemented");
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <T extends Message> T readMessage(MessageReader reader) {
         if (!msgTypeDone) {
@@ -1212,6 +1223,7 @@ public class DirectByteBufferStreamImplV1 implements DirectByteBufferStream {
 
                 break;
 
+            case AFFINITY_TOPOLOGY_VERSION:
             case MSG:
                 try {
                     if (val != null)
@@ -1298,6 +1310,7 @@ public class DirectByteBufferStreamImplV1 implements DirectByteBufferStream {
             case IGNITE_UUID:
                 return readIgniteUuid();
 
+            case AFFINITY_TOPOLOGY_VERSION:
             case MSG:
                 return readMessage(reader);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java
index e338bc0..fd93cfb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java
@@ -29,6 +29,7 @@ import java.util.RandomAccess;
 import java.util.UUID;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.direct.stream.DirectByteBufferStream;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -298,7 +299,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     private long uuidLocId;
 
     /** */
-    private boolean lastFinished;
+    protected boolean lastFinished;
 
     /**
      * @param msgFactory Message factory.
@@ -658,6 +659,11 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     }
 
     /** {@inheritDoc} */
+    @Override public void writeAffinityTopologyVersion(AffinityTopologyVersion val) {
+        throw new UnsupportedOperationException("Not implemented");
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeMessage(Message msg, MessageWriter writer) {
         if (msg != null) {
             if (buf.hasRemaining()) {
@@ -1153,6 +1159,11 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     }
 
     /** {@inheritDoc} */
+    @Override public AffinityTopologyVersion readAffinityTopologyVersion() {
+        throw new UnsupportedOperationException("Not implemented");
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <T extends Message> T readMessage(MessageReader reader) {
         if (!msgTypeDone) {
@@ -1587,7 +1598,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
      * @param val Value.
      * @param writer Writer.
      */
-    private void write(MessageCollectionItemType type, Object val, MessageWriter writer) {
+    protected void write(MessageCollectionItemType type, Object val, MessageWriter writer) {
         switch (type) {
             case BYTE:
                 writeByte((Byte)val);
@@ -1689,6 +1700,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
 
                 break;
 
+            case AFFINITY_TOPOLOGY_VERSION:
             case MSG:
                 try {
                     if (val != null)
@@ -1713,7 +1725,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
      * @param reader Reader.
      * @return Value.
      */
-    private Object read(MessageCollectionItemType type, MessageReader reader) {
+    protected Object read(MessageCollectionItemType type, MessageReader reader) {
         switch (type) {
             case BYTE:
                 return readByte();
@@ -1775,6 +1787,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
             case IGNITE_UUID:
                 return readIgniteUuid();
 
+            case AFFINITY_TOPOLOGY_VERSION:
             case MSG:
                 return readMessage(reader);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v3/DirectByteBufferStreamImplV3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v3/DirectByteBufferStreamImplV3.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v3/DirectByteBufferStreamImplV3.java
new file mode 100644
index 0000000..89043eb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v3/DirectByteBufferStreamImplV3.java
@@ -0,0 +1,298 @@
+/*
+ * 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.direct.stream.v3;
+
+import java.util.BitSet;
+import java.util.UUID;
+import org.apache.ignite.internal.direct.stream.v2.DirectByteBufferStreamImplV2;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageFactory;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class DirectByteBufferStreamImplV3 extends DirectByteBufferStreamImplV2 {
+    /** */
+    private byte topVerState;
+
+    /** */
+    private long topVerMajor;
+
+    /** */
+    private int topVerMinor;
+
+    /**
+     * @param msgFactory Message factory.
+     */
+    public DirectByteBufferStreamImplV3(MessageFactory msgFactory) {
+        super(msgFactory);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeAffinityTopologyVersion(AffinityTopologyVersion val) {
+        if (val != null) {
+            switch (topVerState) {
+                case 0:
+                    writeInt(val.minorTopologyVersion());
+
+                    if (!lastFinished)
+                        return;
+
+                    topVerState++;
+
+                case 1:
+                    writeLong(val.topologyVersion());
+
+                    if (!lastFinished)
+                        return;
+
+                    topVerState = 0;
+            }
+        }
+        else
+            writeInt(-1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public AffinityTopologyVersion readAffinityTopologyVersion() {
+        switch (topVerState) {
+            case 0:
+                topVerMinor = readInt();
+
+                if (!lastFinished || topVerMinor == -1)
+                    return null;
+
+                topVerState++;
+
+            case 1:
+                topVerMajor = readLong();
+
+                if (!lastFinished)
+                    return null;
+
+                topVerState = 0;
+        }
+
+        return new AffinityTopologyVersion(topVerMajor, topVerMinor);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void write(MessageCollectionItemType type, Object val, MessageWriter writer) {
+        switch (type) {
+            case BYTE:
+                writeByte((Byte)val);
+
+                break;
+
+            case SHORT:
+                writeShort((Short)val);
+
+                break;
+
+            case INT:
+                writeInt((Integer)val);
+
+                break;
+
+            case LONG:
+                writeLong((Long)val);
+
+                break;
+
+            case FLOAT:
+                writeFloat((Float)val);
+
+                break;
+
+            case DOUBLE:
+                writeDouble((Double)val);
+
+                break;
+
+            case CHAR:
+                writeChar((Character)val);
+
+                break;
+
+            case BOOLEAN:
+                writeBoolean((Boolean)val);
+
+                break;
+
+            case BYTE_ARR:
+                writeByteArray((byte[])val);
+
+                break;
+
+            case SHORT_ARR:
+                writeShortArray((short[])val);
+
+                break;
+
+            case INT_ARR:
+                writeIntArray((int[])val);
+
+                break;
+
+            case LONG_ARR:
+                writeLongArray((long[])val);
+
+                break;
+
+            case FLOAT_ARR:
+                writeFloatArray((float[])val);
+
+                break;
+
+            case DOUBLE_ARR:
+                writeDoubleArray((double[])val);
+
+                break;
+
+            case CHAR_ARR:
+                writeCharArray((char[])val);
+
+                break;
+
+            case BOOLEAN_ARR:
+                writeBooleanArray((boolean[])val);
+
+                break;
+
+            case STRING:
+                writeString((String)val);
+
+                break;
+
+            case BIT_SET:
+                writeBitSet((BitSet)val);
+
+                break;
+
+            case UUID:
+                writeUuid((UUID)val);
+
+                break;
+
+            case IGNITE_UUID:
+                writeIgniteUuid((IgniteUuid)val);
+
+                break;
+
+            case AFFINITY_TOPOLOGY_VERSION:
+                writeAffinityTopologyVersion((AffinityTopologyVersion)val);
+
+                break;
+            case MSG:
+                try {
+                    if (val != null)
+                        writer.beforeInnerMessageWrite();
+
+                    writeMessage((Message)val, writer);
+                }
+                finally {
+                    if (val != null)
+                        writer.afterInnerMessageWrite(lastFinished);
+                }
+
+                break;
+
+            default:
+                throw new IllegalArgumentException("Unknown type: " + type);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object read(MessageCollectionItemType type, MessageReader reader) {
+        switch (type) {
+            case BYTE:
+                return readByte();
+
+            case SHORT:
+                return readShort();
+
+            case INT:
+                return readInt();
+
+            case LONG:
+                return readLong();
+
+            case FLOAT:
+                return readFloat();
+
+            case DOUBLE:
+                return readDouble();
+
+            case CHAR:
+                return readChar();
+
+            case BOOLEAN:
+                return readBoolean();
+
+            case BYTE_ARR:
+                return readByteArray();
+
+            case SHORT_ARR:
+                return readShortArray();
+
+            case INT_ARR:
+                return readIntArray();
+
+            case LONG_ARR:
+                return readLongArray();
+
+            case FLOAT_ARR:
+                return readFloatArray();
+
+            case DOUBLE_ARR:
+                return readDoubleArray();
+
+            case CHAR_ARR:
+                return readCharArray();
+
+            case BOOLEAN_ARR:
+                return readBooleanArray();
+
+            case STRING:
+                return readString();
+
+            case BIT_SET:
+                return readBitSet();
+
+            case UUID:
+                return readUuid();
+
+            case IGNITE_UUID:
+                return readIgniteUuid();
+
+            case AFFINITY_TOPOLOGY_VERSION:
+                return readAffinityTopologyVersion();
+
+            case MSG:
+                return readMessage(reader);
+
+            default:
+                throw new IllegalArgumentException("Unknown type: " + type);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java
index 8b21ff2..4b25e0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java
@@ -177,4 +177,4 @@ public class GridCheckpointRequest implements Message {
     @Override public String toString() {
         return S.toString(GridCheckpointRequest.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index ecf99f1..491baa2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -135,7 +135,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     public static final String DIRECT_PROTO_VER_ATTR = "comm.direct.proto.ver";
 
     /** Direct protocol version. */
-    public static final byte DIRECT_PROTO_VER = 2;
+    public static final byte DIRECT_PROTO_VER = 3;
 
     /** Current IO policy. */
     private static final ThreadLocal<Byte> CUR_PLC = new ThreadLocal<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java
index 332a9de..408fad7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java
@@ -358,4 +358,4 @@ public class GridIoUserMessage implements Message {
     @Override public String toString() {
         return S.toString(GridIoUserMessage.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
index 0a8b2b7..a6a2469 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.managers.communication;
 import java.nio.ByteBuffer;
 import java.util.UUID;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -28,6 +29,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 /**
  *
  */
+@IgniteCodeGeneratingFail
 public class IgniteIoTestMessage implements Message {
     /** */
     private static byte FLAG_PROC_FROM_NIO = 1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
index 7f58ce3..72f5ec6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
@@ -277,4 +277,4 @@ public class GridDeploymentInfoBean implements Message, GridDeploymentInfo, Exte
     @Override public String toString() {
         return S.toString(GridDeploymentInfoBean.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java
index 729cf4c..708c648 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java
@@ -278,4 +278,4 @@ public class GridDeploymentRequest implements Message {
     @Override public String toString() {
         return S.toString(GridDeploymentRequest.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java
index d1b0384..591957d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java
@@ -197,4 +197,4 @@ public class GridDeploymentResponse implements Message {
     @Override public String toString() {
         return S.toString(GridDeploymentResponse.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java
index 515500b..fd5326c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java
@@ -445,4 +445,4 @@ public class GridEventStorageMessage implements Message {
     @Override public String toString() {
         return S.toString(GridEventStorageMessage.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
index 44b2753..333841d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
@@ -219,4 +219,4 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
     @Override public String toString() {
         return S.toString(AffinityTopologyVersion.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 902de47..4d5cf27 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -689,19 +689,41 @@ public class GridAffinityAssignmentCache {
      * @return Cached affinity.
      */
     public AffinityAssignment cachedAffinity(AffinityTopologyVersion topVer) {
+        AffinityTopologyVersion lastAffChangeTopVer =
+            ctx.cache().context().exchange().lastAffinityChangedTopologyVersion(topVer);
+
+        return cachedAffinity(topVer, lastAffChangeTopVer);
+    }
+
+    /**
+     * Get cached affinity for specified topology version.
+     *
+     * @param topVer Topology version.
+     * @return Cached affinity.
+     */
+    public AffinityAssignment cachedAffinity(AffinityTopologyVersion topVer, AffinityTopologyVersion lastAffChangeTopVer) {
         if (topVer.equals(AffinityTopologyVersion.NONE))
-            topVer = lastVersion();
-        else
-            awaitTopologyVersion(topVer);
+            topVer = lastAffChangeTopVer = lastVersion();
+        else {
+            if (lastAffChangeTopVer.equals(AffinityTopologyVersion.NONE))
+                lastAffChangeTopVer = topVer;
+
+            awaitTopologyVersion(lastAffChangeTopVer);
+        }
 
         assert topVer.topologyVersion() >= 0 : topVer;
 
         AffinityAssignment cache = head.get();
 
-        if (!cache.topologyVersion().equals(topVer)) {
-            cache = affCache.get(topVer);
+        if (!(cache.topologyVersion().compareTo(lastAffChangeTopVer) >= 0 &&
+            cache.topologyVersion().compareTo(topVer) <= 0)) {
 
-            if (cache == null) {
+            Map.Entry<AffinityTopologyVersion, HistoryAffinityAssignment> e = affCache.ceilingEntry(lastAffChangeTopVer);
+
+            if (e != null)
+                cache = e.getValue();
+
+            if (cache == null || cache.topologyVersion().compareTo(topVer) > 0) {
                 throw new IllegalStateException("Getting affinity for topology version earlier than affinity is " +
                     "calculated [locNode=" + ctx.discovery().localNode() +
                     ", grp=" + cacheOrGrpName +
@@ -712,7 +734,8 @@ public class GridAffinityAssignmentCache {
             }
         }
 
-        assert cache.topologyVersion().equals(topVer) : "Invalid cached affinity: " + cache;
+        assert cache.topologyVersion().compareTo(lastAffChangeTopVer) >= 0 &&
+            cache.topologyVersion().compareTo(topVer) <= 0 : "Invalid cached affinity: [cache=" + cache + ", topVer=" + topVer + ", lastAffChangedTopVer=" + lastAffChangeTopVer + "]";
 
         return cache;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAuthenticateResponseMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAuthenticateResponseMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAuthenticateResponseMessage.java
index d86b1ad..e3dee3c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAuthenticateResponseMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAuthenticateResponseMessage.java
@@ -98,7 +98,6 @@ public class UserAuthenticateResponseMessage implements Message {
 
                 writer.incrementState();
 
-
         }
 
         return true;
@@ -127,6 +126,7 @@ public class UserAuthenticateResponseMessage implements Message {
                     return false;
 
                 reader.incrementState();
+
         }
 
         return reader.afterMessageRead(UserAuthenticateResponseMessage.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
index 614d7c0..968afd5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
@@ -128,4 +128,4 @@ public class CacheEntryInfoCollection implements Message {
     @Override public byte fieldsCount() {
         return 1;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicate.java
index 61cbb9e..36312a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicate.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicate.java
@@ -42,4 +42,4 @@ public interface CacheEntryPredicate extends IgnitePredicate<GridCacheEntryEx>,
      * @param locked Entry locked
      */
     public void entryLocked(boolean locked);
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateAdapter.java
index fff7959..6eb9a51 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateAdapter.java
@@ -96,4 +96,4 @@ public abstract class CacheEntryPredicateAdapter implements CacheEntryPredicate
     @Override public void onAckReceived() {
         // No-op.
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
index 76806a4..73a98b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
@@ -140,4 +140,4 @@ public class CacheEntryPredicateContainsValue extends CacheEntryPredicateAdapter
     @Override public String toString() {
         return S.toString(CacheEntryPredicateContainsValue.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateHasValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateHasValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateHasValue.java
index cac0435..210cc70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateHasValue.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateHasValue.java
@@ -28,4 +28,4 @@ public class CacheEntryPredicateHasValue extends CacheEntryPredicateAdapter {
     @Override public boolean apply(GridCacheEntryEx e) {
         return peekVisibleValue(e) != null;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateNoValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateNoValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateNoValue.java
index 2790170..4c8917f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateNoValue.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateNoValue.java
@@ -28,4 +28,4 @@ public class CacheEntryPredicateNoValue extends CacheEntryPredicateAdapter {
     @Override public boolean apply(GridCacheEntryEx e) {
         return peekVisibleValue(e) == null;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java
index 9057e41..2574336 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java
@@ -156,4 +156,4 @@ public class CacheEntrySerializablePredicate implements CacheEntryPredicate {
     @Override public byte fieldsCount() {
         return 1;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionEntry.java
index 2717b1e..96b85df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionEntry.java
@@ -185,4 +185,4 @@ public class CacheEvictionEntry implements Message {
     @Override public byte fieldsCount() {
         return 3;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java
index 3b463af..3f88033 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java
@@ -267,4 +267,4 @@ public class CacheInvokeDirectResult implements Message {
     @Override public String toString() {
         return S.toString(CacheInvokeDirectResult.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
index 3bc2a6d..f9f384a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
@@ -118,4 +118,4 @@ public interface CacheObject extends Message {
      * @throws IgniteCheckedException If failed.
      */
     public void prepareMarshal(CacheObjectValueContext ctx) throws IgniteCheckedException;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
index 67ee410..c6d9002 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
@@ -247,4 +247,4 @@ public abstract class CacheObjectAdapter implements CacheObject, Externalizable
 
         return true;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
index 57a70f8..de5a919 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
@@ -191,4 +191,4 @@ public class CacheObjectByteArrayImpl implements CacheObject, Externalizable {
     @Override public String toString() {
         return "CacheObjectByteArrayImpl [arrLen=" + (val != null ? val.length : 0) + ']';
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectImpl.java
index 2124a97..b29c19e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectImpl.java
@@ -152,4 +152,4 @@ public class CacheObjectImpl extends CacheObjectAdapter {
     @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
         return this;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
index cf4344d..f811f6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
@@ -233,15 +233,25 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @return Affinity assignment.
      */
     public AffinityAssignment assignment(AffinityTopologyVersion topVer) {
+        return assignment(topVer, cctx.shared().exchange().lastAffinityChangedTopologyVersion(topVer));
+    }
+
+    /**
+     * Get affinity assignment for the given topology version.
+     *
+     * @param topVer Topology version.
+     * @return Affinity assignment.
+     */
+    public AffinityAssignment assignment(AffinityTopologyVersion topVer, AffinityTopologyVersion lastAffChangedTopVer) {
         if (cctx.isLocal())
-            topVer = LOC_CACHE_TOP_VER;
+            topVer = lastAffChangedTopVer = LOC_CACHE_TOP_VER;
 
         GridAffinityAssignmentCache aff0 = aff;
 
         if (aff0 == null)
             throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name());
 
-        return aff0.cachedAffinity(topVer);
+        return aff0.cachedAffinity(topVer, lastAffChangedTopVer);
     }
 
     public MvccCoordinator mvccCoordinator(AffinityTopologyVersion topVer) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGroupIdMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGroupIdMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGroupIdMessage.java
index 09c143b..bfdce35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGroupIdMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGroupIdMessage.java
@@ -50,7 +50,7 @@ public abstract class GridCacheGroupIdMessage extends GridCacheMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 3;
+        return 4;
     }
 
     /** {@inheritDoc} */
@@ -68,7 +68,7 @@ public abstract class GridCacheGroupIdMessage extends GridCacheMessage {
         }
 
         switch (writer.state()) {
-            case 2:
+            case 3:
                 if (!writer.writeInt("grpId", grpId))
                     return false;
 
@@ -90,7 +90,7 @@ public abstract class GridCacheGroupIdMessage extends GridCacheMessage {
             return false;
 
         switch (reader.state()) {
-            case 2:
+            case 3:
                 grpId = reader.readInt("grpId");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIdMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIdMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIdMessage.java
index 6c20bdd..e094439 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIdMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIdMessage.java
@@ -52,7 +52,7 @@ public abstract class GridCacheIdMessage extends GridCacheMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 3;
+        return 4;
     }
 
     /** {@inheritDoc} */
@@ -70,7 +70,7 @@ public abstract class GridCacheIdMessage extends GridCacheMessage {
         }
 
         switch (writer.state()) {
-            case 2:
+            case 3:
                 if (!writer.writeInt("cacheId", cacheId))
                     return false;
 
@@ -92,7 +92,7 @@ public abstract class GridCacheIdMessage extends GridCacheMessage {
             return false;
 
         switch (reader.state()) {
-            case 2:
+            case 3:
                 cacheId = reader.readInt("cacheId");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 4180116..5b27f4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -164,6 +164,11 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             final GridCacheMessage cacheMsg = (GridCacheMessage)msg;
 
+            AffinityTopologyVersion rmtAffVer = cacheMsg.topologyVersion();
+            AffinityTopologyVersion lastAffChangedVer = cacheMsg.lastAffinityChangedTopologyVersion();
+
+            cctx.exchange().lastAffinityChangedTopologyVersion(rmtAffVer, lastAffChangedVer);
+
             IgniteInternalFuture<?> fut = null;
 
             if (cacheMsg.partitionExchangeMessage()) {
@@ -221,9 +226,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             }
             else {
                 AffinityTopologyVersion locAffVer = cctx.exchange().readyAffinityVersion();
-                AffinityTopologyVersion rmtAffVer = cacheMsg.topologyVersion();
 
-                if (locAffVer.compareTo(rmtAffVer) < 0) {
+                if (locAffVer.compareTo(lastAffChangedVer) < 0) {
                     IgniteLogger log = cacheMsg.messageLogger(cctx);
 
                     if (log.isDebugEnabled()) {
@@ -233,12 +237,13 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                         msg0.append(", locTopVer=").append(locAffVer).
                             append(", rmtTopVer=").append(rmtAffVer).
+                            append(", lastAffChangedVer=").append(lastAffChangedVer).
                             append(']');
 
                         log.debug(msg0.toString());
                     }
 
-                    fut = cctx.exchange().affinityReadyFuture(rmtAffVer);
+                    fut = cctx.exchange().affinityReadyFuture(lastAffChangedVer);
                 }
             }
 
@@ -1152,6 +1157,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
     public void send(ClusterNode node, GridCacheMessage msg, byte plc) throws IgniteCheckedException {
         assert !node.isLocal() : node;
 
+        msg.lastAffinityChangedTopologyVersion(cctx.exchange().lastAffinityChangedTopologyVersion(msg.topologyVersion()));
+
         if (!onSend(msg, node.id()))
             return;
 
@@ -1219,6 +1226,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         if (!onSend(msg, node.id()))
             return;
 
+        msg.lastAffinityChangedTopologyVersion(cctx.exchange().lastAffinityChangedTopologyVersion(msg.topologyVersion()));
+
         int cnt = 0;
 
         while (cnt <= retryCnt) {
@@ -1275,6 +1284,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         if (!onSend(msg, null))
             return;
 
+        msg.lastAffinityChangedTopologyVersion(cctx.exchange().lastAffinityChangedTopologyVersion(msg.topologyVersion()));
+
         try {
             cctx.gridIO().sendToGridTopic(node, TOPIC_CACHE, msg, plc);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a63a81a5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index 11916e9..f6c2e2e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -69,6 +69,10 @@ public abstract class GridCacheMessage implements Message {
     private GridDeploymentInfoBean depInfo;
 
     /** */
+    @GridToStringInclude
+    private @Nullable AffinityTopologyVersion lastAffChangedTopVer;
+
+    /** */
     @GridDirectTransient
     protected boolean addDepInfo;
 
@@ -185,6 +189,27 @@ public abstract class GridCacheMessage implements Message {
     }
 
     /**
+     * Returns the earliest affinity topology version for which this message is valid.
+     *
+     * @return Last affinity topology version when affinity was modified.
+     */
+    public AffinityTopologyVersion lastAffinityChangedTopologyVersion() {
+        if (lastAffChangedTopVer == null || lastAffChangedTopVer.topologyVersion() <= 0)
+            return topologyVersion();
+
+        return lastAffChangedTopVer;
+    }
+
+    /**
+     * Sets the earliest affinity topology version for which this message is valid.
+     *
+     * @param topVer Last affinity topology version when affinity was modified.
+     */
+    public void lastAffinityChangedTopologyVersion(AffinityTopologyVersion topVer) {
+        lastAffChangedTopVer = topVer;
+    }
+
+    /**
      *  Deployment enabled flag indicates whether deployment info has to be added to this message.
      *
      * @return {@code true} or if deployment info must be added to the the message, {@code false} otherwise.
@@ -637,7 +662,7 @@ public abstract class GridCacheMessage implements Message {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 2;
+        return 3;
     }
 
     /** {@inheritDoc} */
@@ -659,6 +684,12 @@ public abstract class GridCacheMessage implements Message {
                 writer.incrementState();
 
             case 1:
+                if (!writer.writeAffinityTopologyVersion("lastAffChangedTopVer", lastAffChangedTopVer))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
                 if (!writer.writeLong("msgId", msgId))
                     return false;
 
@@ -686,6 +717,14 @@ public abstract class GridCacheMessage implements Message {
                 reader.incrementState();
 
             case 1:
+                lastAffChangedTopVer = reader.readAffinityTopologyVersion("lastAffChangedTopVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
                 msgId = reader.readLong("msgId");
 
                 if (!reader.isLastRead())


[12/50] [abbrv] ignite git commit: IGNITE-10294: SQL: Set proper subjectId in SqlFielsQuery event. This closes #5408.

Posted by sb...@apache.org.
IGNITE-10294: SQL: Set proper subjectId in SqlFielsQuery event. This closes #5408.


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

Branch: refs/heads/ignite-10044
Commit: 16dc88fc228eb15669922e3878badc8ffc16dda5
Parents: 7076f42
Author: devozerov <pp...@gmail.com>
Authored: Sat Nov 17 13:48:24 2018 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Sat Nov 17 13:48:24 2018 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/query/GridQueryProcessor.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/16dc88fc/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 30c28b1..c4f0197 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -2469,7 +2469,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 null,
                 null,
                 params,
-                null,
+                ctx.localNodeId(),
                 null));
         }
     }


[04/50] [abbrv] ignite git commit: ignite-10043

Posted by sb...@apache.org.
ignite-10043


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

Branch: refs/heads/ignite-10044
Commit: 9a6d642f840897f8b70dc8b7bb7d7cbfedcbd0bd
Parents: 51933a8
Author: sboikov <sb...@apache.org>
Authored: Fri Nov 16 12:58:21 2018 +0300
Committer: sboikov <sb...@apache.org>
Committed: Fri Nov 16 12:58:21 2018 +0300

----------------------------------------------------------------------
 .../IgniteCachePartitionLossPolicySelfTest.java        | 13 ++++++++-----
 1 file changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9a6d642f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
index 9b1ea41..a016865 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
@@ -520,13 +520,16 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
             info("Newly started node: " + grd.cluster().localNode().id());
 
             // Check that partition state does not change after we start each node.
-            for (Ignite ig : G.allGrids()) {
-                verifyCacheOps(canWrite, safe, ig);
+            // TODO With persistence enabled LOST partitions become OWNING after a node joins back - https://issues.apache.org/jira/browse/IGNITE-10044.
+            if (!isPersistenceEnabled) {
+                for (Ignite ig : G.allGrids()) {
+                    verifyCacheOps(canWrite, safe, ig);
 
-                // TODO Query effectively waits for rebalance due to https://issues.apache.org/jira/browse/IGNITE-10057
-                // TODO and after resetLostPartition there is another OWNING copy in the cluster due to https://issues.apache.org/jira/browse/IGNITE-10058.
-                // TODO Uncomment after https://issues.apache.org/jira/browse/IGNITE-10058 is fixed.
+                    // TODO Query effectively waits for rebalance due to https://issues.apache.org/jira/browse/IGNITE-10057
+                    // TODO and after resetLostPartition there is another OWNING copy in the cluster due to https://issues.apache.org/jira/browse/IGNITE-10058.
+                    // TODO Uncomment after https://issues.apache.org/jira/browse/IGNITE-10058 is fixed.
 //                    validateQuery(safe, ig);
+                }
             }
         }
 


[13/50] [abbrv] ignite git commit: IGNITE-10303: SQL: Moved H2 connection management logic into separate class to simplify further development. This closes #5418.

Posted by sb...@apache.org.
IGNITE-10303: SQL: Moved H2 connection management logic into separate class to simplify further development. This closes #5418.


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

Branch: refs/heads/ignite-10044
Commit: 8526adda0bbb069f0179343234b6021e09cc390e
Parents: 16dc88f
Author: devozerov <pp...@gmail.com>
Authored: Sat Nov 17 14:09:59 2018 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Sat Nov 17 14:09:59 2018 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryIndexing.java     |   2 +-
 .../processors/query/GridQueryProcessor.java    |   2 +-
 .../internal/processors/query/QueryUtils.java   |   3 +
 ...IgniteClientCacheInitializationFailTest.java |   2 +-
 .../processors/query/h2/ConnectionManager.java  | 443 ++++++++++++++++++
 .../processors/query/h2/IgniteH2Indexing.java   | 451 ++-----------------
 .../processors/query/h2/dml/UpdatePlan.java     |   2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   2 +-
 .../IgniteCacheQueryH2IndexingLeakTest.java     |  10 +-
 .../cache/index/H2ConnectionLeaksSelfTest.java  |   4 +-
 .../query/h2/sql/GridQueryParsingTest.java      |   2 +-
 12 files changed, 504 insertions(+), 421 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 3eb732c..dab2516 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -330,7 +330,7 @@ public interface GridQueryIndexing {
     /**
      * Cancels all executing queries.
      */
-    public void cancelAllQueries();
+    public void onKernalStop();
 
     /**
      * Gets database schema from cache name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index c4f0197..3842d77 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -267,7 +267,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         if (cancel && idx != null) {
             try {
                 while (!busyLock.tryBlock(500))
-                    idx.cancelAllQueries();
+                    idx.onKernalStop();
 
                 return;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index e530ab1..6a2c22a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -80,6 +80,9 @@ public class QueryUtils {
     /** Schema for system view. */
     public static final String SCHEMA_SYS = "IGNITE";
 
+    /** Schema for system view. */
+    public static final String SCHEMA_INFORMATION = "INFORMATION_SCHEMA";
+
     /** Field name for key. */
     public static final String KEY_FIELD_NAME = "_KEY";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
index ba2fec6..5432257 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
@@ -371,7 +371,7 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
-        @Override public void cancelAllQueries() {
+        @Override public void onKernalStop() {
             // No-op
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
new file mode 100644
index 0000000..09400c8
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2PlainRowFactory;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.h2.server.web.WebServer;
+import org.h2.tools.Server;
+import org.jetbrains.annotations.Nullable;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE_PORT;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT;
+import static org.apache.ignite.IgniteSystemProperties.getInteger;
+import static org.apache.ignite.IgniteSystemProperties.getString;
+
+/**
+ * H2 connection manager.
+ */
+public class ConnectionManager {
+    /** Default DB options. */
+    private static final String DB_OPTIONS = ";LOCK_MODE=3;MULTI_THREADED=1;DB_CLOSE_ON_EXIT=FALSE" +
+        ";DEFAULT_LOCK_TIMEOUT=10000;FUNCTIONS_IN_SCHEMA=true;OPTIMIZE_REUSE_RESULTS=0;QUERY_CACHE_SIZE=0" +
+        ";MAX_OPERATION_MEMORY=0;BATCH_JOINS=1" +
+        ";ROW_FACTORY=\"" + GridH2PlainRowFactory.class.getName() + "\"" +
+        ";DEFAULT_TABLE_ENGINE=" + GridH2DefaultTableEngine.class.getName();
+
+    /** The period of clean up the {@link #threadConns}. */
+    private static final Long CONN_CLEANUP_PERIOD = 2000L;
+
+    /** The period of clean up the statement cache. */
+    private static final Long STMT_CLEANUP_PERIOD = Long.getLong(IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD, 10_000);
+
+    /** The timeout to remove entry from the statement cache if the thread doesn't perform any queries. */
+    private static final Long STMT_TIMEOUT = Long.getLong(IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT, 600 * 1000);
+
+    /*
+     * Initialize system properties for H2.
+     */
+    static {
+        System.setProperty("h2.objectCache", "false");
+        System.setProperty("h2.serializeJavaObject", "false");
+        System.setProperty("h2.objectCacheMaxPerElementSize", "0"); // Avoid ValueJavaObject caching.
+        System.setProperty("h2.optimizeTwoEquals", "false"); // Makes splitter fail on subqueries in WHERE.
+        System.setProperty("h2.dropRestrict", "false"); // Drop schema with cascade semantics.
+    }
+
+    /** Shared connection pool. */
+    private final ThreadLocalObjectPool<H2ConnectionWrapper> connPool =
+        new ThreadLocalObjectPool<>(this::newConnectionWrapper, 5);
+
+    /** Per-thread connections. */
+    private final ConcurrentMap<Thread, H2ConnectionWrapper> threadConns = new ConcurrentHashMap<>();
+
+    /** Connection cache. */
+    private final ThreadLocal<ThreadLocalObjectPool.Reusable<H2ConnectionWrapper>> threadConn =
+        new ThreadLocal<ThreadLocalObjectPool.Reusable<H2ConnectionWrapper>>() {
+        @Override public ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> get() {
+            ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> reusable = super.get();
+
+            boolean reconnect = true;
+
+            try {
+                reconnect = reusable == null || reusable.object().connection().isClosed();
+            }
+            catch (SQLException e) {
+                U.warn(log, "Failed to check connection status.", e);
+            }
+
+            if (reconnect) {
+                reusable = initialValue();
+
+                set(reusable);
+            }
+
+            return reusable;
+        }
+
+        @Override protected ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> initialValue() {
+            ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> reusableConnection = connPool.borrow();
+
+            threadConns.put(Thread.currentThread(), reusableConnection.object());
+
+            return reusableConnection;
+        }
+    };
+
+    /** Database URL. */
+    private final String dbUrl;
+
+    /** Connection cleanup task. */
+    private final GridTimeoutProcessor.CancelableTask connCleanupTask;
+
+    /** Statement cleanup task. */
+    private final GridTimeoutProcessor.CancelableTask stmtCleanupTask;
+
+    /** H2 connection for INFORMATION_SCHEMA. Holds H2 open until node is stopped. */
+    private volatile Connection sysConn;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ConnectionManager(GridKernalContext ctx) throws IgniteCheckedException {
+        dbUrl = "jdbc:h2:mem:" + ctx.localNodeId() + DB_OPTIONS;
+
+        log = ctx.log(ConnectionManager.class);
+
+        org.h2.Driver.load();
+
+        sysConn = connectionNoCache(QueryUtils.SCHEMA_INFORMATION);
+
+        stmtCleanupTask = ctx.timeout().schedule(new Runnable() {
+            @Override public void run() {
+                cleanupStatements();
+            }
+        }, STMT_CLEANUP_PERIOD, STMT_CLEANUP_PERIOD);
+
+        connCleanupTask = ctx.timeout().schedule(new Runnable() {
+            @Override public void run() {
+                cleanupConnections();
+            }
+        }, CONN_CLEANUP_PERIOD, CONN_CLEANUP_PERIOD);
+
+        startDebugConsole();
+    }
+
+    /**
+     * Gets DB connection.
+     *
+     * @param schema Whether to set schema for connection or not.
+     * @return DB connection.
+     */
+    public Connection connectionForThread(@Nullable String schema) {
+        H2ConnectionWrapper c = threadConn.get().object();
+
+        if (c == null)
+            throw new IgniteSQLException("Failed to get DB connection for thread (check log for details).");
+
+        if (schema != null && !F.eq(c.schema(), schema)) {
+            try {
+                c.connection().setSchema(schema);
+                c.schema(schema);
+
+                if (log.isDebugEnabled())
+                    log.debug("Set schema: " + schema);
+            }
+            catch (SQLException e) {
+                throw new IgniteSQLException("Failed to set schema for DB connection for thread [schema=" +
+                    schema + "]", e);
+            }
+        }
+
+        return c.connection();
+    }
+
+    /**
+     * @return Per-thread connections (for testing purposes only).
+     */
+    public Map<Thread, H2ConnectionWrapper> connectionsForThread() {
+        return threadConns;
+    }
+
+    /**
+     * Removes from cache and returns associated with current thread connection.
+     *
+     * @return Connection associated with current thread.
+     */
+    public ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> detachThreadConnection() {
+        Thread key = Thread.currentThread();
+
+        ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> reusableConnection = threadConn.get();
+
+        H2ConnectionWrapper connection = threadConns.remove(key);
+
+        threadConn.remove();
+
+        assert reusableConnection.object().connection() == connection.connection();
+
+        return reusableConnection;
+    }
+
+    /**
+     * Get connection without cache.
+     *
+     * @param schema Schema name.
+     * @return Connection.
+     */
+    public Connection connectionNoCache(String schema) throws IgniteSQLException {
+        try {
+            Connection conn = DriverManager.getConnection(dbUrl);
+
+            conn.setSchema(schema);
+
+            return conn;
+        }
+        catch (SQLException e) {
+            throw new IgniteSQLException("Failed to initialize system DB connection: " + dbUrl, e);
+        }
+    }
+
+    /**
+     * @return {@link H2StatementCache} associated with current thread.
+     */
+    public H2StatementCache statementCacheForThread() {
+        H2StatementCache statementCache = threadConn.get().object().statementCache();
+
+        statementCache.updateLastUsage();
+
+        return statementCache;
+    }
+
+    /**
+     * Execute SQL statement on specific schema.
+     *
+     * @param schema Schema
+     * @param sql SQL statement.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void executeStatement(String schema, String sql) throws IgniteCheckedException {
+        Statement stmt = null;
+
+        try {
+            Connection c = connectionForThread(schema);
+
+            stmt = c.createStatement();
+
+            stmt.executeUpdate(sql);
+        }
+        catch (SQLException e) {
+            onSqlException();
+
+            throw new IgniteSQLException("Failed to execute statement: " + sql, e);
+        }
+        finally {
+            U.close(stmt, log);
+        }
+    }
+
+    /**
+     * Execute statement on H2 INFORMATION_SCHEMA.
+     *
+     * @param sql SQL statement.
+     */
+    public void executeSystemStatement(String sql) {
+        Statement stmt = null;
+
+        try {
+            stmt = sysConn.createStatement();
+
+            stmt.executeUpdate(sql);
+        }
+        catch (SQLException e) {
+            onSqlException();
+
+            throw new IgniteSQLException("Failed to execute system statement: " + sql, e);
+        }
+        finally {
+            U.close(stmt, log);
+        }
+    }
+
+    /**
+     * Clear statement cache when cache is unregistered..
+     */
+    public void onCacheUnregistered() {
+        threadConns.values().forEach(H2ConnectionWrapper::clearStatementCache);
+    }
+
+    /**
+     * Cancel all queries.
+     */
+    public void onKernalStop() {
+        for (H2ConnectionWrapper c : threadConns.values())
+            U.close(c, log);
+    }
+
+    /**
+     * Close executor.
+     */
+    public void stop() {
+        for (H2ConnectionWrapper c : threadConns.values())
+            U.close(c, log);
+
+        threadConns.clear();
+
+        try (Connection c = connectionNoCache(QueryUtils.SCHEMA_INFORMATION); Statement s = c.createStatement()) {
+            s.execute("SHUTDOWN");
+        }
+        catch (SQLException e) {
+            U.error(log, "Failed to shutdown database.", e);
+        }
+
+        if (stmtCleanupTask != null)
+            stmtCleanupTask.close();
+
+        if (connCleanupTask != null)
+            connCleanupTask.close();
+
+        if (sysConn != null) {
+            U.close(sysConn, log);
+
+            sysConn = null;
+        }
+    }
+
+    /**
+     * Handles SQL exception.
+     */
+    public void onSqlException() {
+        Connection conn = threadConn.get().object().connection();
+
+        threadConn.set(null);
+
+        if (conn != null) {
+            threadConns.remove(Thread.currentThread());
+
+            // Reset connection to receive new one at next call.
+            U.close(conn, log);
+        }
+    }
+
+    /**
+     * Start debug console if needed.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startDebugConsole() throws IgniteCheckedException {
+        try {
+            if (getString(IGNITE_H2_DEBUG_CONSOLE) != null) {
+                Connection c = DriverManager.getConnection(dbUrl);
+
+                int port = getInteger(IGNITE_H2_DEBUG_CONSOLE_PORT, 0);
+
+                WebServer webSrv = new WebServer();
+                Server web = new Server(webSrv, "-webPort", Integer.toString(port));
+                web.start();
+                String url = webSrv.addSession(c);
+
+                U.quietAndInfo(log, "H2 debug console URL: " + url);
+
+                try {
+                    Server.openBrowser(url);
+                }
+                catch (Exception e) {
+                    U.warn(log, "Failed to open browser: " + e.getMessage());
+                }
+            }
+        }
+        catch (SQLException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     * Create new connection wrapper.
+     *
+     * @return Connection wrapper.
+     */
+    private H2ConnectionWrapper newConnectionWrapper() {
+        try {
+            return new H2ConnectionWrapper(DriverManager.getConnection(dbUrl));
+        }
+        catch (SQLException e) {
+            throw new IgniteSQLException("Failed to initialize DB connection: " + dbUrl, e);
+        }
+    }
+
+    /**
+     * Called periodically to cleanup connections.
+     */
+    private void cleanupConnections() {
+        for (Iterator<Map.Entry<Thread, H2ConnectionWrapper>> it = threadConns.entrySet().iterator(); it.hasNext(); ) {
+            Map.Entry<Thread, H2ConnectionWrapper> entry = it.next();
+
+            Thread t = entry.getKey();
+
+            if (t.getState() == Thread.State.TERMINATED) {
+                U.close(entry.getValue(), log);
+
+                it.remove();
+            }
+        }
+    }
+
+    /**
+     * Called periodically to clean up the statement cache.
+     */
+    private void cleanupStatements() {
+        long now = U.currentTimeMillis();
+
+        for (Iterator<Map.Entry<Thread, H2ConnectionWrapper>> it = threadConns.entrySet().iterator(); it.hasNext(); ) {
+            Map.Entry<Thread, H2ConnectionWrapper> entry = it.next();
+
+            Thread t = entry.getKey();
+
+            if (t.getState() == Thread.State.TERMINATED) {
+                U.close(entry.getValue(), log);
+
+                it.remove();
+            }
+            else if (now - entry.getValue().statementCache().lastUsage() > STMT_TIMEOUT)
+                entry.getValue().clearStatementCache();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 58e09cb..937363a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -22,7 +22,6 @@ import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Connection;
-import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
@@ -112,9 +111,7 @@ import org.apache.ignite.internal.processors.query.h2.database.io.H2MvccLeafIO;
 import org.apache.ignite.internal.processors.query.h2.ddl.DdlStatementsProcessor;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2PlainRowFactory;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
@@ -141,7 +138,6 @@ import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryReq
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorImpl;
-import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.sql.SqlParseException;
 import org.apache.ignite.internal.sql.SqlParser;
 import org.apache.ignite.internal.sql.SqlStrictParseException;
@@ -188,19 +184,11 @@ import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Index;
 import org.h2.jdbc.JdbcStatement;
-import org.h2.server.web.WebServer;
 import org.h2.table.IndexColumn;
-import org.h2.tools.Server;
 import org.h2.util.JdbcUtils;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE_PORT;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT;
-import static org.apache.ignite.IgniteSystemProperties.getInteger;
-import static org.apache.ignite.IgniteSystemProperties.getString;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.checkActive;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccEnabled;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.tx;
@@ -237,27 +225,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      */
     static {
         PageIO.registerH2(H2InnerIO.VERSIONS, H2LeafIO.VERSIONS, H2MvccInnerIO.VERSIONS, H2MvccLeafIO.VERSIONS);
+
         H2ExtrasInnerIO.register();
         H2ExtrasLeafIO.register();
-
-        // Initialize system properties for H2.
-        System.setProperty("h2.objectCache", "false");
-        System.setProperty("h2.serializeJavaObject", "false");
-        System.setProperty("h2.objectCacheMaxPerElementSize", "0"); // Avoid ValueJavaObject caching.
-        System.setProperty("h2.optimizeTwoEquals", "false"); // Makes splitter fail on subqueries in WHERE.
-        System.setProperty("h2.dropRestrict", "false"); // Drop schema with cascade semantics.
     }
 
-    /** Default DB options. */
-    private static final String DB_OPTIONS = ";LOCK_MODE=3;MULTI_THREADED=1;DB_CLOSE_ON_EXIT=FALSE" +
-        ";DEFAULT_LOCK_TIMEOUT=10000;FUNCTIONS_IN_SCHEMA=true;OPTIMIZE_REUSE_RESULTS=0;QUERY_CACHE_SIZE=0" +
-        ";MAX_OPERATION_MEMORY=0;BATCH_JOINS=1" +
-        ";ROW_FACTORY=\"" + GridH2PlainRowFactory.class.getName() + "\"" +
-        ";DEFAULT_TABLE_ENGINE=" + GridH2DefaultTableEngine.class.getName();
-
-        // Uncomment this setting to get debug output from H2 to sysout.
-//        ";TRACE_LEVEL_SYSTEM_OUT=3";
-
     /** Dummy metadata for update result. */
     public static final List<GridQueryFieldMetadata> UPDATE_RESULT_META = Collections.<GridQueryFieldMetadata>
         singletonList(new H2SqlFieldMetadata(null, null, "UPDATED", Long.class.getName(), -1, -1));
@@ -265,23 +237,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** */
     private static final int TWO_STEP_QRY_CACHE_SIZE = 1024;
 
-    /** The period of clean up the statement cache. */
-    private final Long CLEANUP_STMT_CACHE_PERIOD = Long.getLong(IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD, 10_000);
-
-    /** The period of clean up the {@link #conns}. */
-    @SuppressWarnings("FieldCanBeLocal")
-    private final Long CLEANUP_CONNECTIONS_PERIOD = 2000L;
-
-    /** The timeout to remove entry from the statement cache if the thread doesn't perform any queries. */
-    private final Long STATEMENT_CACHE_THREAD_USAGE_TIMEOUT =
-        Long.getLong(IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT, 600 * 1000);
-
-    /** */
-    private GridTimeoutProcessor.CancelableTask stmtCacheCleanupTask;
-
-    /** */
-    private GridTimeoutProcessor.CancelableTask connCleanupTask;
-
     /** Logger. */
     @LoggerResource
     private IgniteLogger log;
@@ -296,13 +251,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private final ConcurrentMap<String, H2Schema> schemas = new ConcurrentHashMap<>();
 
     /** */
-    private String dbUrl = "jdbc:h2:mem:";
-
-    /** */
-    // TODO https://issues.apache.org/jira/browse/IGNITE-9062
-    private final ConcurrentMap<Thread, H2ConnectionWrapper> conns = new ConcurrentHashMap<>();
-
-    /** */
     private GridMapQueryExecutor mapQryExec;
 
     /** */
@@ -327,43 +275,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private final H2RowCacheRegistry rowCache = new H2RowCacheRegistry();
 
     /** */
-    // TODO https://issues.apache.org/jira/browse/IGNITE-9062
-    private final ThreadLocalObjectPool<H2ConnectionWrapper> connectionPool = new ThreadLocalObjectPool<>(IgniteH2Indexing.this::newConnectionWrapper, 5);
-
-    /** */
-    // TODO https://issues.apache.org/jira/browse/IGNITE-9062
-    private final ThreadLocal<ThreadLocalObjectPool.Reusable<H2ConnectionWrapper>> connCache = new ThreadLocal<ThreadLocalObjectPool.Reusable<H2ConnectionWrapper>>() {
-        @Override public ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> get() {
-            ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> reusable = super.get();
-
-            boolean reconnect = true;
-
-            try {
-                reconnect = reusable == null || reusable.object().connection().isClosed();
-            }
-            catch (SQLException e) {
-                U.warn(log, "Failed to check connection status.", e);
-            }
-
-            if (reconnect) {
-                reusable = initialValue();
-
-                set(reusable);
-            }
-
-            return reusable;
-        }
-
-        @Override protected ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> initialValue() {
-            ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> reusableConnection = connectionPool.borrow();
-
-            conns.put(Thread.currentThread(), reusableConnection.object());
-
-            return reusableConnection;
-        }
-    };
-
-    /** */
     protected volatile GridKernalContext ctx;
 
     /** Cache object value context. */
@@ -394,8 +305,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
     };
 
-    /** H2 JDBC connection for INFORMATION_SCHEMA. Holds H2 open until node is stopped. */
-    private Connection sysConn;
+    /** Query executor. */
+    private ConnectionManager connMgr;
 
     /**
      * @return Kernal context.
@@ -405,48 +316,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * @param schema Schema.
-     * @return Connection.
-     */
-    public Connection connectionForSchema(String schema) {
-        try {
-            return connectionForThread(schema);
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteException(e);
-        }
-    }
-
-    /**
-     * @return H2 JDBC connection to INFORMATION_SCHEMA.
-     */
-    private Connection systemConnection() {
-        assert Thread.holdsLock(schemaMux);
-
-        if (sysConn == null) {
-            try {
-                sysConn = DriverManager.getConnection(dbUrl);
-
-                sysConn.setSchema("INFORMATION_SCHEMA");
-            }
-            catch (SQLException e) {
-                throw new IgniteSQLException("Failed to initialize system DB connection: " + dbUrl, e);
-            }
-        }
-
-        return sysConn;
-    }
-
-    /** */
-    private H2ConnectionWrapper newConnectionWrapper() {
-        try {
-            return new H2ConnectionWrapper(DriverManager.getConnection(dbUrl));
-        } catch (SQLException e) {
-            throw new IgniteSQLException("Failed to initialize DB connection: " + dbUrl, e);
-        }
-    }
-
-    /**
      * @param c Connection.
      * @param sql SQL.
      * @return <b>Cached</b> prepared statement.
@@ -489,7 +358,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         assert useStmtCache || !cachedOnly;
 
         if (useStmtCache) {
-            H2StatementCache cache = getStatementsCacheForCurrentThread();
+            H2StatementCache cache = connMgr.statementCacheForThread();
 
             H2CachedStatementKey key = new H2CachedStatementKey(c.getSchema(), sql);
 
@@ -538,63 +407,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             return c.prepareStatement(sql, ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_READ_ONLY);
     }
 
-    /**
-     * @return {@link H2StatementCache} associated with current thread.
-     */
-    @NotNull private H2StatementCache getStatementsCacheForCurrentThread() {
-        H2StatementCache statementCache = connCache.get().object().statementCache();
-
-        statementCache.updateLastUsage();
-
-        return statementCache;
-    }
-
     /** {@inheritDoc} */
     @Override public PreparedStatement prepareNativeStatement(String schemaName, String sql) {
-        Connection conn = connectionForSchema(schemaName);
+        Connection conn = connMgr.connectionForThread(schemaName);
 
         return prepareStatementAndCaches(conn, sql);
     }
 
     /**
-     * Gets DB connection.
-     *
-     * @param schema Whether to set schema for connection or not.
-     * @return DB connection.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private Connection connectionForThread(@Nullable String schema) throws IgniteCheckedException {
-        H2ConnectionWrapper c = connCache.get().object();
-
-        if (c == null)
-            throw new IgniteCheckedException("Failed to get DB connection for thread (check log for details).");
-
-        if (schema != null && !F.eq(c.schema(), schema)) {
-            Statement stmt = null;
-
-            try {
-                stmt = c.connection().createStatement();
-
-                stmt.executeUpdate("SET SCHEMA " + H2Utils.withQuotes(schema));
-
-                if (log.isDebugEnabled())
-                    log.debug("Set schema: " + schema);
-
-                c.schema(schema);
-            }
-            catch (SQLException e) {
-                throw new IgniteSQLException("Failed to set schema for DB connection for thread [schema=" +
-                    schema + "]", e);
-            }
-            finally {
-                U.close(stmt, log);
-            }
-        }
-
-        return c.connection();
-    }
-
-    /**
      * Create and register schema if needed.
      *
      * @param schemaName Schema name.
@@ -642,7 +462,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param schema Schema name.
      */
     private void createSchema0(String schema) {
-        executeSystemStatement("CREATE SCHEMA IF NOT EXISTS " + H2Utils.withQuotes(schema));
+        connMgr.executeSystemStatement("CREATE SCHEMA IF NOT EXISTS " + H2Utils.withQuotes(schema));
 
         if (log.isDebugEnabled())
             log.debug("Created H2 schema for index database: " + schema);
@@ -654,62 +474,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param schema Schema name.
      */
     private void dropSchema(String schema) {
-        executeSystemStatement("DROP SCHEMA IF EXISTS " + H2Utils.withQuotes(schema));
+        connMgr.executeSystemStatement("DROP SCHEMA IF EXISTS " + H2Utils.withQuotes(schema));
 
         if (log.isDebugEnabled())
             log.debug("Dropped H2 schema for index database: " + schema);
     }
 
     /**
-     * @param schema Schema
-     * @param sql SQL statement.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void executeStatement(String schema, String sql) throws IgniteCheckedException {
-        Statement stmt = null;
-
-        try {
-            Connection c = connectionForThread(schema);
-
-            stmt = c.createStatement();
-
-            stmt.executeUpdate(sql);
-        }
-        catch (SQLException e) {
-            onSqlException();
-
-            throw new IgniteSQLException("Failed to execute statement: " + sql, e);
-        }
-        finally {
-            U.close(stmt, log);
-        }
-    }
-
-    /**
-     * Execute statement on H2 INFORMATION_SCHEMA.
-     * @param sql SQL statement.
-     */
-    public void executeSystemStatement(String sql) {
-        assert Thread.holdsLock(schemaMux);
-
-        Statement stmt = null;
-
-        try {
-            stmt = systemConnection().createStatement();
-
-            stmt.executeUpdate(sql);
-        }
-        catch (SQLException e) {
-            onSqlException();
-
-            throw new IgniteSQLException("Failed to execute statement: " + sql, e);
-        }
-        finally {
-            U.close(stmt, log);
-        }
-    }
-
-    /**
      * Binds object to prepared statement.
      *
      * @param stmt SQL statement.
@@ -734,22 +505,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
     }
 
-    /**
-     * Handles SQL exception.
-     */
-    private void onSqlException() {
-        Connection conn = connCache.get().object().connection();
-
-        connCache.set(null);
-
-        if (conn != null) {
-            conns.remove(Thread.currentThread());
-
-            // Reset connection to receive new one at next call.
-            U.close(conn, log);
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public void store(GridCacheContext cctx,
         GridQueryTypeDescriptor type,
@@ -811,7 +566,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (log.isDebugEnabled())
             log.debug("Removing query index table: " + tbl.fullTableName());
 
-        Connection c = connectionForThread(tbl.schemaName());
+        Connection c = connMgr.connectionForThread(tbl.schemaName());
 
         Statement stmt = null;
 
@@ -826,7 +581,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             stmt.executeUpdate(sql);
         }
         catch (SQLException e) {
-            onSqlException();
+            connMgr.onSqlException();
 
             throw new IgniteSQLException("Failed to drop database index table [type=" + tbl.type().name() +
                 ", table=" + tbl.fullTableName() + "]", IgniteQueryErrorCode.TABLE_DROP_FAILED, e);
@@ -970,7 +725,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      */
     private void executeSql(String schemaName, String sql) throws IgniteCheckedException {
         try {
-            Connection conn = connectionForSchema(schemaName);
+            Connection conn = connMgr.connectionForThread(schemaName);
 
             try (PreparedStatement stmt = prepareStatement(conn, sql, false)) {
                 stmt.execute();
@@ -1081,7 +836,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         assert mvccEnabled || mvccTracker == null;
 
         try {
-            final Connection conn = connectionForSchema(schemaName);
+            final Connection conn = connMgr.connectionForThread(schemaName);
 
             H2Utils.setupConnection(conn, false, enforceJoinOrder);
 
@@ -1283,9 +1038,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      */
     public static int operationTimeout(int qryTimeout, IgniteTxAdapter tx) {
         if (tx != null) {
-            int tm1 = (int)tx.remainingTime(), tm2 = qryTimeout;
+            int remaining = (int)tx.remainingTime();
 
-            return tm1 > 0 && tm2 > 0 ? Math.min(tm1, tm2) : Math.max(tm1, tm2);
+            return remaining > 0 && qryTimeout > 0 ? Math.min(remaining, qryTimeout) : Math.max(remaining, qryTimeout);
         }
 
         return qryTimeout;
@@ -1294,7 +1049,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** {@inheritDoc} */
     @Override public long streamUpdateQuery(String schemaName, String qry,
         @Nullable Object[] params, IgniteDataStreamer<?, ?> streamer) throws IgniteCheckedException {
-        final Connection conn = connectionForSchema(schemaName);
+        final Connection conn = connMgr.connectionForThread(schemaName);
 
         final PreparedStatement stmt;
 
@@ -1318,7 +1073,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             return zeroBatchedStreamedUpdateResult(params.size());
         }
 
-        final Connection conn = connectionForSchema(schemaName);
+        final Connection conn = connMgr.connectionForThread(schemaName);
 
         final PreparedStatement stmt = prepareStatementAndCaches(conn, qry);
 
@@ -1512,7 +1267,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             return rs;
         }
         catch (SQLException e) {
-            onSqlException();
+            connMgr.onSqlException();
 
             throw new IgniteCheckedException(e);
         }
@@ -1728,6 +1483,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
     @Override public SqlFieldsQuery generateFieldsQuery(String cacheName, SqlQuery qry) {
         String schemaName = schema(cacheName);
 
@@ -2096,7 +1852,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 // Second, let's check if we already have a parsed statement...
                 PreparedStatement cachedStmt;
 
-                if ((cachedStmt = cachedStatement(connectionForSchema(schemaName), qry.getSql())) != null) {
+                if ((cachedStmt = cachedStatement(connMgr.connectionForThread(schemaName), qry.getSql())) != null) {
                     Prepared prepared = GridSqlQueryParser.prepared(cachedStmt);
 
                     // We may use this cached statement only for local queries and non queries.
@@ -2189,7 +1945,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (!prepared.isQuery()) {
             if (DmlStatementsProcessor.isDmlStatement(prepared)) {
                 try {
-                    Connection conn = connectionForSchema(schemaName);
+                    Connection conn = connMgr.connectionForThread(schemaName);
 
                     if (!loc)
                         return dmlProc.updateSqlFieldsDistributed(schemaName, conn, prepared, qry, cancel);
@@ -2287,7 +2043,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      *     metadata for two-step query (if needed), evaluated query local execution flag.
      */
     private ParsingResult parseAndSplit(String schemaName, SqlFieldsQuery qry, int firstArg) {
-        Connection c = connectionForSchema(schemaName);
+        Connection c = connMgr.connectionForThread(schemaName);
 
         // For queries that are explicitly local, we rely on the flag specified in the query
         // because this parsing result will be cached and used for queries directly.
@@ -2379,7 +2135,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         // Let's not cache multiple statements and distributed queries as whole two step query will be cached later on.
         if (remainingSql != null || hasTwoStep)
-            getStatementsCacheForCurrentThread().remove(schemaName, qry.getSql());
+            connMgr.statementCacheForThread().remove(schemaName, qry.getSql());
 
         if (!hasTwoStep)
             return new ParsingResult(prepared, newQry, remainingSql, null, null, null);
@@ -2448,7 +2204,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      */
     private GridCacheTwoStepQuery split(Prepared prepared, SqlFieldsQuery qry) throws IgniteCheckedException,
         SQLException {
-        GridCacheTwoStepQuery res = GridSqlQuerySplitter.split(connectionForThread(qry.getSchema()), prepared,
+        GridCacheTwoStepQuery res = GridSqlQuerySplitter.split(connMgr.connectionForThread(qry.getSchema()), prepared,
             qry.getArgs(), qry.isCollocated(), qry.isDistributedJoins(), qry.isEnforceJoinOrder(), this);
 
         List<Integer> cacheIds = collectCacheIds(null, res);
@@ -2474,6 +2230,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param qry Sql fields query.autoStartTx(qry)
      * @return {@code True} if need to start transaction.
      */
+    @SuppressWarnings("SimplifiableIfStatement")
     public boolean autoStartTx(SqlFieldsQuery qry) {
         if (!mvccEnabled(ctx))
             return false;
@@ -2512,7 +2269,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             loc = false;
         }
 
-        Connection conn = connectionForSchema(schema);
+        Connection conn = connMgr.connectionForThread(schema);
 
         H2Utils.setupConnection(conn, false, fldsQry.isEnforceJoinOrder());
 
@@ -2619,7 +2376,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      */
     public UpdateResult mapDistributedUpdate(String schemaName, SqlFieldsQuery fldsQry, IndexingQueryFilter filter,
         GridQueryCancel cancel, boolean local) throws IgniteCheckedException {
-        Connection conn = connectionForSchema(schemaName);
+        Connection conn = connMgr.connectionForThread(schemaName);
 
         H2Utils.setupConnection(conn, false, fldsQry.isEnforceJoinOrder());
 
@@ -2703,14 +2460,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         H2TableDescriptor tbl = new H2TableDescriptor(this, schema, type, cctx, isSql);
 
         try {
-            Connection conn = connectionForThread(schemaName);
+            Connection conn = connMgr.connectionForThread(schemaName);
 
             createTable(schemaName, schema, tbl, conn);
 
             schema.add(tbl);
         }
         catch (SQLException e) {
-            onSqlException();
+            connMgr.onSqlException();
 
             throw new IgniteCheckedException("Failed to register query type: " + type, e);
         }
@@ -2938,62 +2695,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Called periodically by {@link GridTimeoutProcessor} to clean up the statement cache.
-     */
-    private void cleanupStatementCache() {
-        long now = U.currentTimeMillis();
-
-        for (Iterator<Map.Entry<Thread, H2ConnectionWrapper>> it = conns.entrySet().iterator(); it.hasNext(); ) {
-            Map.Entry<Thread, H2ConnectionWrapper> entry = it.next();
-
-            Thread t = entry.getKey();
-
-            if (t.getState() == Thread.State.TERMINATED) {
-                U.close(entry.getValue(), log);
-
-                it.remove();
-            }
-            else if (now - entry.getValue().statementCache().lastUsage() > STATEMENT_CACHE_THREAD_USAGE_TIMEOUT)
-                entry.getValue().clearStatementCache();
-        }
-    }
-
-    /**
-     * Called periodically by {@link GridTimeoutProcessor} to clean up the {@link #conns}.
-     */
-    private void cleanupConnections() {
-        for (Iterator<Map.Entry<Thread, H2ConnectionWrapper>> it = conns.entrySet().iterator(); it.hasNext(); ) {
-            Map.Entry<Thread, H2ConnectionWrapper> entry = it.next();
-
-            Thread t = entry.getKey();
-
-            if (t.getState() == Thread.State.TERMINATED) {
-                U.close(entry.getValue(), log);
-
-                it.remove();
-            }
-        }
-    }
-
-    /**
-     * Removes from cache and returns associated with current thread connection.
-     * @return Connection associated with current thread.
-     */
-    public ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> detach() {
-        Thread key = Thread.currentThread();
-
-        ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> reusableConnection = connCache.get();
-
-        H2ConnectionWrapper connection = conns.remove(key);
-
-        connCache.remove();
-
-        assert reusableConnection.object().connection() == connection.connection();
-
-        return reusableConnection;
-    }
-
-    /**
      * Rebuild indexes from hash index.
      *
      * @param cacheName Cache name.
@@ -3060,36 +2761,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             SysProperties.serializeJavaObject = false;
         }
 
-        String dbName = (ctx != null ? ctx.localNodeId() : UUID.randomUUID()).toString();
-
-        dbUrl = "jdbc:h2:mem:" + dbName + DB_OPTIONS;
-
-        org.h2.Driver.load();
-
-        try {
-            if (getString(IGNITE_H2_DEBUG_CONSOLE) != null) {
-                Connection c = DriverManager.getConnection(dbUrl);
-
-                int port = getInteger(IGNITE_H2_DEBUG_CONSOLE_PORT, 0);
-
-                WebServer webSrv = new WebServer();
-                Server web = new Server(webSrv, "-webPort", Integer.toString(port));
-                web.start();
-                String url = webSrv.addSession(c);
-
-                U.quietAndInfo(log, "H2 debug console URL: " + url);
-
-                try {
-                    Server.openBrowser(url);
-                }
-                catch (Exception e) {
-                    U.warn(log, "Failed to open browser: " + e.getMessage());
-                }
-            }
-        }
-        catch (SQLException e) {
-            throw new IgniteCheckedException(e);
-        }
+        connMgr = new ConnectionManager(ctx);
 
         if (ctx == null) {
             // This is allowed in some tests.
@@ -3129,12 +2801,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             mapQryExec.start(ctx, this);
             rdcQryExec.start(ctx, this);
 
-            stmtCacheCleanupTask = ctx.timeout().schedule(new Runnable() {
-                @Override public void run() {
-                    cleanupStatementCache();
-                }
-            }, CLEANUP_STMT_CACHE_PERIOD, CLEANUP_STMT_CACHE_PERIOD);
-
             dmlProc = new DmlStatementsProcessor();
             ddlProc = new DdlStatementsProcessor();
 
@@ -3150,17 +2816,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                         createSchema0(QueryUtils.SCHEMA_SYS);
                     }
 
-                    Connection c = connectionForSchema(QueryUtils.SCHEMA_SYS);
-
-                    for (SqlSystemView view : systemViews(ctx))
-                        SqlSystemTableEngine.registerView(c, view);
+                    try (Connection c = connMgr.connectionNoCache(QueryUtils.SCHEMA_SYS)) {
+                        for (SqlSystemView view : systemViews(ctx))
+                            SqlSystemTableEngine.registerView(c, view);
+                    }
                 }
                 catch (SQLException e) {
                     throw new IgniteCheckedException("Failed to register system view.", e);
                 }
-
-                // Caching this connection in ThreadLocal may lead to memory leaks.
-                connCache.set(null);
             }
             else {
                 if (log.isDebugEnabled())
@@ -3175,12 +2838,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         JdbcUtils.serializer = h2Serializer();
 
         assert ctx != null;
-
-        connCleanupTask = ctx.timeout().schedule(new Runnable() {
-            @Override public void run() {
-                cleanupConnections();
-            }
-        }, CLEANUP_CONNECTIONS_PERIOD, CLEANUP_CONNECTIONS_PERIOD);
     }
 
     /**
@@ -3348,7 +3005,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                         " FOR \"") +
                         cls.getName() + '.' + m.getName() + '"';
 
-                    executeStatement(schema, clause);
+                    connMgr.executeStatement(schema, clause);
                 }
             }
         }
@@ -3361,39 +3018,18 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         mapQryExec.cancelLazyWorkers();
 
-        for (H2ConnectionWrapper c : conns.values())
-            U.close(c, log);
-
-        conns.clear();
         schemas.clear();
         cacheName2schema.clear();
 
-        try (Connection c = DriverManager.getConnection(dbUrl); Statement s = c.createStatement()) {
-            s.execute("SHUTDOWN");
-        }
-        catch (SQLException e) {
-            U.error(log, "Failed to shutdown database.", e);
-        }
-
-        if (stmtCacheCleanupTask != null)
-            stmtCacheCleanupTask.close();
-
-        if (connCleanupTask != null)
-            connCleanupTask.close();
-
         GridH2QueryContext.clearLocalNodeStop(nodeId);
 
-        if (log.isDebugEnabled())
-            log.debug("Cache query index stopped.");
-
         // Close system H2 connection to INFORMATION_SCHEMA
         synchronized (schemaMux) {
-            if (sysConn != null) {
-                U.close(sysConn, log);
-
-                sysConn = null;
-            }
+            connMgr.stop();
         }
+
+        if (log.isDebugEnabled())
+            log.debug("Cache query index stopped.");
     }
 
     /** {@inheritDoc} */
@@ -3471,7 +3107,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 }
             }
 
-            conns.values().forEach(H2ConnectionWrapper::clearStatementCache);
+            connMgr.onCacheUnregistered();
 
             for (H2TableDescriptor tbl : rmvTbls) {
                 for (Index idx : tbl.table().getIndexes())
@@ -3628,18 +3264,17 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void cancelAllQueries() {
+    @Override public void onKernalStop() {
         mapQryExec.cancelLazyWorkers();
 
-        for (H2ConnectionWrapper c : conns.values())
-            U.close(c, log);
+        connMgr.onKernalStop();
     }
 
     /**
-     * @return Per-thread connections.
+     * @return Query executor.
      */
-    public Map<Thread, ?> perThreadConnections() {
-        return conns;
+    public ConnectionManager connections() {
+        return connMgr;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
index ba4b12b..0f8b6d8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
@@ -647,7 +647,7 @@ public final class UpdatePlan {
 
         /** {@inheritDoc} */
         @Override public void beforeDetach() {
-            ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> conn0 = conn = idx.detach();
+            ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> conn0 = conn = idx.connections().detachThreadConnection();
 
             if (isClosed())
                 conn0.recycle();

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index c2ea9a8..569cb60 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -875,7 +875,7 @@ public class GridMapQueryExecutor {
                 .mvccSnapshot(mvccSnapshot)
                 .lazyWorker(worker);
 
-            Connection conn = h2.connectionForSchema(schemaName);
+            Connection conn = h2.connections().connectionForThread(schemaName);
 
             H2Utils.setupConnection(conn, distributedJoinMode != OFF, enforceJoinOrder);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 87c8ce9..36287b3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -656,7 +656,7 @@ public class GridReduceQueryExecutor {
             }
 
             final ReduceQueryRun r = new ReduceQueryRun(qryReqId, qry.originalSql(), schemaName,
-                h2.connectionForSchema(schemaName), qry.mapQueries().size(), qry.pageSize(),
+                h2.connections().connectionForThread(schemaName), qry.mapQueries().size(), qry.pageSize(),
                 U.currentTimeMillis(), sfuFut, cancel);
 
             Collection<ClusterNode> nodes;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java
index 14593de..9458887 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.concurrent.ConcurrentMap;
+import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteCache;
@@ -119,9 +119,10 @@ public class IgniteCacheQueryH2IndexingLeakTest extends GridCommonAbstractTest {
      * @return size of statement cache.
      */
     private static int getStatementCacheSize(GridQueryProcessor qryProcessor) {
-        IgniteH2Indexing h2Idx = GridTestUtils.getFieldValue(qryProcessor, GridQueryProcessor.class, "idx");
 
-        ConcurrentMap<Thread, H2ConnectionWrapper> conns = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "conns");
+        IgniteH2Indexing h2Idx = (IgniteH2Indexing)qryProcessor.getIndexing();
+
+        Map<Thread, H2ConnectionWrapper> conns = h2Idx.connections().connectionsForThread();
 
         int cntr = 0;
 
@@ -145,10 +146,10 @@ public class IgniteCacheQueryH2IndexingLeakTest extends GridCommonAbstractTest {
             // Open iterator on the created cursor: add entries to the cache.
             IgniteInternalFuture<?> fut = multithreadedAsync(
                 new CAX() {
+                    @SuppressWarnings("unchecked")
                     @Override public void applyx() throws IgniteCheckedException {
                         while (!stop.get()) {
                             c.query(new SqlQuery(Integer.class, "_val >= 0")).getAll();
-
                             c.query(new SqlQuery(Integer.class, "_val >= 1")).getAll();
                         }
                     }
@@ -197,6 +198,7 @@ public class IgniteCacheQueryH2IndexingLeakTest extends GridCommonAbstractTest {
             // Open iterator on the created cursor: add entries to the cache
             IgniteInternalFuture<?> fut = multithreadedAsync(
                 new CAX() {
+                    @SuppressWarnings("unchecked")
                     @Override public void applyx() throws IgniteCheckedException {
                         c.query(new SqlQuery(Integer.class, "_val >= 0")).getAll();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java
index 7713004..d8d6735 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java
@@ -108,7 +108,7 @@ public class H2ConnectionLeaksSelfTest extends GridCommonAbstractTest {
                     try {
                         IgniteH2Indexing idx = (IgniteH2Indexing)grid(1).context().query().getIndexing();
 
-                        idx.executeStatement(CACHE_NAME, "select *");
+                        idx.connections().executeStatement(CACHE_NAME, "select *");
                     }
                     catch (Exception e) {
                         // No-op.
@@ -172,6 +172,6 @@ public class H2ConnectionLeaksSelfTest extends GridCommonAbstractTest {
      * @return Per-thread connections.
      */
     private Map<Thread, ?> perThreadConnections(int nodeIdx) {
-        return ((IgniteH2Indexing)grid(nodeIdx).context().query().getIndexing()).perThreadConnections();
+        return ((IgniteH2Indexing)grid(nodeIdx).context().query().getIndexing()).connections().connectionsForThread();
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8526adda/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index ca24e57..1a4b248 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -1008,7 +1008,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
 
         String schemaName = idx.schema(DEFAULT_CACHE_NAME);
 
-        return (JdbcConnection)idx.connectionForSchema(schemaName);
+        return (JdbcConnection)idx.connections().connectionForThread(schemaName);
     }
 
     /**


[40/50] [abbrv] ignite git commit: merge

Posted by sb...@apache.org.
 merge


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

Branch: refs/heads/ignite-10044
Commit: c634be2485cc9798851e3645526d139b18381111
Parents: 9a03519 bc1ac15
Author: sboikov <sb...@apache.org>
Authored: Tue Nov 20 14:44:51 2018 +0300
Committer: sboikov <sb...@apache.org>
Committed: Tue Nov 20 14:44:51 2018 +0300

----------------------------------------------------------------------
 .../internal/pagemem/PageIdAllocator.java       |   3 +
 .../processors/cache/CacheGroupContext.java     |   1 -
 .../cache/GridCacheConcurrentMapImpl.java       |   1 -
 .../processors/cache/GridCacheContext.java      |  34 +-
 .../cache/GridCacheEvictionManager.java         |   1 -
 .../cache/distributed/dht/GridDhtGetFuture.java |  40 +-
 .../distributed/dht/GridDhtGetSingleFuture.java |  41 +-
 .../dht/GridDhtTopologyFutureAdapter.java       | 233 ++++---
 .../dht/GridPartitionedGetFuture.java           |   2 +-
 .../dht/GridPartitionedSingleGetFuture.java     |   9 +-
 .../dht/preloader/GridDhtPartitionDemander.java |   1 -
 .../GridDhtPartitionsExchangeFuture.java        |   4 +-
 .../dht/topology/GridDhtLocalPartition.java     |   1 -
 .../distributed/near/GridNearGetFuture.java     |   2 +-
 .../persistence/metastorage/MetaStorage.java    |   7 +
 .../metastorage/MetastorageTree.java            |   8 +
 .../wal/reader/IgniteWalIteratorFactory.java    |   3 +-
 .../reader/StandaloneWalRecordsIterator.java    |   6 +-
 .../wal/reader/StrictBoundsCheckException.java  |  35 +
 .../odbc/odbc/OdbcRequestHandler.java           |  12 +-
 .../processors/odbc/odbc/OdbcUtils.java         |  22 +
 .../cache/ClientCacheNodePartitionsRequest.java |   3 +-
 .../processors/query/GridQueryIndexing.java     |  23 +-
 .../processors/query/GridQueryProcessor.java    | 275 ++++----
 .../processors/query/QueryKeyValueIterable.java |  53 ++
 .../processors/query/QueryKeyValueIterator.java |  68 ++
 .../visor/node/VisorNodeDataCollectorJob.java   |   3 +-
 ...IgniteClientCacheInitializationFailTest.java |  19 +-
 ...CacheResultIsNotNullOnPartitionLossTest.java |  23 +-
 .../IgniteCachePartitionLossPolicySelfTest.java | 651 +++++++------------
 .../metastorage/IgniteMetaStorageBasicTest.java |  77 +++
 .../processors/query/h2/IgniteH2Indexing.java   | 285 ++------
 ...CacheAbstractQueryDetailMetricsSelfTest.java |  15 +-
 .../CacheAbstractQueryMetricsSelfTest.java      |  13 -
 ...niteCacheLockPartitionOnAffinityRunTest.java |   2 +
 ...ndexingCachePartitionLossPolicySelfTest.java |   2 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 639 +-----------------
 ...H2ResultSetIteratorNullifyOnEndSelfTest.java | 186 ------
 .../cpp/odbc-test/src/meta_queries_test.cpp     |  43 ++
 .../cpp/odbc-test/src/sql_get_info_test.cpp     |   2 +-
 .../cpp/odbc/src/config/connection_info.cpp     |   2 +-
 .../thin-client-test/src/cache_client_test.cpp  |  27 +
 42 files changed, 1055 insertions(+), 1822 deletions(-)
----------------------------------------------------------------------



[03/50] [abbrv] ignite git commit: IGNITE-10293 Collect caches info only on active cluster.

Posted by sb...@apache.org.
IGNITE-10293 Collect caches info only on active cluster.


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

Branch: refs/heads/ignite-10044
Commit: eed26bcfb26d58372459f84173c4d0969dc08c8a
Parents: a9c5aed
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Nov 16 16:17:01 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Nov 16 16:17:01 2018 +0700

----------------------------------------------------------------------
 .../ignite/internal/visor/node/VisorNodeDataCollectorJob.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/eed26bcf/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
index 5fab8d1..9025ed0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
@@ -335,7 +335,8 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
         if (debug)
             start0 = log(ignite.log(), "Collected memory metrics", getClass(), start0);
 
-        caches(res, arg);
+        if (ignite.cluster().active())
+            caches(res, arg);
 
         if (debug)
             start0 = log(ignite.log(), "Collected caches", getClass(), start0);


[36/50] [abbrv] ignite git commit: IGNITE-8765: SQL: Fixed event type when SQL or SQL_FIELDS query is executed. This closes #5421.

Posted by sb...@apache.org.
IGNITE-8765: SQL: Fixed event type when SQL or SQL_FIELDS query is executed. This closes #5421.


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

Branch: refs/heads/ignite-10044
Commit: ae7a1d0b8cf51f6ccd13fc5d945c37a0d019e11b
Parents: 2c23d68
Author: Pavel Kuznetsov <pa...@gmail.com>
Authored: Tue Nov 20 12:15:38 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Nov 20 12:15:38 2018 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java      | 18 ++++++++++++------
 .../IgniteCacheReplicatedQuerySelfTest.java       |  9 ---------
 2 files changed, 12 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ae7a1d0b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 3842d77..82fd51b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -2175,7 +2175,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                             idx.querySqlFields(schemaName, qry, cliCtx, keepBinary, failOnMultipleStmts, null, cancel);
 
                         if (cctx != null)
-                            sendQueryExecutedEvent(qry.getSql(), qry.getArgs(), cctx);
+                            sendQueryExecutedEvent(qry.getSql(), qry.getArgs(), cctx, qryType);
 
                         return res;
                     }
@@ -2454,15 +2454,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
     /**
      * @param sqlQry Sql query.
-     * @param params Params.
-     */
-    private void sendQueryExecutedEvent(String sqlQry, Object[] params, GridCacheContext<?, ?> cctx) {
+     * @param params Params of the query.
+     * @param cctx cache context.
+     * @param qryType actual query type, usually either SQL or SQL_FIELDS.
+     */
+    private void sendQueryExecutedEvent(
+            String sqlQry,
+            Object[] params,
+            GridCacheContext<?, ?> cctx,
+            GridCacheQueryType qryType) {
         if (cctx.events().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
             ctx.event().record(new CacheQueryExecutedEvent<>(
                 ctx.discovery().localNode(),
-                "SQL query executed.",
+                qryType.name() + " query executed.",
                 EVT_CACHE_QUERY_EXECUTED,
-                CacheQueryType.SQL.name(),
+                qryType.name(),
                 cctx.name(),
                 null,
                 sqlQry,

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae7a1d0b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
index 8635416..f741ef2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
@@ -422,15 +422,6 @@ public class IgniteCacheReplicatedQuerySelfTest extends IgniteCacheAbstractQuery
     }
 
     /**
-     * @throws Exception If failed.
-     */
-    @Override public void testSqlQueryEvents() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-8765");
-
-        super.testSqlQueryEvents();
-    }
-
-    /**
      * Cache key.
      */
     public static class CacheKey implements Externalizable {


[31/50] [abbrv] ignite git commit: IGNITE-10229 Web Console: Cleanup ClusterLoginService on disconnect from cluster.

Posted by sb...@apache.org.
IGNITE-10229 Web Console: Cleanup ClusterLoginService on disconnect from cluster.


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

Branch: refs/heads/ignite-10044
Commit: 351d0807529a96aa524ae7d02b08f33587c501b6
Parents: a198b3f
Author: Alexander Kalinin <ve...@yandex.ru>
Authored: Tue Nov 20 08:57:13 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Nov 20 08:57:13 2018 +0700

----------------------------------------------------------------------
 .../frontend/app/modules/agent/AgentManager.service.js   | 11 ++++++++++-
 .../modules/agent/components/cluster-login/service.js    |  5 +++++
 2 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/351d0807/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
index c59d6a1..dad625d 100644
--- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
+++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
@@ -30,6 +30,7 @@ import Worker from './decompress.worker';
 import SimpleWorkerPool from '../../utils/SimpleWorkerPool';
 import maskNull from 'app/core/utils/maskNull';
 
+import {CancellationError} from 'app/errors/CancellationError';
 import {ClusterSecretsManager} from './types/ClusterSecretsManager';
 import ClusterLoginService from './components/cluster-login/service';
 
@@ -379,11 +380,13 @@ export default class AgentManager {
 
                     case State.AGENT_DISCONNECTED:
                         this.agentModal.agentDisconnected(this.backText, this.backState);
+                        this.ClusterLoginSrv.cancel();
 
                         break;
 
                     case State.CLUSTER_DISCONNECTED:
                         this.agentModal.clusterDisconnected(this.backText, this.backState);
+                        this.ClusterLoginSrv.cancel();
 
                         break;
 
@@ -544,7 +547,13 @@ export default class AgentManager {
 
                 return {cluster, credentials: {}};
             })
-            .then(({cluster, credentials}) => this._executeOnActiveCluster(cluster, credentials, event, params));
+            .then(({cluster, credentials}) => this._executeOnActiveCluster(cluster, credentials, event, params))
+            .catch((err) => {
+                if (err instanceof CancellationError)
+                    return;
+
+                return err;
+            });
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/351d0807/modules/web-console/frontend/app/modules/agent/components/cluster-login/service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/components/cluster-login/service.js b/modules/web-console/frontend/app/modules/agent/components/cluster-login/service.js
index 955d0a3..406152d 100644
--- a/modules/web-console/frontend/app/modules/agent/components/cluster-login/service.js
+++ b/modules/web-console/frontend/app/modules/agent/components/cluster-login/service.js
@@ -77,4 +77,9 @@ export default class ClusterLoginService {
                 modal.hide();
             });
     }
+
+    cancel() {
+        if (this.deferred)
+            this.deferred.reject(new CancellationError());
+    }
 }


[50/50] [abbrv] ignite git commit: ignite-10044

Posted by sb...@apache.org.
ignite-10044


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

Branch: refs/heads/ignite-10044
Commit: 763ea3253ec6f2963ba09678ef0073931ee3daf9
Parents: 5466373
Author: sboikov <sb...@apache.org>
Authored: Tue Nov 20 17:53:48 2018 +0300
Committer: sboikov <sb...@apache.org>
Committed: Tue Nov 20 17:53:48 2018 +0300

----------------------------------------------------------------------
 .../preloader/GridDhtPartitionsExchangeFuture.java |  2 ++
 .../dht/topology/GridClientPartitionTopology.java  |  3 ---
 .../IgniteCachePartitionLossPolicySelfTest.java    | 17 +++++++++--------
 3 files changed, 11 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/763ea325/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 0cef226..0d594c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -3001,6 +3001,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      * Detect lost partitions.
      *
      * @param resTopVer Result topology version.
+     * @param crd {@code True} if run on coordinator.
      */
     private void detectLostPartitions(AffinityTopologyVersion resTopVer, boolean crd) {
         boolean detected = false;
@@ -3042,6 +3043,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
     /**
      * @param cacheNames Cache names.
+     * @param crd {@code True} if run on coordinator.
      */
     private void resetLostPartitions(Collection<String> cacheNames, boolean crd) {
         assert !exchCtx.mergeExchanges();

http://git-wip-us.apache.org/repos/asf/ignite/blob/763ea325/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
index a28b3b0..20fa45c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
@@ -32,10 +32,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
-import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
@@ -62,7 +60,6 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.EVICTED;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.LOST;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING;

http://git-wip-us.apache.org/repos/asf/ignite/blob/763ea325/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
index 226ae22..be16e60 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java
@@ -526,17 +526,18 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe
 
             info("Newly started node: " + grd.cluster().localNode().id());
 
+            verifyLostPartitions(grd, lostParts);
+
             // Check that partition state does not change after we start each node.
-            // TODO With persistence enabled LOST partitions become OWNING after a node joins back - https://issues.apache.org/jira/browse/IGNITE-10044.
-            if (!isPersistenceEnabled) {
-                for (Ignite ig : G.allGrids()) {
-                    verifyCacheOps(canWrite, safe, ig);
+            for (Ignite ig : G.allGrids()) {
+                verifyLostPartitions(ig, lostParts);
 
-                    // TODO Query effectively waits for rebalance due to https://issues.apache.org/jira/browse/IGNITE-10057
-                    // TODO and after resetLostPartition there is another OWNING copy in the cluster due to https://issues.apache.org/jira/browse/IGNITE-10058.
-                    // TODO Uncomment after https://issues.apache.org/jira/browse/IGNITE-10058 is fixed.
+                verifyCacheOps(canWrite, safe, ig);
+
+                // TODO Query effectively waits for rebalance due to https://issues.apache.org/jira/browse/IGNITE-10057
+                // TODO and after resetLostPartition there is another OWNING copy in the cluster due to https://issues.apache.org/jira/browse/IGNITE-10058.
+                // TODO Uncomment after https://issues.apache.org/jira/browse/IGNITE-10058 is fixed.
 //                    validateQuery(safe, ig);
-                }
             }
         }
 


[15/50] [abbrv] ignite git commit: IGNITE-10303: SQL test failure.

Posted by sb...@apache.org.
IGNITE-10303: SQL test failure.


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

Branch: refs/heads/ignite-10044
Commit: 28cb3a09526db7042e4f5a3d8bf3d84406b56a60
Parents: 43ee832
Author: devozerov <pp...@gmail.com>
Authored: Sat Nov 17 22:27:01 2018 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Sat Nov 17 22:27:01 2018 +0300

----------------------------------------------------------------------
 .../internal/processors/query/h2/ConnectionManager.java     | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/28cb3a09/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
index 09400c8..1d62b87 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
@@ -62,10 +62,11 @@ public class ConnectionManager {
     private static final Long CONN_CLEANUP_PERIOD = 2000L;
 
     /** The period of clean up the statement cache. */
-    private static final Long STMT_CLEANUP_PERIOD = Long.getLong(IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD, 10_000);
+    @SuppressWarnings("FieldCanBeLocal")
+    private final Long stmtCleanupPeriod = Long.getLong(IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD, 10_000);
 
     /** The timeout to remove entry from the statement cache if the thread doesn't perform any queries. */
-    private static final Long STMT_TIMEOUT = Long.getLong(IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT, 600 * 1000);
+    private final Long stmtTimeout = Long.getLong(IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT, 600 * 1000);
 
     /*
      * Initialize system properties for H2.
@@ -151,7 +152,7 @@ public class ConnectionManager {
             @Override public void run() {
                 cleanupStatements();
             }
-        }, STMT_CLEANUP_PERIOD, STMT_CLEANUP_PERIOD);
+        }, stmtCleanupPeriod, stmtCleanupPeriod);
 
         connCleanupTask = ctx.timeout().schedule(new Runnable() {
             @Override public void run() {
@@ -436,7 +437,7 @@ public class ConnectionManager {
 
                 it.remove();
             }
-            else if (now - entry.getValue().statementCache().lastUsage() > STMT_TIMEOUT)
+            else if (now - entry.getValue().statementCache().lastUsage() > stmtTimeout)
                 entry.getValue().clearStatementCache();
         }
     }


[37/50] [abbrv] ignite git commit: IGNITE-9554 Web Agent: Exclude caches created via SQL from random data population in "Demo" mode.

Posted by sb...@apache.org.
IGNITE-9554 Web Agent: Exclude caches created via SQL from random data population in "Demo" mode.


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

Branch: refs/heads/ignite-10044
Commit: 4fb1fc37a5016fb0386cb654cc527b685cb78349
Parents: ae7a1d0
Author: Vasiliy Sisko <vs...@gridgain.com>
Authored: Tue Nov 20 16:30:35 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Nov 20 16:30:35 2018 +0700

----------------------------------------------------------------------
 .../service/DemoRandomCacheLoadService.java     | 22 +++++++++++---------
 1 file changed, 12 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4fb1fc37/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoRandomCacheLoadService.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoRandomCacheLoadService.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoRandomCacheLoadService.java
index e6242fc..5b73e96 100644
--- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoRandomCacheLoadService.java
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoRandomCacheLoadService.java
@@ -29,6 +29,8 @@ import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.services.Service;
 import org.apache.ignite.services.ServiceContext;
 
+import static org.apache.ignite.internal.processors.query.QueryUtils.DFLT_SCHEMA;
+
 /**
  * Demo service. Create cache and populate it by random int pairs.
  */
@@ -78,19 +80,19 @@ public class DemoRandomCacheLoadService implements Service {
             @Override public void run() {
                 try {
                     for (String cacheName : ignite.cacheNames()) {
-                        if (!DemoCachesLoadService.DEMO_CACHES.contains(cacheName)) {
-                            IgniteCache<Integer, Integer> cache = ignite.cache(cacheName);
+                        IgniteCache<Integer, Integer> cache = ignite.cache(cacheName);
 
-                            if (cache != null) {
-                                for (int i = 0, n = 1; i < cnt; i++, n++) {
-                                    Integer key = rnd.nextInt(RND_CNT);
-                                    Integer val = rnd.nextInt(RND_CNT);
+                        if (cache != null &&
+                            !DemoCachesLoadService.DEMO_CACHES.contains(cacheName) &&
+                            !DFLT_SCHEMA.equalsIgnoreCase(cache.getConfiguration(CacheConfiguration.class).getSqlSchema())) {
+                            for (int i = 0, n = 1; i < cnt; i++, n++) {
+                                Integer key = rnd.nextInt(RND_CNT);
+                                Integer val = rnd.nextInt(RND_CNT);
 
-                                    cache.put(key, val);
+                                cache.put(key, val);
 
-                                    if (rnd.nextInt(100) < 30)
-                                        cache.remove(key);
-                                }
+                                if (rnd.nextInt(100) < 30)
+                                    cache.remove(key);
                             }
                         }
                     }


[29/50] [abbrv] ignite git commit: IGNITE-10321 Fix flaky test CacheContinuousWithTransformerReplicatedSelfTest.LocalEventListener - Fixes #5427.

Posted by sb...@apache.org.
IGNITE-10321 Fix flaky test CacheContinuousWithTransformerReplicatedSelfTest.LocalEventListener - Fixes #5427.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


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

Branch: refs/heads/ignite-10044
Commit: c711d4f6f7ecc6ad59643a2a82d7d487d8d45a90
Parents: acfdcda
Author: ibessonov <be...@gmail.com>
Authored: Mon Nov 19 17:02:48 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Nov 19 17:02:48 2018 +0300

----------------------------------------------------------------------
 .../CacheContinuousWithTransformerReplicatedSelfTest.java        | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c711d4f6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousWithTransformerReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousWithTransformerReplicatedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousWithTransformerReplicatedSelfTest.java
index 60e2f80..3768aad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousWithTransformerReplicatedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousWithTransformerReplicatedSelfTest.java
@@ -492,10 +492,10 @@ public class CacheContinuousWithTransformerReplicatedSelfTest extends GridCommon
         /** {@inheritDoc} */
         @Override public void onUpdated(Iterable<? extends String> events) throws CacheEntryListenerException {
             for (String evt : events) {
+                cnt.incrementAndGet();
+
                 if (evt.startsWith(SARAH_CONNOR))
                     cntLatch.countDown();
-
-                cnt.incrementAndGet();
             }
         }
     }


[20/50] [abbrv] ignite git commit: IGNITE-10142 Scale factor added to several tests from Cache 8 suite. - Fixes #5253.

Posted by sb...@apache.org.
IGNITE-10142 Scale factor added to several tests from Cache 8 suite. - Fixes #5253.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


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

Branch: refs/heads/ignite-10044
Commit: c9906aab8efa2c70a164fb593d9e22ef1edaa9ec
Parents: 166e87e
Author: ibessonov <be...@gmail.com>
Authored: Mon Nov 19 12:50:56 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Nov 19 12:50:56 2018 +0300

----------------------------------------------------------------------
 .../GridCacheRebalancingSyncSelfTest.java       | 79 +++++++++++++-------
 .../GridCacheConcurrentEvictionsSelfTest.java   | 21 +++---
 2 files changed, 65 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c9906aab/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
index bb08650..a95ad43 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
@@ -21,11 +21,15 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -35,12 +39,12 @@ import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.util.lang.GridAbsPredicateX;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.PA;
@@ -54,6 +58,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.GridTestUtils.SF;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
@@ -67,10 +72,10 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
     protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** */
-    private static final int TEST_SIZE = 100_000;
+    private static final int TEST_SIZE = SF.applyLB(100_000, 10_000);
 
     /** */
-    private static final long TOPOLOGY_STILLNESS_TIME = 30_000L;
+    private static final long TOPOLOGY_STILLNESS_TIME = SF.applyLB(30_000, 5_000);
 
     /** partitioned cache name. */
     protected static final String CACHE_NAME_DHT_PARTITIONED = "cacheP";
@@ -133,7 +138,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         cachePCfg2.setRebalanceMode(CacheRebalanceMode.SYNC);
         cachePCfg2.setBackups(1);
         cachePCfg2.setRebalanceOrder(2);
-        cachePCfg2.setRebalanceDelay(5000);
+        cachePCfg2.setRebalanceDelay(SF.applyLB(5000, 500));
 
         CacheConfiguration<Integer, Integer> cacheRCfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME);
 
@@ -177,12 +182,16 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
      * @param iter Iteration.
      */
     protected void generateData(Ignite ignite, String name, int from, int iter) {
-        for (int i = from; i < from + TEST_SIZE; i++) {
-            if ((i + 1) % (TEST_SIZE / 10) == 0)
-                log.info("Prepared " + (i + 1) * 100 / (TEST_SIZE) + "% entries. [count=" + TEST_SIZE +
-                    ", iteration=" + iter + ", cache=" + name + "]");
+        try (IgniteDataStreamer<Integer, Integer> dataStreamer = ignite.dataStreamer(name)) {
+            dataStreamer.allowOverwrite(true);
 
-            ignite.cache(name).put(i, i + name.hashCode() + iter);
+            for (int i = from; i < from + TEST_SIZE; i++) {
+                if ((i + 1) % (TEST_SIZE / 10) == 0)
+                    log.info("Prepared " + (i + 1) * 100 / (TEST_SIZE) + "% entries. [count=" + TEST_SIZE +
+                        ", iteration=" + iter + ", cache=" + name + "]");
+
+                dataStreamer.addData(i, i + name.hashCode() + iter);
+            }
         }
     }
 
@@ -192,26 +201,46 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
      * @param iter Iteration.
      */
     protected void checkData(Ignite ignite, int from, int iter) {
-        checkData(ignite, CACHE_NAME_DHT_PARTITIONED, from, iter);
-        checkData(ignite, CACHE_NAME_DHT_PARTITIONED_2, from, iter);
-        checkData(ignite, CACHE_NAME_DHT_REPLICATED, from, iter);
-        checkData(ignite, CACHE_NAME_DHT_REPLICATED_2, from, iter);
+        checkData(ignite, CACHE_NAME_DHT_PARTITIONED, from, iter, true);
+        checkData(ignite, CACHE_NAME_DHT_PARTITIONED_2, from, iter, true);
+        checkData(ignite, CACHE_NAME_DHT_REPLICATED, from, iter, true);
+        checkData(ignite, CACHE_NAME_DHT_REPLICATED_2, from, iter, true);
     }
 
     /**
      * @param ignite Ignite.
+     * @param name Cache name.
      * @param from Start from key.
      * @param iter Iteration.
-     * @param name Cache name.
+     * @param scan If true then "scan" query will be used instead of "get" in a loop. Should be "false" when run in
+     *      parallel with other operations. Otherwise should be "true", because it's much faster in such situations.
      */
-    protected void checkData(Ignite ignite, String name, int from, int iter) {
-        for (int i = from; i < from + TEST_SIZE; i++) {
-            if ((i + 1) % (TEST_SIZE / 10) == 0)
-                log.info("<" + name + "> Checked " + (i + 1) * 100 / (TEST_SIZE) + "% entries. [count=" + TEST_SIZE +
-                    ", iteration=" + iter + ", cache=" + name + "]");
-
-            assertEquals("Value does not match [key=" + i + ", cache=" + name + ']',
-                ignite.cache(name).get(i), i + name.hashCode() + iter);
+    protected void checkData(Ignite ignite, String name, int from, int iter, boolean scan) {
+        IgniteCache<Integer, Integer> cache = ignite.cache(name);
+
+        if (scan) {
+            AtomicInteger cnt = new AtomicInteger();
+
+            cache.query(new ScanQuery<Integer, Integer>((k, v) -> k >= from && k < from + TEST_SIZE)).forEach(entry -> {
+                if (cnt.incrementAndGet() % (TEST_SIZE / 10) == 0)
+                    log.info("<" + name + "> Checked " + cnt.get() * 100 / TEST_SIZE + "% entries. [count=" +
+                        TEST_SIZE + ", iteration=" + iter + ", cache=" + name + "]");
+
+                assertEquals("Value does not match [key=" + entry.getKey() + ", cache=" + name + ']',
+                    entry.getValue().intValue(), entry.getKey() + name.hashCode() + iter);
+            });
+
+            assertEquals(TEST_SIZE, cnt.get());
+        }
+        else {
+            for (int i = from; i < from + TEST_SIZE; i++) {
+                if ((i + 1) % (TEST_SIZE / 10) == 0)
+                    log.info("<" + name + "> Checked " + (i + 1) * 100 / (TEST_SIZE) + "% entries. [count=" +
+                        TEST_SIZE + ", iteration=" + iter + ", cache=" + name + "]");
+
+                assertEquals("Value does not match [key=" + i + ", cache=" + name + ']',
+                    cache.get(i).intValue(), i + name.hashCode() + iter);
+            }
         }
     }
 
@@ -309,7 +338,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         Thread t2 = new Thread() {
             @Override public void run() {
                 while (!concurrentStartFinished)
-                    checkData(ignite, CACHE_NAME_DHT_PARTITIONED, 0, 0);
+                    checkData(ignite, CACHE_NAME_DHT_PARTITIONED, 0, 0, false);
             }
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c9906aab/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
index 45d98bf..c986961 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils.SF;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
@@ -66,7 +67,7 @@ public class GridCacheConcurrentEvictionsSelfTest extends GridCommonAbstractTest
         c.getTransactionConfiguration().setDefaultTxConcurrency(PESSIMISTIC);
         c.getTransactionConfiguration().setDefaultTxIsolation(READ_COMMITTED);
 
-        CacheConfiguration cc = defaultCacheConfiguration();
+        CacheConfiguration<?, ?> cc = defaultCacheConfiguration();
 
         cc.setCacheMode(mode);
 
@@ -105,8 +106,8 @@ public class GridCacheConcurrentEvictionsSelfTest extends GridCommonAbstractTest
         plc.setMaxSize(1000);
 
         this.plc = plc;
-        warmUpPutsCnt = 100000;
-        iterCnt = 100000;
+        warmUpPutsCnt = SF.applyLB(100_000, 10_000);
+        iterCnt = SF.applyLB(100_000, 10_000);
 
         checkConcurrentPuts();
     }
@@ -121,8 +122,8 @@ public class GridCacheConcurrentEvictionsSelfTest extends GridCommonAbstractTest
         plc.setMaxSize(1000);
 
         this.plc = plc;
-        warmUpPutsCnt = 100000;
-        iterCnt = 100000;
+        warmUpPutsCnt = SF.applyLB(100_000, 10_000);
+        iterCnt = SF.applyLB(100_000, 10_000);
 
         checkConcurrentPuts();
     }
@@ -137,8 +138,8 @@ public class GridCacheConcurrentEvictionsSelfTest extends GridCommonAbstractTest
         plc.setMaxSize(1000);
 
         this.plc = plc;
-        warmUpPutsCnt = 100000;
-        iterCnt = 100000;
+        warmUpPutsCnt = SF.applyLB(100_000, 10_000);
+        iterCnt = SF.applyLB(100_000, 10_000);
 
         checkConcurrentPuts();
     }
@@ -166,19 +167,19 @@ public class GridCacheConcurrentEvictionsSelfTest extends GridCommonAbstractTest
 
             final AtomicInteger idx = new AtomicInteger();
 
-            int threadCnt = 30;
+            int threadCnt = SF.applyLB(30, 8);
 
             long start = System.currentTimeMillis();
 
             IgniteInternalFuture<?> fut = multithreadedAsync(
                 new Callable<Object>() {
-                    @Override public Object call() throws Exception {
+                    @Override public Object call() {
                         for (int i = 0; i < iterCnt; i++) {
                             int j = idx.incrementAndGet();
 
                             cache.put(j, j);
 
-                            if (i != 0 && i % 10000 == 0)
+                            if (i != 0 && i % 1000 == 0)
                                 // info("Puts count: " + i);
                                 info("Stats [putsCnt=" + i + ", size=" + cache.size() + ']');
                         }


[49/50] [abbrv] ignite git commit: merge

Posted by sb...@apache.org.
merge


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

Branch: refs/heads/ignite-10044
Commit: 5466373deae48ab79e5aa12d18580dea8e823bd9
Parents: 5f9bdc7 8c08de7
Author: sboikov <sb...@apache.org>
Authored: Tue Nov 20 17:38:09 2018 +0300
Committer: sboikov <sb...@apache.org>
Committed: Tue Nov 20 17:38:09 2018 +0300

----------------------------------------------------------------------
 .../IgnitePKIndexesMigrationToUnwrapPkTest.java |   6 +-
 .../apache/ignite/IgniteSystemProperties.java   |   7 +
 .../processors/cache/CacheGroupContext.java     | 148 -----
 .../processors/cache/GridCacheMapEntry.java     |  12 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../processors/cache/GridCacheProcessor.java    |  44 +-
 .../cache/IgniteCacheOffheapManager.java        |  15 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  11 +-
 .../cache/PartitionUpdateCounter.java           |  28 +-
 .../GridDistributedTxRemoteAdapter.java         |   2 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |  53 +-
 .../dht/GridDhtTxAbstractEnlistFuture.java      |  10 +-
 .../GridDhtPartitionsExchangeFuture.java        |  62 +-
 .../topology/GridClientPartitionTopology.java   |   8 +-
 .../dht/topology/GridDhtLocalPartition.java     |   7 +-
 .../dht/topology/GridDhtPartitionTopology.java  |   9 +-
 .../topology/GridDhtPartitionTopologyImpl.java  |  51 +-
 .../cache/mvcc/MvccCachingManager.java          |  30 +-
 .../persistence/DatabaseLifecycleListener.java  |  43 +-
 .../GridCacheDatabaseSharedManager.java         | 425 ++++++-------
 .../persistence/GridCacheOffheapManager.java    | 145 ++++-
 .../IgniteCacheDatabaseSharedManager.java       |  12 +-
 .../cache/persistence/wal/FileWALPointer.java   |   2 +-
 .../wal/serializer/RecordDataV1Serializer.java  |   2 +-
 .../continuous/CacheContinuousQueryManager.java |  33 ++
 .../cache/transactions/IgniteTxAdapter.java     |   3 +
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../continuous/GridContinuousProcessor.java     |   6 +-
 .../IgniteCachePartitionLossPolicySelfTest.java | 590 +++++++++++++------
 .../db/IgniteLogicalRecoveryTest.java           | 284 +++++----
 .../file/IgnitePdsDiskErrorsRecoveringTest.java |  58 --
 .../IgniteNodeStoppedDuringDisableWALTest.java  |   3 +-
 .../persistence/db/wal/WalCompactionTest.java   |  13 +-
 ...tinuousQueryAsyncFailoverMvccTxSelfTest.java |  53 ++
 ...ContinuousQueryFailoverAbstractSelfTest.java |  90 ++-
 ...usQueryFailoverMvccTxReplicatedSelfTest.java |  31 +
 ...heContinuousQueryFailoverMvccTxSelfTest.java |  48 ++
 .../processors/query/h2/H2TableDescriptor.java  |  28 +-
 .../internal/processors/query/h2/H2Utils.java   |  13 +
 .../index/ComplexPrimaryKeyUnwrapSelfTest.java  |  18 +-
 .../ComplexSecondaryKeyUnwrapSelfTest.java      | 249 ++++++++
 .../mvcc/CacheMvccBasicContinuousQueryTest.java | 363 ++++++++++++
 .../IgniteBinaryCacheQueryTestSuite.java        |   2 +
 .../IgniteCacheQuerySelfTestSuite4.java         |   6 +
 44 files changed, 2159 insertions(+), 870 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5466373d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/5466373d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/5466373d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------


[34/50] [abbrv] ignite git commit: IGNITE-10162 Fixed failing testTwoObjectsTextSearch - Fixes #5347.

Posted by sb...@apache.org.
IGNITE-10162 Fixed failing testTwoObjectsTextSearch - Fixes #5347.

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/2a59d831
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/2a59d831
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/2a59d831

Branch: refs/heads/ignite-10044
Commit: 2a59d831158ea3dceb5d053d5d1f67486def99b5
Parents: 7da1864
Author: Alexey Platonov <ap...@gmail.com>
Authored: Tue Nov 20 11:14:01 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 20 11:15:39 2018 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheAbstractQuerySelfTest.java        | 13 ++++++++-----
 1 file changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2a59d831/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index a00cc83..bf6d6da 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -1160,13 +1160,16 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
 
     /**
      * JUnit.
-     *
-     * @throws Exception In case of error.
      */
-    public void _testTwoObjectsTextSearch() throws Exception {
-        fail("http://atlassian.gridgain.com/jira/browse/GG-11216");
+    public void testTwoObjectsTextSearch() {
+        CacheConfiguration<Object, Object> conf = new CacheConfiguration<>(cacheConfiguration());
+
+        conf.setQueryEntities(Arrays.asList(
+           new QueryEntity(Integer.class, ObjectValue.class),
+           new QueryEntity(String.class, ObjectValueOther.class)
+        ));
 
-        IgniteCache<Object, Object> c = jcache(Object.class, Object.class);
+        IgniteCache<Object, Object> c = jcache(ignite(), conf, Object.class, Object.class);
 
         c.put(1, new ObjectValue("ObjectValue str", 1));
         c.put("key", new ObjectValueOther("ObjectValueOther str"));


[09/50] [abbrv] ignite git commit: IGNITE-10156 Fixed invalid conversion of DynamicCacheDescriptor to StoredCacheData - Fixes #5332.

Posted by sb...@apache.org.
IGNITE-10156 Fixed invalid conversion of DynamicCacheDescriptor to StoredCacheData - Fixes #5332.

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/2311afd3
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/2311afd3
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/2311afd3

Branch: refs/heads/ignite-10044
Commit: 2311afd32e2afc0aa0ebc350e6a3d964206b477f
Parents: 34c059a
Author: Anton Kalashnikov <ka...@yandex.ru>
Authored: Fri Nov 16 19:11:16 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Nov 16 19:11:16 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/CachesRegistry.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2311afd3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java
index fe55f97..d37f69c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java
@@ -280,7 +280,7 @@ public class CachesRegistry {
      */
     private IgniteInternalFuture<?> persistCacheConfigurations(List<DynamicCacheDescriptor> cacheDescriptors) {
         List<StoredCacheData> cacheConfigsToPersist = cacheDescriptors.stream()
-            .map(cacheDesc -> new StoredCacheData(cacheDesc.cacheConfiguration()).sql(cacheDesc.sql()))
+            .map(DynamicCacheDescriptor::toStoredData)
             .collect(Collectors.toList());
 
         // Pre-create cache work directories if they don't exist.


[48/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-10044

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


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

Branch: refs/heads/ignite-10044
Commit: 5f9bdc773607479967a07d5ad4d5159c08aa6b4c
Parents: 93b9967 c8fd687
Author: sboikov <sb...@apache.org>
Authored: Tue Nov 20 17:36:26 2018 +0300
Committer: sboikov <sb...@apache.org>
Committed: Tue Nov 20 17:36:26 2018 +0300

----------------------------------------------------------------------
 ...ggedLogisticRegressionSGDTrainerExample.java | 108 +++++
 .../logistic/bagged/package-info.java           |  22 +
 .../ignite/codegen/MessageCodeGenerator.java    |  20 +-
 .../ignite/internal/GridJobCancelRequest.java   |   2 +-
 .../ignite/internal/GridJobExecuteRequest.java  |   4 +-
 .../ignite/internal/GridJobExecuteResponse.java |   4 +-
 .../ignite/internal/GridJobSiblingsRequest.java |   2 +-
 .../internal/GridJobSiblingsResponse.java       |   2 +-
 .../ignite/internal/GridTaskCancelRequest.java  |   2 +-
 .../ignite/internal/GridTaskSessionRequest.java |   2 +-
 .../internal/direct/DirectMessageReader.java    |  30 ++
 .../internal/direct/DirectMessageWriter.java    |  26 ++
 .../direct/stream/DirectByteBufferStream.java   |  11 +
 .../stream/v1/DirectByteBufferStreamImplV1.java |  13 +
 .../stream/v2/DirectByteBufferStreamImplV2.java |  19 +-
 .../stream/v3/DirectByteBufferStreamImplV3.java | 298 ++++++++++++
 .../checkpoint/GridCheckpointRequest.java       |   2 +-
 .../managers/communication/GridIoManager.java   |   2 +-
 .../communication/GridIoUserMessage.java        |   2 +-
 .../communication/IgniteIoTestMessage.java      |   2 +
 .../deployment/GridDeploymentInfoBean.java      |   2 +-
 .../deployment/GridDeploymentRequest.java       |   2 +-
 .../deployment/GridDeploymentResponse.java      |   2 +-
 .../encryption/GridEncryptionManager.java       |   2 +-
 .../eventstorage/GridEventStorageMessage.java   |   2 +-
 .../affinity/AffinityTopologyVersion.java       |   2 +-
 .../affinity/GridAffinityAssignmentCache.java   |  37 +-
 .../UserAuthenticateResponseMessage.java        |   2 +-
 .../cache/CacheEntryInfoCollection.java         |   2 +-
 .../processors/cache/CacheEntryPredicate.java   |   2 +-
 .../cache/CacheEntryPredicateAdapter.java       |   2 +-
 .../cache/CacheEntryPredicateContainsValue.java |   2 +-
 .../cache/CacheEntryPredicateHasValue.java      |   2 +-
 .../cache/CacheEntryPredicateNoValue.java       |   2 +-
 .../cache/CacheEntrySerializablePredicate.java  |   2 +-
 .../processors/cache/CacheEvictionEntry.java    |   2 +-
 .../cache/CacheInvokeDirectResult.java          |   2 +-
 .../internal/processors/cache/CacheObject.java  |   2 +-
 .../processors/cache/CacheObjectAdapter.java    |   2 +-
 .../cache/CacheObjectByteArrayImpl.java         |   2 +-
 .../processors/cache/CacheObjectImpl.java       |   2 +-
 .../processors/cache/CachesRegistry.java        |   2 +-
 .../processors/cache/ClusterCachesInfo.java     |   8 -
 .../cache/GridCacheAffinityManager.java         |  14 +-
 .../cache/GridCacheGroupIdMessage.java          |   6 +-
 .../processors/cache/GridCacheIdMessage.java    |   6 +-
 .../processors/cache/GridCacheIoManager.java    |  17 +-
 .../processors/cache/GridCacheMessage.java      |  41 +-
 .../GridCachePartitionExchangeManager.java      | 202 +++++++--
 .../GridChangeGlobalStateMessageResponse.java   |  10 +-
 .../processors/cache/KeyCacheObject.java        |   2 +-
 .../processors/cache/WalStateAckMessage.java    |   2 +
 .../distributed/GridCacheTtlUpdateRequest.java  |  30 +-
 .../distributed/GridCacheTxRecoveryRequest.java |  26 +-
 .../GridCacheTxRecoveryResponse.java            |  14 +-
 .../distributed/GridDistributedBaseMessage.java |  16 +-
 .../distributed/GridDistributedLockRequest.java |  54 +--
 .../GridDistributedLockResponse.java            |  14 +-
 .../GridDistributedTxFinishRequest.java         |  62 +--
 .../GridDistributedTxFinishResponse.java        |  18 +-
 .../GridDistributedTxPrepareRequest.java        |  54 +--
 .../GridDistributedTxPrepareResponse.java       |  14 +-
 .../GridDistributedUnlockRequest.java           |   6 +-
 .../dht/ClientCacheDhtTopologyFuture.java       |   5 +
 .../dht/GridDhtAffinityAssignmentRequest.java   |  18 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |  26 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  10 +-
 .../distributed/dht/GridDhtLockRequest.java     |  50 +-
 .../distributed/dht/GridDhtLockResponse.java    |  18 +-
 .../distributed/dht/GridDhtTopologyFuture.java  |   6 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |   7 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |  54 +--
 .../dht/GridDhtTxFinishResponse.java            |  14 +-
 .../dht/GridDhtTxOnePhaseCommitAckRequest.java  |   6 +-
 .../dht/GridDhtTxPrepareRequest.java            |  44 +-
 .../dht/GridDhtTxPrepareResponse.java           |  22 +-
 .../dht/GridDhtTxQueryEnlistRequest.java        |  30 +-
 .../dht/GridDhtTxQueryEnlistResponse.java       |  14 +-
 .../dht/GridDhtTxQueryFirstEnlistRequest.java   |  42 +-
 .../distributed/dht/GridDhtUnlockRequest.java   |   6 +-
 .../dht/PartitionUpdateCountersMessage.java     |   2 +
 .../GridDhtAtomicAbstractUpdateRequest.java     |  42 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   2 +-
 .../GridDhtAtomicDeferredUpdateResponse.java    |   6 +-
 .../dht/atomic/GridDhtAtomicNearResponse.java   |  22 +-
 .../GridDhtAtomicSingleUpdateRequest.java       |  18 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  66 +--
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |  18 +-
 .../GridNearAtomicAbstractUpdateRequest.java    |  34 +-
 .../GridNearAtomicCheckUpdateRequest.java       |  10 +-
 .../atomic/GridNearAtomicFullUpdateRequest.java |  38 +-
 ...GridNearAtomicSingleUpdateFilterRequest.java |   6 +-
 ...GridNearAtomicSingleUpdateInvokeRequest.java |  10 +-
 .../GridNearAtomicSingleUpdateRequest.java      |  10 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |  34 +-
 .../dht/preloader/GridDhtForceKeysRequest.java  |  22 +-
 .../dht/preloader/GridDhtForceKeysResponse.java |  22 +-
 .../GridDhtPartitionDemandLegacyMessage.java    |  40 +-
 .../GridDhtPartitionDemandMessage.java          |  32 +-
 .../preloader/GridDhtPartitionExchangeId.java   |   6 +-
 .../GridDhtPartitionSupplyMessage.java          |  44 +-
 .../GridDhtPartitionSupplyMessageV2.java        |   6 +-
 .../GridDhtPartitionsAbstractMessage.java       |  14 +-
 .../GridDhtPartitionsExchangeFuture.java        |  22 +-
 .../preloader/GridDhtPartitionsFullMessage.java |  58 +--
 .../GridDhtPartitionsSingleMessage.java         |  64 +--
 .../GridDhtPartitionsSingleRequest.java         |   6 +-
 .../dht/preloader/latch/LatchAckMessage.java    |   6 +-
 .../topology/GridDhtPartitionTopologyImpl.java  | 112 +++--
 .../distributed/near/CacheVersionedValue.java   |   2 +-
 .../distributed/near/GridNearGetRequest.java    |  58 +--
 .../distributed/near/GridNearGetResponse.java   |  34 +-
 .../distributed/near/GridNearLockRequest.java   |  46 +-
 .../distributed/near/GridNearLockResponse.java  |  30 +-
 .../near/GridNearSingleGetRequest.java          |  46 +-
 .../near/GridNearSingleGetResponse.java         |  26 +-
 .../near/GridNearTxEnlistRequest.java           |  70 +--
 .../near/GridNearTxEnlistResponse.java          |  34 +-
 .../near/GridNearTxFinishRequest.java           |  10 +-
 .../near/GridNearTxFinishResponse.java          |  14 +-
 .../near/GridNearTxPrepareRequest.java          |  34 +-
 .../near/GridNearTxPrepareResponse.java         |  70 +--
 .../near/GridNearTxQueryEnlistRequest.java      |  78 ++--
 .../near/GridNearTxQueryEnlistResponse.java     |  44 +-
 .../GridNearTxQueryResultsEnlistRequest.java    |  62 +--
 .../GridNearTxQueryResultsEnlistResponse.java   |  12 +-
 .../distributed/near/GridNearUnlockRequest.java |   2 +-
 .../mvcc/msg/MvccAckRequestTxAndQueryId.java    |   1 +
 .../cache/mvcc/msg/MvccSnapshotResponse.java    |   2 +
 .../cache/mvcc/msg/MvccTxSnapshotRequest.java   |   2 +
 .../PartitionCountersNeighborcastRequest.java   |  10 +-
 .../PartitionCountersNeighborcastResponse.java  |   6 +-
 .../GridCacheDatabaseSharedManager.java         |   5 +
 .../persistence/pagemem/PageMemoryImpl.java     |  10 +-
 .../wal/serializer/RecordDataV1Serializer.java  |  18 +-
 .../cache/query/GridCacheQueryRequest.java      | 112 ++---
 .../cache/query/GridCacheQueryResponse.java     |  26 +-
 .../CacheContinuousQueryBatchAck.java           |  16 +-
 .../continuous/CacheContinuousQueryEntry.java   |   4 +-
 .../cache/transactions/IgniteTxHandler.java     |   7 +-
 .../cache/transactions/IgniteTxKey.java         |   2 +-
 .../cache/transactions/TxLocksRequest.java      |  10 +-
 .../cache/transactions/TxLocksResponse.java     |  18 +-
 .../version/GridCacheRawVersionedEntry.java     |   4 +-
 .../continuous/GridContinuousMessage.java       |   2 +-
 .../datastreamer/DataStreamerRequest.java       |   4 +-
 .../datastreamer/DataStreamerResponse.java      |   2 +-
 .../shuffle/HadoopDirectShuffleMessage.java     |   4 +-
 .../hadoop/shuffle/HadoopShuffleMessage.java    |   2 +
 .../processors/igfs/IgfsAckMessage.java         |   2 +-
 .../internal/processors/igfs/IgfsBlockKey.java  |   2 +-
 .../processors/igfs/IgfsBlocksMessage.java      |   2 +-
 .../igfs/IgfsCommunicationMessage.java          |   2 +-
 .../processors/igfs/IgfsDeleteMessage.java      |   2 +-
 .../igfs/IgfsFragmentizerRequest.java           |   2 +-
 .../igfs/IgfsFragmentizerResponse.java          |   2 +-
 .../processors/igfs/IgfsSyncMessage.java        |   2 +-
 .../cache/ClientCacheNodePartitionsRequest.java |   1 -
 .../utils/PlatformConfigurationUtils.java       |  12 +
 .../processors/query/GridQueryIndexing.java     |   2 +-
 .../processors/query/GridQueryProcessor.java    |  22 +-
 .../internal/processors/query/QueryUtils.java   |   3 +
 .../messages/GridQueryNextPageResponse.java     |   4 +-
 .../message/SchemaOperationStatusMessage.java   |   2 +
 .../handlers/task/GridTaskResultRequest.java    |   2 +-
 .../handlers/task/GridTaskResultResponse.java   |   2 +-
 .../ignite/internal/util/GridByteArrayList.java |   2 +-
 .../ignite/internal/util/GridIntList.java       |   2 +-
 .../ignite/internal/util/GridLongList.java      |   2 +
 .../ignite/internal/util/IgniteUtils.java       | 130 +++++-
 .../internal/util/io/GridUnsafeDataOutput.java  |  76 ++--
 .../MessageCollectionItemType.java              |   5 +-
 .../extensions/communication/MessageReader.java |   9 +
 .../extensions/communication/MessageWriter.java |  10 +
 .../jobstealing/JobStealingRequest.java         |   2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  15 +-
 .../tcp/messages/HandshakeMessage.java          |   2 +
 .../tcp/messages/HandshakeMessage2.java         |   2 +
 .../tcp/messages/NodeIdMessage.java             |   2 +
 .../messages/RecoveryLastReceivedMessage.java   |   2 +
 .../optimized/OptimizedMarshallerTest.java      |  56 +--
 .../cache/CacheNoAffinityExchangeTest.java      | 218 +++++++++
 ...IgniteClientCacheInitializationFailTest.java |   2 +-
 .../IgniteClusterActivateDeactivateTest.java    |   3 +-
 .../distributed/CacheExchangeMergeTest.java     |   2 +
 .../GridCacheRebalancingSyncSelfTest.java       |  79 +++-
 .../GridCacheConcurrentEvictionsSelfTest.java   |  21 +-
 ...gnitePageMemReplaceDelayedWriteUnitTest.java |   1 +
 ...inuousWithTransformerReplicatedSelfTest.java |   4 +-
 .../database/IgniteDbPutGetAbstractTest.java    |  28 +-
 .../IgniteDbSingleNodeTinyPutGetTest.java       |  42 +-
 .../internal/util/IgniteUtilsSelfTest.java      | 147 +++++-
 .../testsuites/IgniteCacheTestSuite6.java       |   3 +
 .../processors/cache/query/QueryTable.java      |   2 +
 .../processors/query/h2/ConnectionManager.java  | 444 ++++++++++++++++++
 .../processors/query/h2/IgniteH2Indexing.java   | 451 ++-----------------
 .../processors/query/h2/dml/UpdatePlan.java     |   2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   2 +-
 .../h2/twostep/msg/GridH2IndexRangeRequest.java |   2 +
 .../twostep/msg/GridH2IndexRangeResponse.java   |   2 +
 .../h2/twostep/msg/GridH2QueryRequest.java      |   6 +-
 .../h2/twostep/msg/GridH2ValueMessage.java      |   2 +
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  50 +-
 .../IgniteCacheQueryH2IndexingLeakTest.java     |  10 +-
 .../IgniteCacheReplicatedQuerySelfTest.java     |  26 +-
 .../cache/index/H2ConnectionLeaksSelfTest.java  |   4 +-
 .../cache/index/H2DynamicTableSelfTest.java     |  11 +-
 .../query/h2/sql/GridQueryParsingTest.java      |   2 +-
 .../ml/composition/BaggingModelTrainer.java     | 200 --------
 .../ignite/ml/dataset/DatasetBuilder.java       |  11 +
 .../ml/dataset/PartitionContextBuilder.java     |  21 +
 .../ignite/ml/dataset/PartitionDataBuilder.java |  11 +-
 .../ignite/ml/dataset/UpstreamTransformer.java  |  42 ++
 .../ml/dataset/UpstreamTransformerChain.java    | 154 +++++++
 .../dataset/impl/cache/CacheBasedDataset.java   |  15 +-
 .../impl/cache/CacheBasedDatasetBuilder.java    |  15 +-
 .../dataset/impl/cache/util/ComputeUtils.java   |  95 +++-
 .../ml/dataset/impl/local/LocalDataset.java     |   2 +-
 .../dataset/impl/local/LocalDatasetBuilder.java |  95 +++-
 .../environment/LearningEnvironmentBuilder.java |   2 +-
 .../binomial/LogisticRegressionSGDTrainer.java  |   9 +-
 .../ignite/ml/trainers/DatasetTrainer.java      |   1 +
 .../ignite/ml/trainers/TrainerTransformers.java | 376 ++++++++++++++++
 .../BaggingUpstreamTransformer.java             |  58 +++
 .../ml/trainers/transformers/package-info.java  |  22 +
 .../impurity/ImpurityHistogramsComputer.java    |   2 +-
 .../java/org/apache/ignite/ml/util/Utils.java   |  32 ++
 .../org/apache/ignite/ml/IgniteMLTestSuite.java |   4 +-
 .../impl/cache/util/ComputeUtilsTest.java       |   3 +
 .../apache/ignite/ml/trainers/BaggingTest.java  | 218 +++++++++
 .../Apache.Ignite.Core.Tests.DotNetCore.csproj  |   1 +
 .../Apache.Ignite.Core.Tests.csproj             |   1 +
 .../ApiParity/TcpCommunicationSpiParityTest.cs  |  84 ++++
 .../IgniteConfigurationTest.cs                  |  16 +-
 .../Communication/Tcp/TcpCommunicationSpi.cs    |  99 +++-
 .../e2e/testcafe/components/FormField.js        |  11 +-
 .../components/form-field-size/controller.js    |  25 +-
 .../components/form-field-size/index.js         |   3 +-
 .../components/form-field-size/style.scss       |  15 +
 .../components/form-field-size/template.pug     |   2 +
 .../form-field/showValidationError.directive.js |  21 +-
 .../app/components/ignite-chart/controller.js   |  48 +-
 .../page-configure/components/pcValidation.js   |  18 +-
 .../app/components/page-profile/controller.js   |  12 +-
 .../app/components/page-profile/template.pug    |   5 +-
 .../app/components/page-signin/controller.ts    |   3 +
 .../app/modules/agent/AgentManager.service.js   |  11 +-
 .../agent/components/cluster-login/service.js   |   5 +
 .../app/primitives/datepicker/index.pug         |   6 +-
 .../app/primitives/form-field/dropdown.pug      |   6 +-
 .../app/primitives/form-field/email.pug         |   6 +-
 .../app/primitives/form-field/number.pug        |   6 +-
 .../app/primitives/form-field/password.pug      |   8 +-
 .../app/primitives/form-field/phone.pug         |   5 +-
 .../frontend/app/primitives/form-field/text.pug |   6 +-
 .../app/primitives/form-field/typeahead.pug     |   6 +-
 .../app/primitives/timepicker/index.pug         |   6 +-
 .../service/DemoRandomCacheLoadService.java     |  22 +-
 259 files changed, 4864 insertions(+), 2283 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5f9bdc77/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f9bdc77/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------


[16/50] [abbrv] ignite git commit: IGNITE-8867: [ML] Bagging on learning sample

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/355ce6fe/modules/ml/src/test/java/org/apache/ignite/ml/trainers/BaggingTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/BaggingTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/BaggingTest.java
new file mode 100644
index 0000000..c22da04
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/BaggingTest.java
@@ -0,0 +1,218 @@
+/*
+ * 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.ml.trainers;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.ml.Model;
+import org.apache.ignite.ml.TestUtils;
+import org.apache.ignite.ml.common.TrainerTest;
+import org.apache.ignite.ml.composition.ModelsComposition;
+import org.apache.ignite.ml.composition.predictionsaggregator.MeanValuePredictionsAggregator;
+import org.apache.ignite.ml.composition.predictionsaggregator.OnMajorityPredictionsAggregator;
+import org.apache.ignite.ml.dataset.Dataset;
+import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+import org.apache.ignite.ml.math.functions.IgniteTriFunction;
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+import org.apache.ignite.ml.math.primitives.vector.VectorUtils;
+import org.apache.ignite.ml.nn.UpdatesStrategy;
+import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate;
+import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator;
+import org.apache.ignite.ml.regressions.logistic.binomial.LogisticRegressionModel;
+import org.apache.ignite.ml.regressions.logistic.binomial.LogisticRegressionSGDTrainer;
+import org.junit.Test;
+
+/**
+ * Tests for bagging algorithm.
+ */
+public class BaggingTest extends TrainerTest {
+    /**
+     * Test that count of entries in context is equal to initial dataset size * subsampleRatio.
+     */
+    @Test
+    public void testBaggingContextCount() {
+        count((ctxCount, countData, integer) -> ctxCount);
+    }
+
+    /**
+     * Test that count of entries in data is equal to initial dataset size * subsampleRatio.
+     */
+    @Test
+    public void testBaggingDataCount() {
+        count((ctxCount, countData, integer) -> countData.cnt);
+    }
+
+    /**
+     * Test that bagged log regression makes correct predictions.
+     */
+    @Test
+    public void testNaiveBaggingLogRegression() {
+        Map<Integer, Double[]> cacheMock = getCacheMock();
+
+        DatasetTrainer<LogisticRegressionModel, Double> trainer =
+            (LogisticRegressionSGDTrainer<?>)new LogisticRegressionSGDTrainer<>()
+                .withUpdatesStgy(new UpdatesStrategy<>(new SimpleGDUpdateCalculator(0.2),
+                    SimpleGDParameterUpdate::sumLocal, SimpleGDParameterUpdate::avg))
+                .withMaxIterations(30000)
+                .withLocIterations(100)
+                .withBatchSize(10)
+                .withSeed(123L);
+
+        DatasetTrainer<ModelsComposition, Double> baggedTrainer =
+            TrainerTransformers.makeBagged(
+                trainer,
+                10,
+                0.7,
+                new OnMajorityPredictionsAggregator());
+
+        ModelsComposition mdl = baggedTrainer.fit(
+            cacheMock,
+            parts,
+            (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)),
+            (k, v) -> v[0]
+        );
+
+        TestUtils.assertEquals(0, mdl.apply(VectorUtils.of(100, 10)), PRECISION);
+        TestUtils.assertEquals(1, mdl.apply(VectorUtils.of(10, 100)), PRECISION);
+    }
+
+    /**
+     * Method used to test counts of data passed in context and in data builders.
+     *
+     * @param counter Function specifying which data we should count.
+     */
+    protected void count(IgniteTriFunction<Long, CountData, Integer, Long> counter) {
+        Map<Integer, Double[]> cacheMock = getCacheMock();
+
+        CountTrainer countTrainer = new CountTrainer(counter);
+
+        double subsampleRatio = 0.3;
+
+        ModelsComposition model = TrainerTransformers.makeBagged(countTrainer, 100, subsampleRatio, new MeanValuePredictionsAggregator())
+            .fit(cacheMock, parts, null, null);
+
+        Double res = model.apply(null);
+
+        TestUtils.assertEquals(twoLinearlySeparableClasses.length * subsampleRatio, res, twoLinearlySeparableClasses.length / 10);
+    }
+
+    /**
+     * Create cache mock.
+     *
+     * @return Cache mock.
+     */
+    private Map<Integer, Double[]> getCacheMock() {
+        Map<Integer, Double[]> cacheMock = new HashMap<>();
+
+        for (int i = 0; i < twoLinearlySeparableClasses.length; i++) {
+            double[] row = twoLinearlySeparableClasses[i];
+            Double[] convertedRow = new Double[row.length];
+            for (int j = 0; j < row.length; j++)
+                convertedRow[j] = row[j];
+            cacheMock.put(i, convertedRow);
+        }
+        return cacheMock;
+    }
+
+    /**
+     * Get sum of two Long values each of which can be null.
+     *
+     * @param a First value.
+     * @param b Second value.
+     * @return Sum of parameters.
+     */
+    protected static Long plusOfNullables(Long a, Long b) {
+        if (a == null) {
+            return b;
+        }
+        if (b == null) {
+            return a;
+        }
+
+        return a + b;
+    }
+
+    /**
+     * Trainer used to count entries in context or in data.
+     */
+    protected static class CountTrainer extends DatasetTrainer<Model<Vector, Double>, Double> {
+        /**
+         * Function specifying which entries to count.
+         */
+        private final IgniteTriFunction<Long, CountData, Integer, Long> counter;
+
+        /**
+         * Construct instance of this class.
+         *
+         * @param counter Function specifying which entries to count.
+         */
+        public CountTrainer(IgniteTriFunction<Long, CountData, Integer, Long> counter) {
+            this.counter = counter;
+        }
+
+        /** {@inheritDoc} */
+        @Override public <K, V> Model<Vector, Double> fit(
+            DatasetBuilder<K, V> datasetBuilder,
+            IgniteBiFunction<K, V, Vector> featureExtractor,
+            IgniteBiFunction<K, V, Double> lbExtractor) {
+            Dataset<Long, CountData> dataset = datasetBuilder.build(
+                (upstreamData, upstreamDataSize) -> upstreamDataSize,
+                (upstreamData, upstreamDataSize, ctx) -> new CountData(upstreamDataSize)
+            );
+
+            Long cnt = dataset.computeWithCtx(counter, BaggingTest::plusOfNullables);
+
+            return x -> Double.valueOf(cnt);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected boolean checkState(Model<Vector, Double> mdl) {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected <K, V> Model<Vector, Double> updateModel(
+            Model<Vector, Double> mdl,
+            DatasetBuilder<K, V> datasetBuilder,
+            IgniteBiFunction<K, V, Vector> featureExtractor, IgniteBiFunction<K, V, Double> lbExtractor) {
+            return fit(datasetBuilder, featureExtractor, lbExtractor);
+        }
+    }
+
+    /** Data for count trainer. */
+    protected static class CountData implements AutoCloseable {
+        /** Counter. */
+        private long cnt;
+
+        /**
+         * Construct instance of this class.
+         *
+         * @param cnt Counter.
+         */
+        public CountData(long cnt) {
+            this.cnt = cnt;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws Exception {
+            // No-op
+        }
+    }
+}