You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2017/12/19 12:17:58 UTC

[01/31] ignite git commit: IGNITE-4490 Query-less INSERT and MERGE

Repository: ignite
Updated Branches:
  refs/heads/ignite-6022-proto [created] 771422cb1


IGNITE-4490 Query-less INSERT and MERGE


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

Branch: refs/heads/ignite-6022-proto
Commit: d362da5fdb0671d17672b0096d064ce30254638f
Parents: c040c37
Author: Alexander Paschenko <al...@gmail.com>
Authored: Mon Dec 26 18:39:10 2016 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Mon Dec 26 18:39:10 2016 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        | 91 +++++++++++++++-----
 .../query/h2/dml/FastUpdateArguments.java       | 36 ++++++++
 .../processors/query/h2/dml/UpdatePlan.java     | 43 +++++----
 .../query/h2/dml/UpdatePlanBuilder.java         | 59 +++++++++----
 .../processors/query/h2/sql/DmlAstUtils.java    | 55 +++---------
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |  2 -
 6 files changed, 184 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d362da5f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 4030758..f660148 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -58,6 +58,7 @@ import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
 import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
 import org.apache.ignite.internal.processors.query.h2.dml.KeyValueSupplier;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
@@ -241,13 +242,15 @@ public class DmlStatementsProcessor {
             return new UpdateResult(doSingleUpdate(plan, params), X.EMPTY_OBJECT_ARRAY);
         }
 
-        assert !F.isEmpty(plan.selectQry);
+        assert !F.isEmpty(plan.rows) ^ !F.isEmpty(plan.selectQry);
 
-        QueryCursorImpl<List<?>> cur;
+        Iterable<List<?>> cur;
 
         // Do a two-step query only if locality flag is not set AND if plan's SELECT corresponds to an actual
         // subquery and not some dummy stuff like "select 1, 2, 3;"
         if (!loc && !plan.isLocSubqry) {
+            assert !F.isEmpty(plan.selectQry);
+
             SqlFieldsQuery newFieldsQry = new SqlFieldsQuery(plan.selectQry, fieldsQry.isCollocated())
                 .setArgs(params)
                 .setDistributedJoins(fieldsQry.isDistributedJoins())
@@ -256,13 +259,13 @@ public class DmlStatementsProcessor {
                 .setPageSize(fieldsQry.getPageSize())
                 .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
 
-            cur = (QueryCursorImpl<List<?>>) indexing.queryTwoStep(cctx, newFieldsQry, cancel);
+            cur = indexing.queryTwoStep(cctx, newFieldsQry, cancel);
         }
-        else {
+        else if (F.isEmpty(plan.rows)) {
             final GridQueryFieldsResult res = indexing.queryLocalSqlFields(cctx.name(), plan.selectQry, F.asList(params),
                 filters, fieldsQry.isEnforceJoinOrder(), fieldsQry.getTimeout(), cancel);
 
-            cur = new QueryCursorImpl<>(new Iterable<List<?>>() {
+            QueryCursorImpl<List<?>> resCur = new QueryCursorImpl<>(new Iterable<List<?>>() {
                 @Override public Iterator<List<?>> iterator() {
                     try {
                         return new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepBinary());
@@ -273,7 +276,34 @@ public class DmlStatementsProcessor {
                 }
             }, cancel);
 
-            cur.fieldsMeta(res.metaData());
+            resCur.fieldsMeta(res.metaData());
+
+            cur = resCur;
+        }
+        else {
+            assert plan.rowsNum > 0 && !F.isEmpty(plan.colNames);
+
+            List<List<?>> args = new ArrayList<>(plan.rowsNum);
+
+            GridH2RowDescriptor desc = plan.tbl.rowDescriptor();
+
+            for (List<FastUpdateArgument> argRow : plan.rows) {
+                List<Object> row = new ArrayList<>();
+
+                for (int j = 0; j < plan.colNames.length; j++) {
+                    Object colVal = argRow.get(j).apply(fieldsQry.getArgs());
+
+                    if (j == plan.keyColIdx || j == plan.valColIdx)
+                        colVal = convert(colVal, j == plan.keyColIdx ? desc.type().keyClass() : desc.type().valueClass(),
+                            desc);
+
+                    row.add(colVal);
+                }
+
+                args.add(row);
+            }
+
+            cur = args;
         }
 
         int pageSize = loc ? 0 : fieldsQry.getPageSize();
@@ -379,7 +409,7 @@ public class DmlStatementsProcessor {
      * @return Results of DELETE (number of items affected AND keys that failed to be updated).
      */
     @SuppressWarnings({"unchecked", "ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
-    private UpdateResult doDelete(GridCacheContext cctx, QueryCursorImpl<List<?>> cursor, int pageSize)
+    private UpdateResult doDelete(GridCacheContext cctx, Iterable<List<?>> cursor, int pageSize)
         throws IgniteCheckedException {
         // With DELETE, we have only two columns - key and value.
         long res = 0;
@@ -449,7 +479,7 @@ public class DmlStatementsProcessor {
      *     had been modified concurrently (arguments for a re-run)].
      */
     @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
-    private UpdateResult doUpdate(UpdatePlan plan, QueryCursorImpl<List<?>> cursor, int pageSize)
+    private UpdateResult doUpdate(UpdatePlan plan, Iterable<List<?>> cursor, int pageSize)
         throws IgniteCheckedException {
         GridH2RowDescriptor desc = plan.tbl.rowDescriptor();
 
@@ -492,7 +522,7 @@ public class DmlStatementsProcessor {
                     continue;
 
                 newColVals.put(plan.colNames[i], convert(e.get(i + 2), plan.colNames[i],
-                    plan.tbl.rowDescriptor(), plan.colTypes[i]));
+                    plan.tbl.rowDescriptor()));
             }
 
             newVal = plan.valSupplier.apply(e);
@@ -585,12 +615,11 @@ public class DmlStatementsProcessor {
      * @param val Source value.
      * @param colName Column name to search for property.
      * @param desc Row descriptor.
-     * @param type Expected column type to convert to.
      * @return Converted object.
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"ConstantConditions", "SuspiciousSystemArraycopy"})
-    private static Object convert(Object val, String colName, GridH2RowDescriptor desc, int type)
+    private static Object convert(Object val, String colName, GridH2RowDescriptor desc)
         throws IgniteCheckedException {
         if (val == null)
             return null;
@@ -601,6 +630,21 @@ public class DmlStatementsProcessor {
 
         Class<?> expCls = prop.type();
 
+        return convert(val, expCls, desc);
+    }
+
+    /**
+     * Convert value to column's expected type by means of H2.
+     *
+     * @param val Source value.
+     * @param expCls Expected property class.
+     * @param desc Row descriptor.
+     * @return Converted object.
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings({"ConstantConditions", "SuspiciousSystemArraycopy"})
+    private static Object convert(Object val, Class<?> expCls, GridH2RowDescriptor desc)
+        throws IgniteCheckedException {
         Class<?> currCls = val.getClass();
 
         if (val instanceof Date && currCls != Date.class && expCls == Date.class) {
@@ -609,6 +653,8 @@ public class DmlStatementsProcessor {
             return new Date(((Date) val).getTime());
         }
 
+        int type = DataType.getTypeFromClass(expCls);
+
         // We have to convert arrays of reference types manually - see https://issues.apache.org/jira/browse/IGNITE-4327
         // Still, we only can convert from Object[] to something more precise.
         if (type == Value.ARRAY && currCls != expCls) {
@@ -689,14 +735,14 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    private long doMerge(UpdatePlan plan, QueryCursorImpl<List<?>> cursor, int pageSize) throws IgniteCheckedException {
+    private long doMerge(UpdatePlan plan, Iterable<List<?>> cursor, int pageSize) throws IgniteCheckedException {
         GridH2RowDescriptor desc = plan.tbl.rowDescriptor();
 
         GridCacheContext cctx = desc.context();
 
         // If we have just one item to put, just do so
         if (plan.rowsNum == 1) {
-            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.colTypes, plan.keySupplier,
+            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.keySupplier,
                 plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc);
 
             cctx.cache().put(t.getKey(), t.getValue());
@@ -709,7 +755,7 @@ public class DmlStatementsProcessor {
             for (Iterator<List<?>> it = cursor.iterator(); it.hasNext();) {
                 List<?> row = it.next();
 
-                IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.colTypes, plan.keySupplier, plan.valSupplier,
+                IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.keySupplier, plan.valSupplier,
                     plan.keyColIdx, plan.valColIdx, desc);
 
                 rows.put(t.getKey(), t.getValue());
@@ -735,14 +781,14 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed, particularly in case of duplicate keys.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    private long doInsert(UpdatePlan plan, QueryCursorImpl<List<?>> cursor, int pageSize) throws IgniteCheckedException {
+    private long doInsert(UpdatePlan plan, Iterable<List<?>> cursor, int pageSize) throws IgniteCheckedException {
         GridH2RowDescriptor desc = plan.tbl.rowDescriptor();
 
         GridCacheContext cctx = desc.context();
 
         // If we have just one item to put, just do so
         if (plan.rowsNum == 1) {
-            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.colTypes,
+            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames,
                 plan.keySupplier, plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc);
 
             if (cctx.cache().putIfAbsent(t.getKey(), t.getValue()))
@@ -768,7 +814,7 @@ public class DmlStatementsProcessor {
             while (it.hasNext()) {
                 List<?> row = it.next();
 
-                final IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.colTypes, plan.keySupplier,
+                final IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.keySupplier,
                     plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc);
 
                 rows.put(t.getKey(), new InsertEntryProcessor(t.getValue()));
@@ -838,7 +884,6 @@ public class DmlStatementsProcessor {
      * @param cctx Cache context.
      * @param row Row to process.
      * @param cols Query cols.
-     * @param colTypes Column types to convert data from {@code row} to.
      * @param keySupplier Key instantiation method.
      * @param valSupplier Key instantiation method.
      * @param keyColIdx Key column index, or {@code -1} if no key column is mentioned in {@code cols}.
@@ -848,10 +893,12 @@ public class DmlStatementsProcessor {
      */
     @SuppressWarnings({"unchecked", "ConstantConditions", "ResultOfMethodCallIgnored"})
     private IgniteBiTuple<?, ?> rowToKeyValue(GridCacheContext cctx, Object[] row, String[] cols,
-        int[] colTypes, KeyValueSupplier keySupplier, KeyValueSupplier valSupplier, int keyColIdx, int valColIdx,
+        KeyValueSupplier keySupplier, KeyValueSupplier valSupplier, int keyColIdx, int valColIdx,
         GridH2RowDescriptor rowDesc) throws IgniteCheckedException {
-        Object key = keySupplier.apply(F.asList(row));
-        Object val = valSupplier.apply(F.asList(row));
+        List<Object> rowList = F.asList(row);
+
+        Object key = keySupplier.apply(rowList);
+        Object val = valSupplier.apply(rowList);
 
         if (key == null)
             throw new IgniteSQLException("Key for INSERT or MERGE must not be null",  IgniteQueryErrorCode.NULL_KEY);
@@ -865,7 +912,7 @@ public class DmlStatementsProcessor {
             if (i == keyColIdx || i == valColIdx)
                 continue;
 
-            desc.setValue(cols[i], key, val, convert(row[i], cols[i], rowDesc, colTypes[i]));
+            desc.setValue(cols[i], key, val, convert(row[i], cols[i], rowDesc));
         }
 
         if (cctx.binaryMarshaller()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/d362da5f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
index cb47704..056dfaa 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
@@ -50,4 +50,40 @@ public final class FastUpdateArguments {
             return null;
         }
     };
+
+    /** Simple constant value based operand. */
+    public final static class ValueArgument implements FastUpdateArgument {
+        /** Value to return. */
+        private final Object val;
+
+        /** */
+        public ValueArgument(Object val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
+            return val;
+        }
+    }
+
+    /** Simple constant value based operand. */
+    public final static class ParamArgument implements FastUpdateArgument {
+        /** Value to return. */
+        private final int paramIdx;
+
+        /** */
+        public ParamArgument(int paramIdx) {
+            assert paramIdx >= 0;
+
+            this.paramIdx = paramIdx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
+            assert arg.length > paramIdx;
+
+            return arg[paramIdx];
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d362da5f/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 b81ac60..9bd1ecf 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2.dml;
 
+import java.util.List;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.util.typedef.F;
 
@@ -33,12 +34,6 @@ public final class UpdatePlan {
     /** Column names to set or update. */
     public final String[] colNames;
 
-    /**
-     * Expected column types to set or insert/merge.
-     * @see org.h2.value.Value
-     */
-    public final int[] colTypes;
-
     /** Method to create key for INSERT or MERGE, ignored for UPDATE and DELETE. */
     public final KeyValueSupplier keySupplier;
 
@@ -58,6 +53,9 @@ public final class UpdatePlan {
     /** Subquery flag - {@code true} if {@link #selectQry} is an actual subquery that retrieves data from some cache. */
     public final boolean isLocSubqry;
 
+    /** */
+    public final Iterable<List<FastUpdateArgument>> rows;
+
     /** Number of rows in rows based MERGE or INSERT. */
     public final int rowsNum;
 
@@ -65,11 +63,11 @@ public final class UpdatePlan {
     public final FastUpdateArguments fastUpdateArgs;
 
     /** */
-    private UpdatePlan(UpdateMode mode, GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
+    private UpdatePlan(UpdateMode mode, GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
         KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-        int rowsNum, FastUpdateArguments fastUpdateArgs) {
+        Iterable<List<FastUpdateArgument>> rows, int rowsNum, FastUpdateArguments fastUpdateArgs) {
         this.colNames = colNames;
-        this.colTypes = colTypes;
+        this.rows = rows;
         this.rowsNum = rowsNum;
         assert mode != null;
         assert tbl != null;
@@ -86,43 +84,44 @@ public final class UpdatePlan {
     }
 
     /** */
-    public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
+    public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
         KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-        int rowsNum) {
+        Iterable<List<FastUpdateArgument>> rows, int rowsNum) {
         assert !F.isEmpty(colNames);
 
-        return new UpdatePlan(UpdateMode.MERGE, tbl, colNames, colTypes, keySupplier, valSupplier, keyColIdx, valColIdx,
-            selectQry, isLocSubqry, rowsNum, null);
+        return new UpdatePlan(UpdateMode.MERGE, tbl, colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+            selectQry, isLocSubqry, rows, rowsNum, null);
     }
 
     /** */
-    public static UpdatePlan forInsert(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
-        KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry, int rowsNum) {
+    public static UpdatePlan forInsert(GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+        KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
+        Iterable<List<FastUpdateArgument>> rows, int rowsNum) {
         assert !F.isEmpty(colNames);
 
-        return new UpdatePlan(UpdateMode.INSERT, tbl, colNames, colTypes, keySupplier, valSupplier, keyColIdx, valColIdx,
-            selectQry, isLocSubqry, rowsNum, null);
+        return new UpdatePlan(UpdateMode.INSERT, tbl, colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+            selectQry, isLocSubqry, rows, rowsNum, null);
     }
 
     /** */
-    public static UpdatePlan forUpdate(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier valSupplier,
+    public static UpdatePlan forUpdate(GridH2Table tbl, String[] colNames, KeyValueSupplier valSupplier,
         int valColIdx, String selectQry) {
         assert !F.isEmpty(colNames);
 
-        return new UpdatePlan(UpdateMode.UPDATE, tbl, colNames, colTypes, null, valSupplier, -1, valColIdx, selectQry,
-            false, 0, null);
+        return new UpdatePlan(UpdateMode.UPDATE, tbl, colNames, null, valSupplier, -1, valColIdx, selectQry,
+            false, null, 0, null);
     }
 
     /** */
     public static UpdatePlan forDelete(GridH2Table tbl, String selectQry) {
-        return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, null, -1, -1, selectQry, false, 0, null);
+        return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, -1, -1, selectQry, false, null, 0, null);
     }
 
     /** */
     public static UpdatePlan forFastUpdate(UpdateMode mode, GridH2Table tbl, FastUpdateArguments fastUpdateArgs) {
         assert mode == UpdateMode.UPDATE || mode == UpdateMode.DELETE;
 
-        return new UpdatePlan(mode, tbl, null, null, null, null, -1, -1, null, false, 0, fastUpdateArgs);
+        return new UpdatePlan(mode, tbl, null, null, null, -1, -1, null, false, null, 0, fastUpdateArgs);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d362da5f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index fdcd164..0303fa4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.h2.dml;
 
 import java.lang.reflect.Constructor;
+import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -36,10 +37,12 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.processors.query.h2.sql.DmlAstUtils;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlColumn;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlConst;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDelete;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlInsert;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlMerge;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlParameter;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuery;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlSelect;
@@ -107,6 +110,10 @@ public final class UpdatePlanBuilder {
 
         GridH2RowDescriptor desc;
 
+        List<GridSqlElement[]> elRows = null;
+
+        List<List<FastUpdateArgument>> rows = null;
+
         if (stmt instanceof GridSqlInsert) {
             GridSqlInsert ins = (GridSqlInsert) stmt;
             target = ins.into();
@@ -116,6 +123,10 @@ public final class UpdatePlanBuilder {
 
             cols = ins.columns();
             sel = DmlAstUtils.selectForInsertOrMerge(cols, ins.rows(), ins.query(), desc);
+
+            if (sel == null)
+                elRows = ins.rows();
+
             isTwoStepSubqry = (ins.query() != null);
             rowsNum = isTwoStepSubqry ? 0 : ins.rows().size();
         }
@@ -137,14 +148,40 @@ public final class UpdatePlanBuilder {
 
             cols = merge.columns();
             sel = DmlAstUtils.selectForInsertOrMerge(cols, merge.rows(), merge.query(), desc);
+
+            if (sel == null)
+                elRows = merge.rows();
+
             isTwoStepSubqry = (merge.query() != null);
             rowsNum = isTwoStepSubqry ? 0 : merge.rows().size();
         }
         else throw new IgniteSQLException("Unexpected DML operation [cls=" + stmt.getClass().getName() + ']',
                 IgniteQueryErrorCode.UNEXPECTED_OPERATION);
 
+        if (elRows != null) {
+            assert sel == null;
+
+            rows = new ArrayList<>(elRows.size());
+
+            for (GridSqlElement[] elRow : elRows) {
+                List<FastUpdateArgument> row = new ArrayList<>(cols.length);
+
+                for (GridSqlElement e : elRow) {
+                    if (e instanceof GridSqlConst)
+                        row.add(new FastUpdateArguments.ValueArgument(((GridSqlConst) e).value().getObject()));
+                    else if (e instanceof GridSqlParameter)
+                        row.add(new FastUpdateArguments.ParamArgument(((GridSqlParameter) e).index()));
+                    else
+                        throw new IgniteSQLException("Unexpected element type: " + e.getClass().getSimpleName(),
+                            IgniteQueryErrorCode.UNEXPECTED_ELEMENT_TYPE);
+                }
+
+                rows.add(row);
+            }
+        }
+
         // Let's set the flag only for subqueries that have their FROM specified.
-        isTwoStepSubqry = (isTwoStepSubqry && (sel instanceof GridSqlUnion ||
+        isTwoStepSubqry &= (sel != null && (sel instanceof GridSqlUnion ||
             (sel instanceof GridSqlSelect && ((GridSqlSelect) sel).from() != null)));
 
         int keyColIdx = -1;
@@ -161,8 +198,6 @@ public final class UpdatePlanBuilder {
 
         String[] colNames = new String[cols.length];
 
-        int[] colTypes = new int[cols.length];
-
         for (int i = 0; i < cols.length; i++) {
             GridSqlColumn col = cols[i];
 
@@ -170,8 +205,6 @@ public final class UpdatePlanBuilder {
 
             colNames[i] = colName;
 
-            colTypes[i] = col.resultType().type();
-
             if (KEY_FIELD_NAME.equals(colName)) {
                 keyColIdx = i;
                 continue;
@@ -196,11 +229,11 @@ public final class UpdatePlanBuilder {
         KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false);
 
         if (stmt instanceof GridSqlMerge)
-            return UpdatePlan.forMerge(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
-                valColIdx, sel.getSQL(), !isTwoStepSubqry, rowsNum);
+            return UpdatePlan.forMerge(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx,
+                valColIdx, sel != null ? sel.getSQL() : null, !isTwoStepSubqry, rows, rowsNum);
         else
-            return UpdatePlan.forInsert(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
-                valColIdx, sel.getSQL(), !isTwoStepSubqry, rowsNum);
+            return UpdatePlan.forInsert(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx,
+                valColIdx, sel != null ? sel.getSQL() : null, !isTwoStepSubqry, rows, rowsNum);
     }
 
     /**
@@ -261,13 +294,9 @@ public final class UpdatePlanBuilder {
 
                 String[] colNames = new String[updatedCols.size()];
 
-                int[] colTypes = new int[updatedCols.size()];
-
                 for (int i = 0; i < updatedCols.size(); i++) {
                     colNames[i] = updatedCols.get(i).columnName();
 
-                    colTypes[i] = updatedCols.get(i).resultType().type();
-
                     if (VAL_FIELD_NAME.equals(colNames[i]))
                         valColIdx = i;
                 }
@@ -299,7 +328,7 @@ public final class UpdatePlanBuilder {
 
                 sel = DmlAstUtils.selectForUpdate((GridSqlUpdate) stmt, errKeysPos);
 
-                return UpdatePlan.forUpdate(gridTbl, colNames, colTypes, newValSupplier, valColIdx, sel.getSQL());
+                return UpdatePlan.forUpdate(gridTbl, colNames, newValSupplier, valColIdx, sel.getSQL());
             }
             else {
                 sel = DmlAstUtils.selectForDelete((GridSqlDelete) stmt, errKeysPos);
@@ -323,7 +352,7 @@ public final class UpdatePlanBuilder {
      */
     @SuppressWarnings({"ConstantConditions", "unchecked"})
     private static KeyValueSupplier createSupplier(final GridCacheContext<?, ?> cctx, GridQueryTypeDescriptor desc,
-                                                   final int colIdx, boolean hasProps, final boolean key) throws IgniteCheckedException {
+        final int colIdx, boolean hasProps, final boolean key) throws IgniteCheckedException {
         final String typeName = key ? desc.keyTypeName() : desc.valueTypeName();
 
         //Try to find class for the key locally.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d362da5f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
index 6deb146..8df786c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
@@ -67,7 +66,7 @@ public final class DmlAstUtils {
      * @param rows Rows to create pseudo-SELECT upon.
      * @param subQry Subquery to use rather than rows.
      * @param desc Row descriptor.
-     * @return Subquery or pseudo-SELECT to evaluate inserted expressions.
+     * @return Subquery or pseudo-SELECT to evaluate inserted expressions, or {@code null} no query needs to be run.
      */
     public static GridSqlQuery selectForInsertOrMerge(GridSqlColumn[] cols, List<GridSqlElement[]> rows,
         GridSqlQuery subQry, GridH2RowDescriptor desc) {
@@ -82,6 +81,8 @@ public final class DmlAstUtils {
 
             GridSqlArray[] args = new GridSqlArray[cols.length];
 
+            boolean noQry = true;
+
             for (int i = 0; i < cols.length; i++) {
                 GridSqlArray arr = new GridSqlArray(rows.size());
 
@@ -105,10 +106,18 @@ public final class DmlAstUtils {
             for (GridSqlElement[] row : rows) {
                 assert cols.length == row.length;
 
-                for (int i = 0; i < row.length; i++)
+                for (int i = 0; i < row.length; i++) {
+                    GridSqlElement el = row[i];
+
+                    noQry &= (el instanceof GridSqlConst || el instanceof GridSqlParameter);
+
                     args[i].addChild(row[i]);
+                }
             }
 
+            if (noQry)
+                return null;
+
             return sel;
         }
         else {
@@ -202,9 +211,9 @@ public final class DmlAstUtils {
             return FastUpdateArguments.NULL_ARGUMENT;
 
         if (el instanceof GridSqlConst)
-            return new ValueArgument(((GridSqlConst)el).value().getObject());
+            return new FastUpdateArguments.ValueArgument(((GridSqlConst)el).value().getObject());
         else
-            return new ParamArgument(((GridSqlParameter)el).index());
+            return new FastUpdateArguments.ParamArgument(((GridSqlParameter)el).index());
     }
 
     /**
@@ -577,40 +586,4 @@ public final class DmlAstUtils {
             }
         });
     }
-
-    /** Simple constant value based operand. */
-    private final static class ValueArgument implements FastUpdateArgument {
-        /** Value to return. */
-        private final Object val;
-
-        /** */
-        private ValueArgument(Object val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
-            return val;
-        }
-    }
-
-    /** Simple constant value based operand. */
-    private final static class ParamArgument implements FastUpdateArgument {
-        /** Value to return. */
-        private final int paramIdx;
-
-        /** */
-        private ParamArgument(int paramIdx) {
-            assert paramIdx >= 0;
-
-            this.paramIdx = paramIdx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
-            assert arg.length > paramIdx;
-
-            return arg[paramIdx];
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d362da5f/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
index 332a082..5ee21b2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
@@ -182,8 +182,6 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
         cache.query(new SqlFieldsQuery("insert into \"AllTypes\"(_key, _val, \"dateCol\", \"booleanCol\"," +
             "\"tsCol\") values(2, ?, '2016-11-30 12:00:00', false, DATE '2016-12-01')").setArgs(new AllTypes(2L)));
 
-        List<?> ll = cache.query(new SqlFieldsQuery("select \"primitiveIntsCol\" from \"AllTypes\"")).getAll();
-
         cache.query(new SqlFieldsQuery("update \"AllTypes\" set \"doubleCol\" = CAST('50' as INT)," +
             " \"booleanCol\" = 80, \"innerTypeCol\" = ?, \"strCol\" = PI(), \"shortCol\" = " +
             "CAST(WEEK(PARSEDATETIME('2016-11-30', 'yyyy-MM-dd')) as VARCHAR), " +


[09/31] ignite git commit: Minors.

Posted by vo...@apache.org.
Minors.


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

Branch: refs/heads/ignite-6022-proto
Commit: fdf388b5091a13f61f21c9869e8d50c86574abc8
Parents: c4ee47d
Author: devozerov <vo...@gridgain.com>
Authored: Fri Sep 8 13:59:02 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Sep 8 13:59:02 2017 +0300

----------------------------------------------------------------------
 .../query/h2/dml/FastUpdateArguments.java       | 20 ++++++++++++++++----
 1 file changed, 16 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf388b5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
index 257014c..9ba66f1 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
@@ -51,12 +51,18 @@ public final class FastUpdateArguments {
         }
     };
 
-    /** Simple constant value based operand. */
+    /**
+     * Simple constant value based operand.
+     */
     public final static class ValueArgument implements FastUpdateArgument {
         /** Value to return. */
         private final Object val;
 
-        /** */
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
         public ValueArgument(Object val) {
             this.val = val;
         }
@@ -67,12 +73,18 @@ public final class FastUpdateArguments {
         }
     }
 
-    /** User given param value operand. */
+    /**
+     * User given param value operand.
+     */
     public final static class ParamArgument implements FastUpdateArgument {
         /** Index of param to take. */
         private final int paramIdx;
 
-        /** */
+        /**
+         * Constructor.
+         *
+         * @param paramIdx Parameter index.
+         */
         public ParamArgument(int paramIdx) {
             assert paramIdx >= 0;
 


[15/31] ignite git commit: Test fix

Posted by vo...@apache.org.
Test fix


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

Branch: refs/heads/ignite-6022-proto
Commit: 478fb439465c55e4bf2e7862cf28a6bcfcd10d96
Parents: 790cd26
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Dec 15 19:31:18 2017 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Fri Dec 15 19:31:18 2017 +0300

----------------------------------------------------------------------
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java |  2 +-
 .../query/h2/DmlStatementsProcessor.java        | 82 +-------------------
 .../processors/query/h2/dml/DmlUtils.java       | 19 +++--
 3 files changed, 16 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/478fb439/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
index 952baa5..fb96f31 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
@@ -107,7 +107,7 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
     public void testDmlErrors() throws SQLException {
         checkErrorState("INSERT INTO \"test\".INTEGER(_key, _val) values(1, null)", "22004");
 
-        checkErrorState("INSERT INTO \"test\".INTEGER(_key, _val) values(1, 'zzz')", "50000");
+        checkErrorState("INSERT INTO \"test\".INTEGER(_key, _val) values(1, 'zzz')", "0700B");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/478fb439/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index dd62c75..8de756c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -17,16 +17,12 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
-import java.lang.reflect.Array;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
-import java.sql.Time;
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Date;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -54,6 +50,7 @@ import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlBatchSender;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlDistributedPlanInfo;
+import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils;
 import org.apache.ignite.internal.processors.query.h2.dml.FastUpdate;
 import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode;
@@ -75,12 +72,6 @@ import org.h2.command.dml.Delete;
 import org.h2.command.dml.Insert;
 import org.h2.command.dml.Merge;
 import org.h2.command.dml.Update;
-import org.h2.util.DateTimeUtils;
-import org.h2.util.LocalDateTimeUtils;
-import org.h2.value.Value;
-import org.h2.value.ValueDate;
-import org.h2.value.ValueTime;
-import org.h2.value.ValueTimestamp;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.createJdbcSqlException;
@@ -437,7 +428,7 @@ public class DmlStatementsProcessor {
                 Object colVal = argRow.get(j).apply(args);
 
                 if (j == plan.keyColumnIndex() || j == plan.valueColumnIndex())
-                    colVal = convert(colVal, desc, j == plan.keyColumnIndex() ? desc.type().keyClass() :
+                    colVal = DmlUtils.convert(colVal, desc, j == plan.keyColumnIndex() ? desc.type().keyClass() :
                         desc.type().valueClass(), plan.columnTypes()[j]);
 
                 row.add(colVal);
@@ -624,75 +615,6 @@ public class DmlStatementsProcessor {
     }
 
     /**
-     * Convert value to column's expected type by means of H2.
-     *
-     * @param val Source value.
-     * @param desc Row descriptor.
-     * @param expCls Expected value class.
-     * @param type Expected column type to convert to.
-     * @return Converted object.
-     * @throws IgniteCheckedException if failed.
-     */
-    @SuppressWarnings({"ConstantConditions", "SuspiciousSystemArraycopy"})
-    private static Object convert(Object val, GridH2RowDescriptor desc, Class<?> expCls, int type)
-        throws IgniteCheckedException {
-        if (val == null)
-            return null;
-
-        Class<?> currCls = val.getClass();
-
-        if (val instanceof Date && currCls != Date.class && expCls == Date.class) {
-            // H2 thinks that java.util.Date is always a Timestamp, while binary marshaller expects
-            // precise Date instance. Let's satisfy it.
-            return new Date(((Date) val).getTime());
-        }
-
-        // User-given UUID is always serialized by H2 to byte array, so we have to deserialize manually
-        if (type == Value.UUID && currCls == byte[].class)
-            return U.unmarshal(desc.context().marshaller(), (byte[]) val,
-                U.resolveClassLoader(desc.context().gridConfig()));
-
-        if (LocalDateTimeUtils.isJava8DateApiPresent()) {
-            if (val instanceof Timestamp && LocalDateTimeUtils.isLocalDateTime(expCls))
-                return LocalDateTimeUtils.valueToLocalDateTime(ValueTimestamp.get((Timestamp)val));
-
-            if (val instanceof Date && LocalDateTimeUtils.isLocalDate(expCls))
-                return LocalDateTimeUtils.valueToLocalDate(ValueDate.fromDateValue(
-                    DateTimeUtils.dateValueFromDate(((Date)val).getTime())));
-
-            if (val instanceof Time && LocalDateTimeUtils.isLocalTime(expCls))
-                return LocalDateTimeUtils.valueToLocalTime(ValueTime.get((Time)val));
-        }
-
-        // We have to convert arrays of reference types manually - see https://issues.apache.org/jira/browse/IGNITE-4327
-        // Still, we only can convert from Object[] to something more precise.
-        if (type == Value.ARRAY && currCls != expCls) {
-            if (currCls != Object[].class)
-                throw new IgniteCheckedException("Unexpected array type - only conversion from Object[] is assumed");
-
-            // Why would otherwise type be Value.ARRAY?
-            assert expCls.isArray();
-
-            Object[] curr = (Object[]) val;
-
-            Object newArr = Array.newInstance(expCls.getComponentType(), curr.length);
-
-            System.arraycopy(curr, 0, newArr, 0, curr.length);
-
-            return newArr;
-        }
-
-        Object res = H2Utils.convert(val, desc, type);
-
-        if (res instanceof Date && res.getClass() != Date.class && expCls == Date.class) {
-            // We can get a Timestamp instead of Date when converting a String to Date without query - let's handle this
-            return new Date(((Date) res).getTime());
-        }
-
-        return res;
-    }
-
-    /**
      * Execute MERGE statement plan.
      * @param cursor Cursor to take inserted data from.
      * @param pageSize Batch size to stream data from {@code cursor}, anything <= 0 for single page operations.

http://git-wip-us.apache.org/repos/asf/ignite/blob/478fb439/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
index 6621fc2..8d4861e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
@@ -17,6 +17,10 @@
 
 package org.apache.ignite.internal.processors.query.h2.dml;
 
+import java.lang.reflect.Array;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Date;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
@@ -30,11 +34,6 @@ import org.h2.value.ValueDate;
 import org.h2.value.ValueTime;
 import org.h2.value.ValueTimestamp;
 
-import java.lang.reflect.Array;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.Date;
-
 /**
  * DML utility methods.
  */
@@ -101,7 +100,15 @@ public class DmlUtils {
                 return newArr;
             }
 
-            return H2Utils.convert(val, desc, type);
+            Object res = H2Utils.convert(val, desc, type);
+
+            if (res instanceof Date && res.getClass() != Date.class && expCls == Date.class) {
+                // We can get a Timestamp instead of Date when converting a String to Date
+                // without query - let's handle this
+                return new Date(((Date) res).getTime());
+            }
+
+            return res;
         }
         catch (Exception e) {
             throw new IgniteSQLException("Value conversion failed [from=" + currCls.getName() + ", to=" +


[22/31] ignite git commit: Done.

Posted by vo...@apache.org.
Done.


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

Branch: refs/heads/ignite-6022-proto
Commit: dd70a84606185d575a1a9339d2a51cfd1993648a
Parents: 5d99beb
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 12:11:18 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 12:11:18 2017 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        | 44 +---------
 .../processors/query/h2/dml/UpdatePlan.java     | 84 ++++++++++----------
 2 files changed, 46 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dd70a846/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 7f1f5d5..9a6b0af 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -48,14 +48,11 @@ import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
-import org.apache.ignite.internal.processors.query.h2.dml.DmlArgument;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlBatchSender;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlDistributedPlanInfo;
-import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.lang.IgniteSingletonIterator;
@@ -282,7 +279,7 @@ public class DmlStatementsProcessor {
                             it = res.iterator();
                         }
                         else
-                            it = planToRows(plan, U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)).iterator();
+                            it = plan.createRows(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)).iterator();
 
                         return new GridQueryCacheObjectsIterator(it, idx.objectContext(), cctx.keepBinary());
                     }
@@ -379,7 +376,9 @@ public class DmlStatementsProcessor {
 
             cur = idx.queryDistributedSqlFields(schemaName, newFieldsQry, true, cancel, mainCacheId, true).get(0);
         }
-        else if (F.isEmpty(plan.rows())) {
+        else if (plan.hasRows())
+            cur = plan.createRows(fieldsQry.getArgs());
+        else {
             final GridQueryFieldsResult res = idx.queryLocalSqlFields(schemaName, plan.selectQuery(),
                 F.asList(fieldsQry.getArgs()), filters, fieldsQry.isEnforceJoinOrder(), fieldsQry.getTimeout(), cancel);
 
@@ -394,8 +393,6 @@ public class DmlStatementsProcessor {
                 }
             }, cancel);
         }
-        else
-            cur = planToRows(plan, fieldsQry.getArgs());
 
         int pageSize = loc ? 0 : fieldsQry.getPageSize();
 
@@ -403,39 +400,6 @@ public class DmlStatementsProcessor {
     }
 
     /**
-     * Extract rows from plan without performing any query.
-     * @param plan Plan.
-     * @param args Original query arguments.
-     * @return Rows from plan.
-     * @throws IgniteCheckedException if failed.
-     */
-    private List<List<?>> planToRows(UpdatePlan plan, Object[] args) throws IgniteCheckedException {
-        assert plan.rowCount() > 0 && !F.isEmpty(plan.columnNames());
-
-        List<List<?>> rows = new ArrayList<>(plan.rowCount());
-
-        GridH2RowDescriptor desc = plan.table().rowDescriptor();
-
-        for (List<DmlArgument> argRow : plan.rows()) {
-            List<Object> row = new ArrayList<>();
-
-            for (int j = 0; j < plan.columnNames().length; j++) {
-                Object colVal = argRow.get(j).get(args);
-
-                if (j == plan.keyColumnIndex() || j == plan.valueColumnIndex())
-                    colVal = DmlUtils.convert(colVal, desc, j == plan.keyColumnIndex() ? desc.type().keyClass() :
-                        desc.type().valueClass(), plan.columnTypes()[j]);
-
-                row.add(colVal);
-            }
-
-            rows.add(row);
-        }
-
-        return rows;
-    }
-
-    /**
      * @param cctx Cache context.
      * @param plan Update plan.
      * @param cursor Cursor over select results.

http://git-wip-us.apache.org/repos/asf/ignite/blob/dd70a846/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 10e7519..6a45c3c 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2.dml;
 
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -360,6 +361,47 @@ public final class UpdatePlan {
     }
 
     /**
+     * @return {@code True} if predefined rows exist.
+     */
+    public boolean hasRows() {
+        return !F.isEmpty(rows);
+    }
+
+    /**
+     * Extract rows from plan without performing any query.
+     * @param args Original query arguments.
+     * @return Rows from plan.
+     * @throws IgniteCheckedException if failed.
+     */
+    public List<List<?>> createRows(Object[] args) throws IgniteCheckedException {
+        assert rowsNum > 0 && !F.isEmpty(colNames);
+
+        List<List<?>> res = new ArrayList<>(rowsNum);
+
+        GridH2RowDescriptor desc = tbl.rowDescriptor();
+
+        for (List<DmlArgument> row : rows) {
+            List<Object> resRow = new ArrayList<>();
+
+            for (int j = 0; j < colNames.length; j++) {
+                Object colVal = row.get(j).get(args);
+
+                if (j == keyColIdx || j == valColIdx) {
+                    Class<?> colCls = j == keyColIdx ? desc.type().keyClass() : desc.type().valueClass();
+
+                    colVal = DmlUtils.convert(colVal, desc, colCls, colTypes[j]);
+                }
+
+                resRow.add(colVal);
+            }
+
+            res.add(resRow);
+        }
+
+        return res;
+    }
+
+    /**
      * @return Update mode.
      */
     public UpdateMode mode() {
@@ -400,46 +442,4 @@ public final class UpdatePlan {
     @Nullable public boolean isLocalSubquery() {
         return isLocSubqry;
     }
-
-    /**
-     * @return Names of affected columns.
-     */
-    public String[] columnNames() {
-        return colNames;
-    }
-
-    /**
-     * @return Types of affected columns.
-     */
-    public int[] columnTypes() {
-        return colTypes;
-    }
-
-    /**
-     * @return Rows for query-less MERGE or INSERT.
-     */
-    public List<List<DmlArgument>> rows() {
-        return rows;
-    }
-
-    /**
-     * @return Key column index.
-     */
-    public int keyColumnIndex() {
-        return keyColIdx;
-    }
-
-    /**
-     * @return Value column index.
-     */
-    public int valueColumnIndex() {
-        return valColIdx;
-    }
-
-    /**
-     * @return Target table.
-     */
-    public GridH2Table table() {
-        return tbl;
-    }
 }


[12/31] ignite git commit: Test fix

Posted by vo...@apache.org.
Test fix


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

Branch: refs/heads/ignite-6022-proto
Commit: e11fee9c416493ef084032520f2e6dec35852f29
Parents: 799098c
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Dec 15 18:42:07 2017 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Fri Dec 15 18:42:07 2017 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        | 135 +++++++------------
 .../processors/query/h2/dml/DmlAstUtils.java    |   4 +-
 .../processors/query/h2/dml/UpdatePlan.java     |  57 +++++++-
 .../query/h2/dml/UpdatePlanBuilder.java         |   4 +-
 4 files changed, 106 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e11fee9c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 9e41bfe..dd62c75 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -17,12 +17,16 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
+import java.lang.reflect.Array;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Date;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -32,7 +36,6 @@ import java.util.concurrent.TimeUnit;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.MutableEntry;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
@@ -49,15 +52,14 @@ import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
-import org.apache.ignite.internal.processors.query.QueryUtils;
-import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
-import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlBatchSender;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlDistributedPlanInfo;
 import org.apache.ignite.internal.processors.query.h2.dml.FastUpdate;
+import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.lang.IgniteSingletonIterator;
@@ -73,14 +75,12 @@ import org.h2.command.dml.Delete;
 import org.h2.command.dml.Insert;
 import org.h2.command.dml.Merge;
 import org.h2.command.dml.Update;
-import org.h2.table.Column;
 import org.h2.util.DateTimeUtils;
 import org.h2.util.LocalDateTimeUtils;
 import org.h2.value.Value;
 import org.h2.value.ValueDate;
 import org.h2.value.ValueTime;
 import org.h2.value.ValueTimestamp;
-import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.createJdbcSqlException;
@@ -258,15 +258,13 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    long streamUpdateQuery(IgniteDataStreamer streamer, PreparedStatement stmt, Object[] args)
+    long streamUpdateQuery(IgniteDataStreamer streamer, PreparedStatement stmt, final Object[] args)
         throws IgniteCheckedException {
-        args = U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY);
-
         Prepared p = GridSqlQueryParser.prepared(stmt);
 
         assert p != null;
 
-        UpdatePlan plan = UpdatePlanBuilder.planForStatement(p, true, idx, null, null, null);
+        final UpdatePlan plan = UpdatePlanBuilder.planForStatement(p, true, idx, null, null, null);
 
         if (!F.eq(streamer.cacheName(), plan.cacheContext().name()))
             throw new IgniteSQLException("Cross cache streaming is not supported, please specify cache explicitly" +
@@ -281,14 +279,22 @@ public class DmlStatementsProcessor {
 
             final ArrayList<List<?>> data = new ArrayList<>(plan.rowCount());
 
-            final GridQueryFieldsResult res = idx.queryLocalSqlFields(idx.schema(cctx.name()), plan.selectQuery(),
-                F.asList(args), null, false, 0, null);
-
             QueryCursorImpl<List<?>> stepCur = new QueryCursorImpl<>(new Iterable<List<?>>() {
                 @Override public Iterator<List<?>> iterator() {
                     try {
-                        return new GridQueryCacheObjectsIterator(res.iterator(), idx.objectContext(),
-                            cctx.keepBinary());
+                        Iterator<List<?>> it;
+
+                        if (!F.isEmpty(plan.selectQuery())) {
+                            GridQueryFieldsResult res = idx.queryLocalSqlFields(idx.schema(cctx.name()),
+                                plan.selectQuery(), F.asList(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)),
+                                null, false, 0, null);
+
+                            it = res.iterator();
+                        }
+                        else
+                            it = planToRows(plan, U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)).iterator();
+
+                        return new GridQueryCacheObjectsIterator(it, idx.objectContext(), cctx.keepBinary());
                     }
                     catch (IgniteCheckedException e) {
                         throw new IgniteException(e);
@@ -370,8 +376,6 @@ public class DmlStatementsProcessor {
                 return result;
         }
 
-        assert !F.isEmpty(plan.selectQuery());
-
         Iterable<List<?>> cur;
 
         // Do a two-step query only if locality flag is not set AND if plan's SELECT corresponds to an actual
@@ -387,11 +391,10 @@ public class DmlStatementsProcessor {
                 .setPageSize(fieldsQry.getPageSize())
                 .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
 
-            cur = (QueryCursorImpl<List<?>>)idx.queryDistributedSqlFields(schemaName, newFieldsQry, true,
-                cancel, mainCacheId, true).get(0);
+            cur = idx.queryDistributedSqlFields(schemaName, newFieldsQry, true, cancel, mainCacheId, true).get(0);
         }
-        else if (F.isEmpty(plan.rows)) {
-            final GridQueryFieldsResult res = idx.queryLocalSqlFields(schemaName, plan.selectQry(),
+        else if (F.isEmpty(plan.rows())) {
+            final GridQueryFieldsResult res = idx.queryLocalSqlFields(schemaName, plan.selectQuery(),
                 F.asList(fieldsQry.getArgs()), filters, fieldsQry.isEnforceJoinOrder(), fieldsQry.getTimeout(), cancel);
 
             cur = new QueryCursorImpl<>(new Iterable<List<?>>() {
@@ -405,35 +408,45 @@ public class DmlStatementsProcessor {
                 }
             }, cancel);
         }
-        else {
-            assert plan.rowsNum > 0 && !F.isEmpty(plan.colNames);
+        else
+            cur = planToRows(plan, fieldsQry.getArgs());
 
-            List<List<?>> args = new ArrayList<>(plan.rowsNum);
+        int pageSize = loc ? 0 : fieldsQry.getPageSize();
 
-            GridH2RowDescriptor desc = plan.tbl.rowDescriptor();
+        return processDmlSelectResult(cctx, plan, cur, pageSize);
+    }
 
-            for (List<FastUpdateArgument> argRow : plan.rows) {
-                List<Object> row = new ArrayList<>();
+    /**
+     * Extract rows from plan without performing any query.
+     * @param plan Plan.
+     * @param args Original query arguments.
+     * @return Rows from plan.
+     * @throws IgniteCheckedException if failed.
+     */
+    private List<List<?>> planToRows(UpdatePlan plan, Object[] args) throws IgniteCheckedException {
+        assert plan.rowCount() > 0 && !F.isEmpty(plan.columnNames());
 
-                for (int j = 0; j < plan.colNames.length; j++) {
-                    Object colVal = argRow.get(j).apply(fieldsQry.getArgs());
+        List<List<?>> rows = new ArrayList<>(plan.rowCount());
 
-                    if (j == plan.keyColIdx || j == plan.valColIdx)
-                        colVal = convert(colVal, desc, j == plan.keyColIdx ? desc.type().keyClass() :
-                            desc.type().valueClass(), plan.colTypes[j]);
+        GridH2RowDescriptor desc = plan.table().rowDescriptor();
 
-                    row.add(colVal);
-                }
+        for (List<FastUpdateArgument> argRow : plan.rows()) {
+            List<Object> row = new ArrayList<>();
+
+            for (int j = 0; j < plan.columnNames().length; j++) {
+                Object colVal = argRow.get(j).apply(args);
+
+                if (j == plan.keyColumnIndex() || j == plan.valueColumnIndex())
+                    colVal = convert(colVal, desc, j == plan.keyColumnIndex() ? desc.type().keyClass() :
+                        desc.type().valueClass(), plan.columnTypes()[j]);
 
-                args.add(row);
+                row.add(colVal);
             }
 
-            cur = args;
+            rows.add(row);
         }
 
-        int pageSize = loc ? 0 : fieldsQry.getPageSize();
-
-        return processDmlSelectResult(cctx, plan, cur, pageSize);
+        return rows;
     }
 
     /**
@@ -680,50 +693,6 @@ public class DmlStatementsProcessor {
     }
 
     /**
-     * Process errors of entry processor - split the keys into duplicated/concurrently modified and those whose
-     * processing yielded an exception.
-     *
-     * @param res Result of {@link GridCacheAdapter#invokeAll)}
-     * @return pair [array of duplicated/concurrently modified keys, SQL exception for erroneous keys] (exception is
-     * null if all keys are duplicates/concurrently modified ones).
-     */
-    private static PageProcessingErrorResult splitErrors(Map<Object, EntryProcessorResult<Boolean>> res) {
-        Set<Object> errKeys = new LinkedHashSet<>(res.keySet());
-
-        SQLException currSqlEx = null;
-
-        SQLException firstSqlEx = null;
-
-        int errors = 0;
-
-        // Let's form a chain of SQL exceptions
-        for (Map.Entry<Object, EntryProcessorResult<Boolean>> e : res.entrySet()) {
-            try {
-                e.getValue().get();
-            }
-            catch (EntryProcessorException ex) {
-                SQLException next = createJdbcSqlException("Failed to process key '" + e.getKey() + '\'',
-                    IgniteQueryErrorCode.ENTRY_PROCESSING);
-
-                next.initCause(ex);
-
-                if (currSqlEx != null)
-                    currSqlEx.setNextException(next);
-                else
-                    firstSqlEx = next;
-
-                currSqlEx = next;
-
-                errKeys.remove(e.getKey());
-
-                errors++;
-            }
-        }
-
-        return new PageProcessingErrorResult(errKeys.toArray(), firstSqlEx, errors);
-    }
-
-    /**
      * Execute MERGE statement plan.
      * @param cursor Cursor to take inserted data from.
      * @param pageSize Batch size to stream data from {@code cursor}, anything <= 0 for single page operations.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e11fee9c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
index a2cd553..b6c4a2a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
@@ -83,7 +83,6 @@ public final class DmlAstUtils {
      * @param cols Columns to insert values into.
      * @param rows Rows to create pseudo-SELECT upon.
      * @param subQry Subquery to use rather than rows.
-     * @param desc Row descriptor.
      * @return Subquery or pseudo-SELECT to evaluate inserted expressions, or {@code null} no query needs to be run.
      */
     public static GridSqlQuery selectForInsertOrMerge(GridSqlColumn[] cols, List<GridSqlElement[]> rows,
@@ -219,8 +218,7 @@ public final class DmlAstUtils {
         if (!(set instanceof GridSqlConst || set instanceof GridSqlParameter))
             return null;
 
-        return new FastUpdateArguments(operandForElement(filter.getKey()), operandForElement(filter.getValue()),
-            operandForElement(set));
+        return FastUpdate.create(filter.getKey(), filter.getValue(), set);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/e11fee9c/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 fa86836..96298d8 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
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.internal.processors.query.h2.dml;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
@@ -35,10 +37,6 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.h2.table.Column;
 import org.jetbrains.annotations.Nullable;
 
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT;
 
 /**
@@ -48,7 +46,7 @@ public final class UpdatePlan {
     /** Initial statement to drive the rest of the logic. */
     private final UpdateMode mode;
 
-    /** Target table to be affected by initial DML statement. */
+    /**  to be affected by initial DML statement. */
     private final GridH2Table tbl;
 
     /** Column names to set or update. */
@@ -75,8 +73,8 @@ public final class UpdatePlan {
     /** Subquery flag - {@code true} if {@link #selectQry} is an actual subquery that retrieves data from some cache. */
     private final boolean isLocSubqry;
 
-    /** */
-    public final List<List<FastUpdateArgument>> rows;
+    /** Rows for query-less MERGE or INSERT. */
+    private final List<List<FastUpdateArgument>> rows;
 
     /** Number of rows in rows based MERGE or INSERT. */
     private final int rowsNum;
@@ -106,6 +104,7 @@ public final class UpdatePlan {
      * @param valColIdx value column index.
      * @param selectQry Select query.
      * @param isLocSubqry Local subquery flag.
+     * @param rows Rows for query-less INSERT or MERGE.
      * @param rowsNum Rows number.
      * @param fastUpdate Fast update (if any).
      * @param distributed Distributed plan (if any)
@@ -121,6 +120,7 @@ public final class UpdatePlan {
         int valColIdx,
         String selectQry,
         boolean isLocSubqry,
+        List<List<FastUpdateArgument>> rows,
         int rowsNum,
         @Nullable FastUpdate fastUpdate,
         @Nullable DmlDistributedPlanInfo distributed
@@ -172,6 +172,7 @@ public final class UpdatePlan {
             -1,
             selectQry,
             false,
+            null,
             0,
             fastUpdate,
             distributed
@@ -398,6 +399,48 @@ public final class UpdatePlan {
         return fastUpdate;
     }
 
+    /**
+     * @return Names of affected columns.
+     */
+    public String[] columnNames() {
+        return colNames;
+    }
+
+    /**
+     * @return Types of affected columns.
+     */
+    public int[] columnTypes() {
+        return colTypes;
+    }
+
+    /**
+     * @return Rows for query-less MERGE or INSERT.
+     */
+    public List<List<FastUpdateArgument>> rows() {
+        return rows;
+    }
+
+    /**
+     * @return Key column index.
+     */
+    public int keyColumnIndex() {
+        return keyColIdx;
+    }
+
+    /**
+     * @return Value column index.
+     */
+    public int valueColumnIndex() {
+        return valColIdx;
+    }
+
+    /**
+     * @return Target table.
+     */
+    public GridH2Table table() {
+        return tbl;
+    }
+
     /*
     public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
                                       KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e11fee9c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index 52efd6d..d04cea9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -247,7 +247,7 @@ public final class UpdatePlanBuilder {
         KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, true, false);
         KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false, false);
 
-        String selectSql = sel.getSQL();
+        String selectSql = sel != null ? sel.getSQL() : null;
 
         DmlDistributedPlanInfo distributed = (rowsNum == 0 && !F.isEmpty(selectSql)) ?
             checkPlanCanBeDistributed(idx, conn, fieldsQuery, loc, selectSql, tbl.dataTable().cacheName()) : null;
@@ -265,6 +265,7 @@ public final class UpdatePlanBuilder {
             valColIdx,
             selectSql,
             !isTwoStepSubqry,
+            rows,
             rowsNum,
             null,
             distributed
@@ -391,6 +392,7 @@ public final class UpdatePlanBuilder {
                     valColIdx,
                     selectSql,
                     false,
+                    null,
                     0,
                     null,
                     distributed


[06/31] ignite git commit: Merge remote-tracking branch 'apache/master' into ignite-4490

Posted by vo...@apache.org.
Merge remote-tracking branch 'apache/master' into ignite-4490

# Conflicts:
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java


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

Branch: refs/heads/ignite-6022-proto
Commit: 336ad01a7ec01698b739dca766962c9f7c8957dd
Parents: e526948 651ffc5
Author: Alexander Paschenko <al...@gmail.com>
Authored: Thu Jul 6 13:17:14 2017 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Thu Jul 6 14:12:36 2017 +0300

----------------------------------------------------------------------
 .gitignore                                      |   31 +-
 DEVNOTES.txt                                    |   77 +-
 NOTICE                                          |    2 +-
 RELEASE_NOTES.txt                               |   56 +
 assembly/LICENSE_FABRIC                         |    2 +-
 assembly/LICENSE_HADOOP                         |    2 +-
 assembly/NOTICE_FABRIC                          |    2 +-
 assembly/NOTICE_HADOOP                          |    2 +-
 assembly/dependencies-fabric-lgpl.xml           |    5 +-
 assembly/dependencies-fabric.xml                |    9 +-
 assembly/dependencies-hadoop.xml                |    3 +
 assembly/dependencies-schema-import.xml         |   57 -
 assembly/libs/README.txt                        |    2 +
 assembly/release-base.xml                       |    7 -
 assembly/release-fabric-base.xml                |   28 +-
 assembly/release-fabric-lgpl.xml                |    1 +
 assembly/release-fabric.xml                     |    1 +
 assembly/release-schema-import.xml              |   50 -
 assembly/release-sources.xml                    |    3 +-
 assembly/release-yardstick.xml                  |   98 +
 bin/control.bat                                 |  229 +
 bin/control.sh                                  |  180 +
 bin/ignite-schema-import.bat                    |  110 -
 bin/ignite-schema-import.sh                     |   87 -
 config/dotnet/default-dotnet.xml                |   45 -
 config/example-kube.xml                         |   53 +
 config/hadoop/default-config.xml                |   29 -
 config/ignite-log4j.xml                         |    6 +-
 config/ignite-log4j2.xml                        |    6 +-
 doap_Ignite.rdf                                 |   14 +-
 examples/README-LGPL.txt                        |    4 +-
 examples/README.txt                             |   12 +-
 examples/config/example-cache.xml               |    1 +
 examples/config/example-memory-policies.xml     |  111 +
 examples/config/filesystem/example-igfs.xml     |   32 +-
 .../example-persistent-store.xml                |   77 +
 examples/config/spark/example-shared-rdd.xml    |   83 +
 examples/pom-standalone-lgpl.xml                |   21 +-
 examples/pom-standalone.xml                     |   21 +-
 examples/pom.xml                                |   75 +-
 examples/redis/redis-example.php                |    4 +-
 examples/redis/redis-example.py                 |    3 +-
 examples/rest/http-rest-example.php             |    4 +-
 examples/schema-import/README.txt               |   38 -
 examples/schema-import/bin/db-init.sql          |   28 -
 examples/schema-import/bin/h2-server.bat        |   80 -
 examples/schema-import/bin/h2-server.sh         |   66 -
 .../schema-import/bin/schema-import.properties  |   28 -
 examples/schema-import/pom-standalone.xml       |   90 -
 examples/schema-import/pom.xml                  |  101 -
 .../org/apache/ignite/schema/CacheConfig.java   |   36 -
 .../java/org/apache/ignite/schema/Demo.java     |  134 -
 .../java/org/apache/ignite/schema/DemoNode.java |   38 -
 .../ignite/schema/H2DataSourceFactory.java      |   35 -
 .../java/org/apache/ignite/schema/Person.java   |   44 -
 .../org/apache/ignite/schema/PersonKey.java     |   37 -
 .../hibernate/HibernateL2CacheExample.java      |  106 +-
 .../datagrid/CacheClientBinaryQueryExample.java |    4 +-
 .../computegrid/ComputeAsyncExample.java        |    8 +-
 .../ComputeFibonacciContinuationExample.java    |   13 +-
 .../examples/datagrid/CacheAsyncApiExample.java |   17 +-
 .../examples/datagrid/CacheQueryExample.java    |   56 +-
 .../datagrid/MemoryPoliciesExample.java         |  114 +
 .../starschema/CacheStarSchemaExample.java      |   59 +-
 .../datastructures/IgniteLockExample.java       |    4 +-
 .../datastructures/IgniteSemaphoreExample.java  |    4 +-
 .../client/memcache/MemcacheRestExample.java    |    2 +-
 .../MemcacheRestExampleNodeStartup.java         |    8 +-
 .../ignite/examples/model/Organization.java     |    9 +
 .../persistentstore/PersistentStoreExample.java |   92 +
 .../PersistentStoreExampleNodeStartup.java      |   33 +
 .../examples/springdata/PersonRepository.java   |   59 +
 .../examples/springdata/SpringAppCfg.java       |   69 +
 .../examples/springdata/SpringDataExample.java  |  154 +
 .../streaming/wordcount/QueryWords.java         |    3 +-
 .../CholeskyDecompositionExample.java           |   80 +
 .../EigenDecompositionExample.java              |   69 +
 .../decompositions/LUDecompositionExample.java  |   83 +
 .../SingularValueDecompositionExample.java      |   70 +
 .../ml/math/decompositions/package-info.java    |   22 +
 .../ml/math/matrix/CacheMatrixExample.java      |  102 +
 .../ml/math/matrix/ExampleMatrixStorage.java    |  163 +
 .../math/matrix/MatrixCustomStorageExample.java |  141 +
 .../examples/ml/math/matrix/MatrixExample.java  |   79 +
 .../ml/math/matrix/MatrixExampleUtil.java       |   52 +
 .../ml/math/matrix/OffHeapMatrixExample.java    |   84 +
 .../matrix/SparseDistributedMatrixExample.java  |   70 +
 .../ml/math/matrix/SparseMatrixExample.java     |   84 +
 .../examples/ml/math/matrix/package-info.java   |   22 +
 .../ignite/examples/ml/math/package-info.java   |   22 +
 .../DistributedRegressionExample.java           |  149 +
 .../examples/ml/math/tracer/TracerExample.java  |   63 +
 .../examples/ml/math/tracer/package-info.java   |   22 +
 .../ml/math/vector/CacheVectorExample.java      |  109 +
 .../ml/math/vector/ExampleVectorStorage.java    |  127 +
 .../ml/math/vector/OffHeapVectorExample.java    |   78 +
 .../ml/math/vector/SparseVectorExample.java     |   76 +
 .../math/vector/VectorCustomStorageExample.java |  120 +
 .../examples/ml/math/vector/VectorExample.java  |   73 +
 .../examples/ml/math/vector/package-info.java   |   22 +
 .../examples/ScalarCacheAffinityExample.scala   |    2 +-
 .../examples/ScalarContinuationExample.scala    |   10 +-
 .../examples/spark/ScalarSharedRDDExample.scala |   89 +
 .../ignite/examples/spark/SharedRDDExample.java |  127 +
 .../ignite/examples/spark/package-info.java     |   22 +
 .../ignite/examples/CacheExamplesSelfTest.java  |    8 +
 .../examples/SpringDataExampleSelfTest.java     |   32 +
 .../testsuites/IgniteExamplesSelfTestSuite.java |    2 +
 .../examples/IndexingBridgeMethodTest.java      |   93 -
 .../IgniteExamplesJ8SelfTestSuite.java          |    3 +-
 .../tests/examples/ScalarExamplesSelfTest.scala |    6 +
 .../examples/SharedRDDExampleSelfTest.java      |   36 +
 .../IgniteExamplesSparkSelfTestSuite.java       |   46 +
 modules/aop/pom.xml                             |    2 +-
 .../aop/aspectj/GridifyAspectJAspect.java       |   15 +-
 .../aspectj/GridifySetToSetAspectJAspect.java   |   15 +-
 .../aspectj/GridifySetToValueAspectJAspect.java |   15 +-
 .../aop/spring/GridifySetToSetSpringAspect.java |   15 +-
 .../spring/GridifySetToValueSpringAspect.java   |   15 +-
 .../gridify/aop/spring/GridifySpringAspect.java |   15 +-
 .../apache/ignite/gridify/AbstractAopTest.java  |   10 +-
 .../ignite/gridify/GridifySetToSetTarget.java   |   18 +-
 .../gridify/GridifySetToSetTargetInterface.java |   18 +-
 .../ignite/gridify/GridifySetToValueTarget.java |   16 +-
 .../GridifySetToValueTargetInterface.java       |   16 +-
 .../ignite/gridify/NonSpringAopSelfTest.java    |    2 +-
 .../ignite/gridify/SpringAopSelfTest.java       |    2 +-
 .../apache/ignite/gridify/TestAopTarget.java    |   16 +-
 .../ignite/gridify/TestAopTargetInterface.java  |   16 +-
 .../gridify/hierarchy/GridifyHierarchyTest.java |    2 +-
 .../ignite/gridify/hierarchy/SuperTarget.java   |    4 +-
 .../optimized/OptimizedMarshallerAopTest.java   |  105 +
 .../optimized/OptimizedMarshallerAopTest.java   |  104 -
 .../apache/ignite/p2p/P2PGridifySelfTest.java   |    8 +-
 .../singlesplit/SingleSplitsLoadTest.java       |   10 +-
 .../gridify/GridifySingleSplitLoadTest.java     |    6 +-
 .../org/test/gridify/ExternalAopTarget.java     |   16 +-
 .../gridify/ExternalNonSpringAopSelfTest.java   |   12 +-
 modules/apache-license-gen/pom.xml              |    2 +-
 modules/aws/pom.xml                             |    8 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |   74 +-
 .../tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java |   31 +-
 .../s3/S3CheckpointManagerSelfTest.java         |    6 +-
 ...3IpFinderAwsCredentialsProviderSelfTest.java |    2 +-
 modules/benchmarks/pom.xml                      |    9 +-
 .../jmh/cache/JmhCacheAbstractBenchmark.java    |   13 +-
 .../jmh/future/JmhFutureAdapterBenchmark.java   |  145 +
 .../benchmarks/jmh/tree/BPlusTreeBenchmark.java |  327 ++
 modules/camel/pom.xml                           |    2 +-
 .../stream/camel/IgniteCamelStreamerTest.java   |   10 +-
 modules/cassandra/pom.xml                       |    2 +-
 modules/cassandra/serializers/pom.xml           |    4 +-
 modules/cassandra/store/pom.xml                 |    4 +-
 .../store/cassandra/CassandraCacheStore.java    |   31 +-
 .../cassandra/common/PropertyMappingHelper.java |   88 +-
 .../store/cassandra/datasource/DataSource.java  |   50 +-
 .../persistence/KeyPersistenceSettings.java     |  161 +-
 .../persistence/PersistenceSettings.java        |   81 +-
 .../store/cassandra/persistence/PojoField.java  |  146 +-
 .../persistence/PojoFieldAccessor.java          |  162 +
 .../cassandra/persistence/PojoKeyField.java     |   18 +-
 .../cassandra/persistence/PojoValueField.java   |   16 +-
 .../persistence/ValuePersistenceSettings.java   |   52 +-
 .../cassandra/session/CassandraSessionImpl.java |   23 +-
 .../session/LoadCacheCustomQueryWorker.java     |   26 +-
 .../cassandra/session/pool/SessionPool.java     |    6 +-
 .../cassandra/session/pool/SessionWrapper.java  |   15 +-
 .../store/cassandra/utils/DDLGenerator.java     |    9 +-
 .../ignite/ignite-cassandra-server-template.xml |    9 -
 .../tests/ignite-cassandra-client-template.xml  |    9 -
 .../tests/CassandraDirectPersistenceTest.java   |   81 +-
 .../apache/ignite/tests/DDLGeneratorTest.java   |    4 +
 .../ignite/tests/IgnitePersistentStoreTest.java |  181 +-
 .../org/apache/ignite/tests/pojos/Person.java   |   10 +-
 .../org/apache/ignite/tests/pojos/Product.java  |    7 -
 .../apache/ignite/tests/pojos/ProductOrder.java |    8 -
 .../apache/ignite/tests/pojos/SimplePerson.java |  186 +
 .../ignite/tests/pojos/SimplePersonId.java      |   89 +
 .../ignite/tests/utils/TestTransaction.java     |   11 +
 .../apache/ignite/tests/utils/TestsHelper.java  |  103 +-
 .../tests/persistence/blob/ignite-config.xml    |    4 +-
 .../persistence/loadall_blob/ignite-config.xml  |   90 +
 .../loadall_blob/persistence-settings.xml       |   29 +
 .../tests/persistence/pojo/ignite-config.xml    |   40 +-
 .../persistence/pojo/persistence-settings-5.xml |   21 +
 .../persistence/pojo/persistence-settings-6.xml |  174 +
 .../persistence/primitive/ignite-config.xml     |    4 +-
 .../primitive/ignite-remote-server-config.xml   |    4 +-
 modules/clients/pom.xml                         |    9 +-
 .../src/test/config/jdbc-config-cache-store.xml |  124 +
 modules/clients/src/test/config/jdbc-config.xml |    8 +-
 .../internal/TaskEventSubjectIdSelfTest.java    |    4 +-
 .../ClientAbstractMultiThreadedSelfTest.java    |   41 +-
 .../client/ClientDefaultCacheSelfTest.java      |   10 +-
 .../internal/client/ClientStartNodeTask.java    |   24 +-
 .../internal/client/ClientStopNodeTask.java     |    2 +-
 .../ClientTcpSslAuthenticationSelfTest.java     |    4 +-
 ...skExecutionAfterTopologyRestartSelfTest.java |    4 +-
 .../internal/client/ClientTestRestServer.java   |    6 +-
 .../ClientAbstractMultiNodeSelfTest.java        |   45 +-
 .../integration/ClientAbstractSelfTest.java     |   19 +-
 .../integration/ClientPreferDirectSelfTest.java |    4 +-
 .../ClientTcpUnreachableMultiNodeSelfTest.java  |    6 +-
 .../client/router/ClientFailedInitSelfTest.java |    4 +-
 .../client/router/RouterFactorySelfTest.java    |    4 +-
 .../client/suite/IgniteClientTestSuite.java     |    8 +-
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |  193 +-
 .../JdbcAbstractUpdateStatementSelfTest.java    |   11 +-
 ...BinaryMarshallerInsertStatementSelfTest.java |   11 +-
 ...cBinaryMarshallerMergeStatementSelfTest.java |   11 +-
 .../ignite/internal/jdbc2/JdbcBlobTest.java     |  485 ++
 .../jdbc2/JdbcComplexQuerySelfTest.java         |   91 +-
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |   73 +-
 .../jdbc2/JdbcDistributedJoinsQueryTest.java    |   12 +-
 .../jdbc2/JdbcDynamicIndexAbstractSelfTest.java |  369 ++
 ...namicIndexAtomicPartitionedNearSelfTest.java |   26 +
 ...bcDynamicIndexAtomicPartitionedSelfTest.java |   39 +
 ...dbcDynamicIndexAtomicReplicatedSelfTest.java |   39 +
 ...dexTransactionalPartitionedNearSelfTest.java |   26 +
 ...icIndexTransactionalPartitionedSelfTest.java |   39 +
 ...micIndexTransactionalReplicatedSelfTest.java |   39 +
 .../internal/jdbc2/JdbcEmptyCacheSelfTest.java  |    6 +-
 .../jdbc2/JdbcInsertStatementSelfTest.java      |   71 +-
 .../internal/jdbc2/JdbcLocalCachesSelfTest.java |   36 +-
 .../jdbc2/JdbcMergeStatementSelfTest.java       |   67 +-
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |  136 +-
 .../internal/jdbc2/JdbcNoDefaultCacheTest.java  |   53 +-
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |  110 +-
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |   36 +-
 .../internal/jdbc2/JdbcSpringSelfTest.java      |  123 +
 .../internal/jdbc2/JdbcStatementSelfTest.java   |   10 +-
 .../internal/jdbc2/JdbcStreamingSelfTest.java   |  185 +
 .../jdbc2/JdbcUpdateStatementSelfTest.java      |   50 +
 .../rest/AbstractRestProcessorSelfTest.java     |    6 +-
 .../rest/ChangeStateCommandHandlerTest.java     |  125 +
 .../rest/ClientMemcachedProtocolSelfTest.java   |    4 +-
 .../JettyRestProcessorAbstractSelfTest.java     |  534 +-
 .../rest/JettyRestProcessorSignedSelfTest.java  |    8 +-
 .../rest/RestBinaryProtocolSelfTest.java        |  105 +-
 .../rest/RestMemcacheProtocolSelfTest.java      |   51 +-
 .../rest/RestProcessorMultiStartSelfTest.java   |    4 +-
 .../rest/RestProcessorStartSelfTest.java        |    8 +-
 .../processors/rest/RestProcessorTest.java      |    2 +-
 .../rest/TaskCommandHandlerSelfTest.java        |   11 +-
 .../processors/rest/TestBinaryClient.java       |   23 +-
 .../processors/rest/TestMemcacheClient.java     |    2 +-
 .../rest/protocols/tcp/MockNioSession.java      |    5 +-
 .../tcp/redis/RedisCommonAbstractTest.java      |  133 +
 .../tcp/redis/RedisProtocolConnectSelfTest.java |   73 +
 .../tcp/redis/RedisProtocolSelfTest.java        |  559 --
 .../tcp/redis/RedisProtocolStringSelfTest.java  |  445 ++
 .../jdbc/AbstractJdbcPojoQuerySelfTest.java     |  167 +
 .../ignite/jdbc/JdbcComplexQuerySelfTest.java   |   50 +-
 .../ignite/jdbc/JdbcConnectionSelfTest.java     |   13 +-
 .../ignite/jdbc/JdbcEmptyCacheSelfTest.java     |    6 +-
 .../ignite/jdbc/JdbcLocalCachesSelfTest.java    |    6 +-
 .../ignite/jdbc/JdbcMetadataSelfTest.java       |  121 +-
 .../ignite/jdbc/JdbcNoDefaultCacheTest.java     |    9 +-
 .../jdbc/JdbcPojoLegacyQuerySelfTest.java       |   44 +
 .../ignite/jdbc/JdbcPojoQuerySelfTest.java      |   56 +
 .../jdbc/JdbcPreparedStatementSelfTest.java     |    8 +-
 .../ignite/jdbc/JdbcResultSetSelfTest.java      |  129 +-
 .../ignite/jdbc/JdbcStatementSelfTest.java      |    8 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   64 +-
 .../JdbcThinAbstractDmlStatementSelfTest.java   |  246 +
 .../jdbc/thin/JdbcThinAbstractSelfTest.java     |   27 +
 ...JdbcThinAbstractUpdateStatementSelfTest.java |   40 +
 .../thin/JdbcThinAutoCloseServerCursorTest.java |  359 ++
 .../jdbc/thin/JdbcThinComplexQuerySelfTest.java |  330 ++
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   |  331 ++
 .../thin/JdbcThinDeleteStatementSelfTest.java   |   49 +
 .../JdbcThinDynamicIndexAbstractSelfTest.java   |  328 ++
 ...namicIndexAtomicPartitionedNearSelfTest.java |   26 +
 ...inDynamicIndexAtomicPartitionedSelfTest.java |   39 +
 ...hinDynamicIndexAtomicReplicatedSelfTest.java |   39 +
 ...dexTransactionalPartitionedNearSelfTest.java |   26 +
 ...icIndexTransactionalPartitionedSelfTest.java |   39 +
 ...micIndexTransactionalReplicatedSelfTest.java |   39 +
 .../jdbc/thin/JdbcThinEmptyCacheSelfTest.java   |  143 +
 .../thin/JdbcThinInsertStatementSelfTest.java   |  185 +
 .../thin/JdbcThinMergeStatementSelfTest.java    |  130 +
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |  357 ++
 .../jdbc/thin/JdbcThinNoDefaultSchemaTest.java  |  158 +
 .../thin/JdbcThinPreparedStatementSelfTest.java |  768 +++
 .../jdbc/thin/JdbcThinResultSetSelfTest.java    |  762 +++
 .../jdbc/thin/JdbcThinStatementSelfTest.java    |  340 ++
 .../thin/JdbcThinUpdateStatementSelfTest.java   |   50 +
 modules/clients/src/test/resources/log4j.xml    |    2 +-
 .../clients/src/test/resources/spring-cache.xml |   10 -
 .../src/test/resources/spring-server-node.xml   |   73 +-
 .../test/resources/spring-server-ssl-node.xml   |   71 +-
 modules/cloud/pom.xml                           |    2 +-
 .../cloud/TcpDiscoveryCloudIpFinder.java        |   51 +-
 modules/codegen/pom.xml                         |    2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |    6 +
 modules/core/pom.xml                            |   19 +-
 modules/core/src/main/java/META-INF/NOTICE      |    2 +-
 .../main/java/META-INF/services/java.sql.Driver |    2 +
 .../src/main/java/org/apache/ignite/Ignite.java |  139 +-
 .../java/org/apache/ignite/IgniteBinary.java    |   58 +-
 .../java/org/apache/ignite/IgniteCache.java     |  772 ++-
 .../ignite/IgniteCacheRestartingException.java  |   62 +
 .../apache/ignite/IgniteCheckedException.java   |   12 +
 .../java/org/apache/ignite/IgniteCluster.java   |  178 +-
 .../java/org/apache/ignite/IgniteCompute.java   |  314 +-
 .../org/apache/ignite/IgniteDataStreamer.java   |   12 +-
 .../java/org/apache/ignite/IgniteEvents.java    |  128 +-
 .../org/apache/ignite/IgniteFileSystem.java     |   98 +-
 .../org/apache/ignite/IgniteJdbcDriver.java     |  116 +-
 .../org/apache/ignite/IgniteJdbcThinDriver.java |  281 +
 .../java/org/apache/ignite/IgniteMessaging.java |   38 +-
 .../java/org/apache/ignite/IgniteServices.java  |  221 +-
 .../apache/ignite/IgniteSystemProperties.java   |  188 +-
 .../main/java/org/apache/ignite/Ignition.java   |   43 +-
 .../org/apache/ignite/IgnitionListener.java     |    2 +-
 .../java/org/apache/ignite/MemoryMetrics.java   |  119 +
 .../org/apache/ignite/PersistenceMetrics.java   |  117 +
 .../binary/BinaryAbstractIdentityResolver.java  |   53 -
 .../binary/BinaryArrayIdentityResolver.java     |  224 -
 .../ignite/binary/BinaryBasicIdMapper.java      |    5 +-
 .../ignite/binary/BinaryBasicNameMapper.java    |   10 +-
 .../binary/BinaryFieldIdentityResolver.java     |  307 --
 .../ignite/binary/BinaryIdentityResolver.java   |   42 -
 .../org/apache/ignite/binary/BinaryObject.java  |    8 +
 .../ignite/binary/BinaryObjectBuilder.java      |   10 -
 .../apache/ignite/binary/BinaryRawReader.java   |   13 +
 .../apache/ignite/binary/BinaryRawWriter.java   |   13 +
 .../org/apache/ignite/binary/BinaryReader.java  |   15 +
 .../org/apache/ignite/binary/BinaryType.java    |    5 +
 .../ignite/binary/BinaryTypeConfiguration.java  |   69 +-
 .../org/apache/ignite/binary/BinaryWriter.java  |   15 +
 .../ignite/cache/CacheAtomicWriteOrderMode.java |   64 -
 .../org/apache/ignite/cache/CacheEntry.java     |   12 -
 .../ignite/cache/CacheKeyConfiguration.java     |   10 +-
 .../org/apache/ignite/cache/CacheManager.java   |    2 +-
 .../apache/ignite/cache/CacheMemoryMode.java    |   63 -
 .../org/apache/ignite/cache/CacheMetrics.java   |  111 +-
 .../org/apache/ignite/cache/CachePeekMode.java  |    6 +-
 .../ignite/cache/CacheTypeFieldMetadata.java    |  149 -
 .../apache/ignite/cache/CacheTypeMetadata.java  |  379 --
 .../cache/CacheWriteSynchronizationMode.java    |    4 +-
 .../ignite/cache/PartitionLossPolicy.java       |   80 +
 .../org/apache/ignite/cache/QueryEntity.java    |  253 +-
 .../org/apache/ignite/cache/QueryIndex.java     |   77 +-
 .../org/apache/ignite/cache/QueryIndexType.java |   17 +-
 .../cache/affinity/AffinityKeyMapped.java       |   14 +-
 .../cache/affinity/AffinityKeyMapper.java       |    3 +
 .../AffinityNodeAddressHashResolver.java        |   44 -
 .../affinity/AffinityNodeHashResolver.java      |   47 -
 .../affinity/AffinityNodeIdHashResolver.java    |   45 -
 .../affinity/fair/FairAffinityFunction.java     | 1144 -----
 .../cache/affinity/fair/package-info.java       |   21 -
 .../rendezvous/RendezvousAffinityFunction.java  |  389 +-
 .../cache/eviction/AbstractEvictionPolicy.java  |  212 +
 .../ignite/cache/eviction/EvictionPolicy.java   |    2 -
 .../cache/eviction/fifo/FifoEvictionPolicy.java |  202 +-
 .../eviction/fifo/FifoEvictionPolicyMBean.java  |    2 +
 .../igfs/IgfsPerBlockLruEvictionPolicy.java     |  131 +-
 .../cache/eviction/lru/LruEvictionPolicy.java   |  201 +-
 .../eviction/random/RandomEvictionPolicy.java   |  124 -
 .../random/RandomEvictionPolicyMBean.java       |   42 -
 .../eviction/sorted/SortedEvictionPolicy.java   |  207 +-
 .../ignite/cache/query/FieldsQueryCursor.java   |   39 +
 .../org/apache/ignite/cache/query/Query.java    |   48 +
 .../ignite/cache/query/SqlFieldsQuery.java      |   83 +-
 .../org/apache/ignite/cache/query/SqlQuery.java |   88 +-
 .../apache/ignite/cache/query/TextQuery.java    |    4 +-
 .../cache/query/annotations/QuerySqlField.java  |    2 +-
 .../cache/query/annotations/QueryTextField.java |    4 +-
 .../store/jdbc/CacheAbstractJdbcStore.java      |  233 +-
 .../cache/store/jdbc/CacheJdbcBlobStore.java    |    5 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   19 +-
 .../apache/ignite/cluster/ClusterMetrics.java   |    4 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   11 +-
 .../apache/ignite/compute/gridify/Gridify.java  |   10 +
 .../ignite/compute/gridify/GridifySetToSet.java |   10 +
 .../compute/gridify/GridifySetToValue.java      |   10 +
 .../configuration/AtomicConfiguration.java      |   60 +-
 .../configuration/BinaryConfiguration.java      |   30 +-
 .../configuration/CacheConfiguration.java       | 1001 ++--
 .../configuration/CollectionConfiguration.java  |   68 +-
 .../configuration/ConnectorConfiguration.java   |  100 +-
 .../configuration/DataPageEvictionMode.java     |   68 +
 .../configuration/ExecutorConfiguration.java    |  115 +
 .../configuration/FileSystemConfiguration.java  |  419 +-
 .../configuration/HadoopConfiguration.java      |   31 +-
 .../configuration/IgniteConfiguration.java      |  432 +-
 .../configuration/MemoryConfiguration.java      |  303 ++
 .../MemoryPolicyConfiguration.java              |  376 ++
 .../configuration/NearCacheConfiguration.java   |    7 +-
 .../ignite/configuration/OdbcConfiguration.java |    4 +
 .../PersistentStoreConfiguration.java           |  540 ++
 .../SqlConnectorConfiguration.java              |  275 +
 .../ignite/configuration/TopologyValidator.java |   10 +-
 .../configuration/TransactionConfiguration.java |   45 +-
 .../apache/ignite/configuration/WALMode.java    |   60 +
 .../org/apache/ignite/events/CacheEvent.java    |    3 -
 .../java/org/apache/ignite/events/Event.java    |    1 -
 .../org/apache/ignite/events/EventType.java     |  111 +-
 .../apache/ignite/events/SwapSpaceEvent.java    |  105 -
 .../org/apache/ignite/hadoop/package-info.java  |   22 +
 .../igfs/IgfsGroupDataBlocksKeyMapper.java      |    5 +-
 .../igfs/IgfsIpcEndpointConfiguration.java      |   30 +-
 .../org/apache/ignite/igfs/IgfsMetrics.java     |    5 +-
 .../java/org/apache/ignite/igfs/IgfsPath.java   |   31 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |    4 +-
 .../local/LocalIgfsSecondaryFileSystem.java     |    8 +-
 .../internal/ComputeTaskInternalFuture.java     |    3 -
 .../ignite/internal/ExecutorAwareMessage.java   |   31 +
 .../ignite/internal/GridCachePluginContext.java |   13 +-
 .../apache/ignite/internal/GridComponent.java   |   14 +-
 .../apache/ignite/internal/GridDiagnostic.java  |   16 +-
 .../internal/GridEventConsumeHandler.java       |   10 +-
 .../ignite/internal/GridJobCancelRequest.java   |    2 +-
 .../ignite/internal/GridJobExecuteRequest.java  |   34 +-
 .../ignite/internal/GridJobExecuteResponse.java |    2 +-
 .../ignite/internal/GridJobSiblingImpl.java     |   34 +-
 .../ignite/internal/GridJobSiblingsRequest.java |    2 +-
 .../internal/GridJobSiblingsResponse.java       |    2 +-
 .../ignite/internal/GridKernalContext.java      |   78 +-
 .../ignite/internal/GridKernalContextImpl.java  |  123 +-
 .../ignite/internal/GridKernalGatewayImpl.java  |   47 +-
 .../apache/ignite/internal/GridLoggerProxy.java |   25 +-
 .../internal/GridMessageListenHandler.java      |   10 +-
 .../internal/GridPerformanceSuggestions.java    |   92 -
 .../ignite/internal/GridTaskCancelRequest.java  |    2 +-
 .../ignite/internal/GridTaskSessionImpl.java    |   15 +-
 .../ignite/internal/GridTaskSessionRequest.java |    2 +-
 .../org/apache/ignite/internal/GridTopic.java   |   16 +-
 .../ignite/internal/IgniteComputeImpl.java      |  609 ++-
 .../ignite/internal/IgniteDiagnosticAware.java  |   28 +
 .../ignite/internal/IgniteDiagnosticInfo.java   |   45 +
 .../internal/IgniteDiagnosticMessage.java       |  484 ++
 .../IgniteDiagnosticPrepareContext.java         |  279 +
 .../ignite/internal/IgniteEventsImpl.java       |   87 +-
 .../org/apache/ignite/internal/IgniteEx.java    |   46 +-
 .../ignite/internal/IgniteInternalFuture.java   |   15 -
 .../apache/ignite/internal/IgniteKernal.java    |  812 ++-
 .../ignite/internal/IgniteMessagingImpl.java    |   67 +-
 .../internal/IgniteNeedReconnectException.java  |   40 +
 .../ignite/internal/IgniteNodeAttributes.java   |   19 +-
 .../ignite/internal/IgniteServicesImpl.java     |  108 +
 .../ignite/internal/IgniteTransactionsEx.java   |    8 +-
 .../ignite/internal/IgniteVersionUtils.java     |    2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  403 +-
 .../apache/ignite/internal/LessNamingBean.java  |   28 -
 .../ignite/internal/MappingStoreTask.java       |    3 +-
 .../ignite/internal/MarshallerContextImpl.java  |  106 +-
 .../ignite/internal/MarshallerPlatformIds.java  |    3 +
 .../internal/StripedExecutorMXBeanAdapter.java  |   90 +
 .../binary/BinaryAbstractIdentityResolver.java  |   56 +
 .../binary/BinaryArrayIdentityResolver.java     |  222 +
 .../binary/BinaryCachingMetadataHandler.java    |   13 +
 .../internal/binary/BinaryClassDescriptor.java  |  142 +-
 .../ignite/internal/binary/BinaryContext.java   |  223 +-
 .../internal/binary/BinaryEnumObjectImpl.java   |   80 +-
 .../internal/binary/BinaryFieldAccessor.java    |   89 +-
 .../ignite/internal/binary/BinaryFieldEx.java   |   50 +
 .../ignite/internal/binary/BinaryFieldImpl.java |  180 +-
 .../internal/binary/BinaryFieldMetadata.java    |  127 +
 .../internal/binary/BinaryIdentityResolver.java |   43 +
 .../ignite/internal/binary/BinaryMetadata.java  |  141 +-
 .../binary/BinaryMetadataCollector.java         |   28 +-
 .../internal/binary/BinaryMetadataHandler.java  |   27 +-
 .../binary/BinaryNoopMetadataHandler.java       |   10 +
 .../internal/binary/BinaryObjectExImpl.java     |   20 +-
 .../internal/binary/BinaryObjectImpl.java       |  223 +-
 .../binary/BinaryObjectOffheapImpl.java         |   56 +-
 .../internal/binary/BinaryReaderExImpl.java     |  366 +-
 .../ignite/internal/binary/BinarySchema.java    |   11 +
 .../binary/BinarySerializedFieldComparator.java |    3 +
 .../ignite/internal/binary/BinaryTreeMap.java   |    6 +-
 .../ignite/internal/binary/BinaryTypeImpl.java  |   15 +
 .../ignite/internal/binary/BinaryTypeProxy.java |    9 +
 .../ignite/internal/binary/BinaryUtils.java     |  288 +-
 .../ignite/internal/binary/BinaryWriteMode.java |    6 +
 .../internal/binary/BinaryWriterExImpl.java     |  162 +-
 .../internal/binary/GridBinaryMarshaller.java   |   11 +-
 .../binary/builder/BinaryBuilderReader.java     |   45 +
 .../binary/builder/BinaryBuilderSerializer.java |   10 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |   89 +-
 .../streams/BinaryByteBufferInputStream.java    |  301 ++
 .../ignite/internal/client/GridClient.java      |    9 +
 .../internal/client/GridClientClusterState.java |   33 +
 .../client/GridClientConfiguration.java         |  100 +-
 .../client/GridClientDataConfiguration.java     |   15 +-
 .../ignite/internal/client/GridClientNode.java  |    9 +-
 .../client/GridClientPartitionAffinity.java     |   15 +-
 .../balancer/GridClientBalancerAdapter.java     |    5 +-
 .../client/impl/GridClientClusterStateImpl.java |   75 +
 .../client/impl/GridClientDataImpl.java         |    3 +-
 .../internal/client/impl/GridClientImpl.java    |   11 +
 .../client/impl/GridClientNodeImpl.java         |   22 +-
 .../impl/connection/GridClientConnection.java   |   17 +
 .../GridClientConnectionManagerAdapter.java     |    2 +-
 .../connection/GridClientNioTcpConnection.java  |   26 +-
 .../GridClientOptimizedMarshaller.java          |    2 +-
 .../GridClientZipOptimizedMarshaller.java       |    4 +-
 .../router/GridTcpRouterConfiguration.java      |   50 +-
 .../router/impl/GridRouterClientImpl.java       |    6 +
 .../client/router/impl/GridTcpRouterImpl.java   |    4 +-
 .../client/util/GridClientConsistentHash.java   |   14 +-
 .../internal/cluster/ClusterGroupAdapter.java   |   35 +-
 .../ignite/internal/cluster/ClusterGroupEx.java |    5 +-
 .../cluster/ClusterNodeLocalMapImpl.java        |    2 +-
 .../cluster/IgniteClusterAsyncImpl.java         |   39 +-
 .../internal/cluster/IgniteClusterImpl.java     |   64 +-
 .../internal/commandline/CommandHandler.java    |   89 +
 .../internal/direct/DirectMessageWriter.java    |    4 +-
 .../stream/v1/DirectByteBufferStreamImplV1.java |    8 +-
 .../stream/v2/DirectByteBufferStreamImplV2.java |   37 +-
 .../internal/executor/GridExecutorService.java  |    3 +-
 .../igfs/common/IgfsControlResponse.java        |   28 +
 .../internal/igfs/common/IgfsIpcCommand.java    |    5 +-
 .../ignite/internal/igfs/common/IgfsLogger.java |   60 +-
 .../internal/igfs/common/IgfsMarshaller.java    |   12 +
 .../igfs/common/IgfsModeResolverRequest.java    |   35 +
 .../internal/jdbc/JdbcDriverPropertyInfo.java   |   37 +
 .../internal/jdbc/thin/JdbcThinConnection.java  |  663 +++
 .../jdbc/thin/JdbcThinPreparedStatement.java    |  485 ++
 .../internal/jdbc/thin/JdbcThinResultSet.java   | 1714 +++++++
 .../jdbc/thin/JdbcThinResultSetMetadata.java    |  161 +
 .../internal/jdbc/thin/JdbcThinStatement.java   |  495 ++
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |  417 ++
 .../internal/jdbc/thin/JdbcThinUtils.java       |  226 +
 .../apache/ignite/internal/jdbc2/JdbcBlob.java  |  191 +
 .../ignite/internal/jdbc2/JdbcConnection.java   |  197 +-
 .../internal/jdbc2/JdbcPreparedStatement.java   |   38 +-
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |   48 +-
 .../ignite/internal/jdbc2/JdbcQueryTaskV2.java  |  406 --
 .../ignite/internal/jdbc2/JdbcResultSet.java    |   75 +-
 .../internal/jdbc2/JdbcSqlFieldsQuery.java      |   57 +
 .../ignite/internal/jdbc2/JdbcStatement.java    |   38 +-
 .../jdbc2/JdbcStreamedPreparedStatement.java    |   60 +
 .../apache/ignite/internal/jdbc2/JdbcUtils.java |    4 +-
 .../internal/managers/GridManagerAdapter.java   |   46 +-
 .../checkpoint/GridCheckpointManager.java       |   51 +-
 .../checkpoint/GridCheckpointRequest.java       |    4 +-
 .../managers/communication/GridIoManager.java   |  960 +++-
 .../managers/communication/GridIoMessage.java   |   20 +-
 .../communication/GridIoMessageFactory.java     |  128 +-
 .../managers/communication/GridIoPolicy.java    |    8 +-
 .../communication/GridIoUserMessage.java        |    2 +-
 .../communication/GridMessageListener.java      |    3 +-
 .../communication/IgniteIoTestMessage.java      |  375 +-
 .../deployment/GridDeploymentClassLoader.java   |    3 +-
 .../deployment/GridDeploymentCommunication.java |   16 +-
 .../deployment/GridDeploymentInfoBean.java      |    2 +-
 .../deployment/GridDeploymentRequest.java       |    2 +-
 .../deployment/GridDeploymentResponse.java      |    2 +-
 .../internal/managers/discovery/DiscoCache.java |  279 +
 .../discovery/DiscoveryCustomMessage.java       |   48 +
 .../discovery/GridDiscoveryManager.java         | 1170 ++---
 .../eventstorage/DiscoveryEventListener.java    |   33 +
 .../eventstorage/GridEventStorageManager.java   |  343 +-
 .../eventstorage/GridEventStorageMessage.java   |    2 +-
 .../failover/GridFailoverContextImpl.java       |   11 -
 .../managers/failover/GridFailoverManager.java  |    3 -
 .../managers/indexing/GridIndexingManager.java  |   63 +-
 .../swapspace/GridSwapSpaceManager.java         |  417 --
 .../managers/swapspace/package-info.java        |   22 -
 .../optimized/OptimizedClassDescriptor.java     | 1141 +++++
 .../optimized/OptimizedFieldType.java           |   50 +
 .../optimized/OptimizedMarshaller.java          |  314 ++
 .../optimized/OptimizedMarshallerIdMapper.java  |   33 +
 .../optimized/OptimizedMarshallerUtils.java     |  551 ++
 .../optimized/OptimizedObjectInputStream.java   | 1231 +++++
 .../optimized/OptimizedObjectOutputStream.java  |  875 ++++
 .../OptimizedObjectStreamRegistry.java          |  244 +
 .../marshaller/optimized/package-info.java      |   22 +
 .../internal/mem/DirectMemoryProvider.java      |   40 +
 .../ignite/internal/mem/DirectMemoryRegion.java |   41 +
 .../mem/IgniteOutOfMemoryException.java         |   51 +
 .../apache/ignite/internal/mem/UnsafeChunk.java |   66 +
 .../ignite/internal/mem/file/MappedFile.java    |  152 +
 .../mem/file/MappedFileMemoryProvider.java      |  148 +
 .../mem/unsafe/UnsafeMemoryProvider.java        |  106 +
 .../ignite/internal/pagemem/FullPageId.java     |  157 +
 .../internal/pagemem/PageIdAllocator.java       |   53 +
 .../ignite/internal/pagemem/PageIdUtils.java    |  201 +
 .../ignite/internal/pagemem/PageMemory.java     |   46 +
 .../ignite/internal/pagemem/PageSupport.java    |  114 +
 .../ignite/internal/pagemem/PageUtils.java      |  201 +
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |  820 +++
 ...ishSnapshotOperationAckDiscoveryMessage.java |   77 +
 .../pagemem/snapshot/SnapshotOperation.java     |  202 +
 .../pagemem/snapshot/SnapshotOperationType.java |   49 +
 ...artSnapshotOperationAckDiscoveryMessage.java |  149 +
 .../StartSnapshotOperationDiscoveryMessage.java |  192 +
 .../pagemem/store/IgnitePageStoreManager.java   |  196 +
 .../internal/pagemem/store/PageStore.java       |   96 +
 .../pagemem/wal/IgniteWriteAheadLogManager.java |  115 +
 .../internal/pagemem/wal/StorageException.java  |   44 +
 .../internal/pagemem/wal/WALIterator.java       |   29 +
 .../ignite/internal/pagemem/wal/WALPointer.java |   28 +
 .../internal/pagemem/wal/record/CacheState.java |  133 +
 .../pagemem/wal/record/CheckpointRecord.java    |  122 +
 .../internal/pagemem/wal/record/DataEntry.java  |  171 +
 .../internal/pagemem/wal/record/DataRecord.java |   70 +
 .../pagemem/wal/record/LazyDataEntry.java       |  116 +
 .../wal/record/MemoryRecoveryRecord.java        |   46 +
 .../pagemem/wal/record/PageSnapshot.java        |   81 +
 .../internal/pagemem/wal/record/TxRecord.java   |  129 +
 .../internal/pagemem/wal/record/WALRecord.java  |  258 +
 .../delta/DataPageInsertFragmentRecord.java     |   84 +
 .../wal/record/delta/DataPageInsertRecord.java  |   66 +
 .../wal/record/delta/DataPageRemoveRecord.java  |   67 +
 .../delta/DataPageSetFreeListPageRecord.java    |   62 +
 .../wal/record/delta/DataPageUpdateRecord.java  |   79 +
 .../record/delta/DeltaApplicationException.java |   35 +
 .../wal/record/delta/FixCountRecord.java        |   57 +
 .../record/delta/FixLeftmostChildRecord.java    |   58 +
 .../pagemem/wal/record/delta/FixRemoveId.java   |   62 +
 .../wal/record/delta/InitNewPageRecord.java     |   98 +
 .../wal/record/delta/InnerReplaceRecord.java    |   96 +
 .../pagemem/wal/record/delta/InsertRecord.java  |  110 +
 .../pagemem/wal/record/delta/MergeRecord.java   |   97 +
 .../wal/record/delta/MetaPageAddRootRecord.java |   57 +
 .../wal/record/delta/MetaPageCutRootRecord.java |   47 +
 .../wal/record/delta/MetaPageInitRecord.java    |   88 +
 .../delta/MetaPageInitRootInlineRecord.java     |   62 +
 .../record/delta/MetaPageInitRootRecord.java    |   61 +
 .../delta/MetaPageUpdateLastAllocatedIndex.java |   62 +
 ...aPageUpdateLastSuccessfulFullSnapshotId.java |   59 +
 .../MetaPageUpdateLastSuccessfulSnapshotId.java |   70 +
 .../delta/MetaPageUpdateNextSnapshotId.java     |   59 +
 .../MetaPageUpdatePartitionDataRecord.java      |  132 +
 .../wal/record/delta/NewRootInitRecord.java     |  115 +
 .../wal/record/delta/PageDeltaRecord.java       |   77 +
 .../delta/PageListMetaResetCountRecord.java     |   47 +
 .../record/delta/PagesListAddPageRecord.java    |   73 +
 .../delta/PagesListInitNewPageRecord.java       |  100 +
 .../record/delta/PagesListRemovePageRecord.java |   74 +
 .../record/delta/PagesListSetNextRecord.java    |   60 +
 .../delta/PagesListSetPreviousRecord.java       |   60 +
 .../record/delta/PartitionDestroyRecord.java    |   73 +
 .../record/delta/PartitionMetaStateRecord.java  |   82 +
 .../pagemem/wal/record/delta/RecycleRecord.java |   55 +
 .../pagemem/wal/record/delta/RemoveRecord.java  |   82 +
 .../pagemem/wal/record/delta/ReplaceRecord.java |   86 +
 .../record/delta/SplitExistingPageRecord.java   |   76 +
 .../record/delta/SplitForwardPageRecord.java    |  115 +
 .../record/delta/TrackingPageDeltaRecord.java   |   85 +
 .../processors/GridProcessorAdapter.java        |   10 +
 .../affinity/AffinityTopologyVersion.java       |   11 +-
 .../affinity/GridAffinityAssignmentCache.java   |   68 +-
 .../affinity/GridAffinityProcessor.java         |  102 +-
 .../cache/CacheAffinitySharedManager.java       | 1224 +++--
 .../CacheClientReconnectDiscoveryData.java      |  193 +
 .../cache/CacheClusterMetricsMXBeanImpl.java    |   85 +-
 .../internal/processors/cache/CacheData.java    |  191 +
 .../CacheDefaultBinaryAffinityKeyMapper.java    |   91 +-
 .../processors/cache/CacheEntryImplEx.java      |    8 +-
 .../cache/CacheEntryInfoCollection.java         |    2 +-
 .../cache/CacheEntryPredicateAdapter.java       |    9 +-
 .../cache/CacheEntryPredicateContainsValue.java |    2 +-
 .../cache/CacheEntrySerializablePredicate.java  |    2 +-
 .../cache/CacheEvictableEntryImpl.java          |   18 +-
 .../processors/cache/CacheEvictionEntry.java    |    2 +-
 .../processors/cache/CacheEvictionManager.java  |   63 +
 .../processors/cache/CacheGroupContext.java     |  965 ++++
 .../processors/cache/CacheGroupData.java        |  147 +
 .../processors/cache/CacheGroupDescriptor.java  |  210 +
 .../cache/CacheInvalidStateException.java       |   65 +
 .../cache/CacheInvokeDirectResult.java          |    2 +-
 .../cache/CacheJoinNodeDiscoveryData.java       |  159 +
 .../processors/cache/CacheLazyEntry.java        |    2 +-
 .../cache/CacheLocalMetricsMXBeanImpl.java      |   85 +-
 .../processors/cache/CacheMetricsImpl.java      |  301 +-
 .../processors/cache/CacheMetricsSnapshot.java  |  209 +-
 .../cache/CacheNodeCommonDiscoveryData.java     |  115 +
 .../internal/processors/cache/CacheObject.java  |   41 +-
 .../processors/cache/CacheObjectAdapter.java    |  127 +-
 .../cache/CacheObjectByteArrayImpl.java         |   36 +-
 .../processors/cache/CacheObjectContext.java    |  198 +-
 .../processors/cache/CacheObjectImpl.java       |   37 +-
 .../processors/cache/CacheObjectUtils.java      |  173 +
 .../cache/CacheObjectValueContext.java          |   50 +
 .../cache/CacheOffheapEvictionManager.java      |   67 +
 .../processors/cache/CacheOperationContext.java |   50 +-
 .../cache/CacheOsConflictResolutionManager.java |    2 +-
 .../cache/CachePartitionExchangeWorkerTask.java |   25 +
 .../processors/cache/CacheStoppedException.java |   35 +
 .../cache/CacheStoreBalancingWrapper.java       |    3 -
 .../internal/processors/cache/CacheType.java    |    7 +-
 .../cache/ChangeGlobalStateMessage.java         |  120 +
 .../ClientCacheChangeDiscoveryMessage.java      |  176 +
 .../ClientCacheChangeDummyDiscoveryMessage.java |  104 +
 .../cache/ClientCacheUpdateTimeout.java         |   44 +
 .../processors/cache/ClusterCachesInfo.java     | 1512 ++++++
 .../cache/ClusterCachesReconnectResult.java     |   61 +
 .../internal/processors/cache/ClusterState.java |   38 +
 .../cache/DynamicCacheChangeBatch.java          |   90 +-
 .../cache/DynamicCacheChangeRequest.java        |  206 +-
 .../cache/DynamicCacheDescriptor.java           |  257 +-
 .../processors/cache/EntryGetResult.java        |   95 +
 .../processors/cache/EntryGetWithTtlResult.java |   58 +
 .../processors/cache/ExchangeActions.java       |  434 ++
 .../processors/cache/GridCacheAdapter.java      | 1594 +++---
 .../cache/GridCacheAffinityManager.java         |   72 +-
 .../processors/cache/GridCacheAtomicFuture.java |    6 -
 .../cache/GridCacheAtomicVersionComparator.java |   24 +-
 .../processors/cache/GridCacheAttributes.java   |   59 +-
 .../cache/GridCacheBatchSwapEntry.java          |   76 -
 .../cache/GridCacheClearAllRunnable.java        |   56 +-
 .../cache/GridCacheCompoundFuture.java          |   63 +
 .../cache/GridCacheCompoundIdentityFuture.java  |   63 +
 .../cache/GridCacheConcurrentMap.java           |   73 +-
 .../cache/GridCacheConcurrentMapImpl.java       |  400 +-
 .../processors/cache/GridCacheContext.java      |  459 +-
 .../GridCacheDefaultAffinityKeyMapper.java      |   38 +-
 .../cache/GridCacheDeploymentManager.java       |   28 +-
 .../processors/cache/GridCacheEntryEx.java      |  127 +-
 .../processors/cache/GridCacheEntryInfo.java    |   99 +-
 .../GridCacheEntryInfoCollectSwapListener.java  |   70 -
 .../processors/cache/GridCacheEventManager.java |   74 +-
 .../cache/GridCacheEvictionManager.java         | 1912 +------
 .../cache/GridCacheEvictionRequest.java         |  248 -
 .../cache/GridCacheEvictionResponse.java        |  225 -
 .../processors/cache/GridCacheFuture.java       |   15 +
 .../cache/GridCacheFutureAdapter.java           |   62 +
 .../processors/cache/GridCacheGateway.java      |   55 +-
 .../cache/GridCacheGroupIdMessage.java          |  110 +
 .../processors/cache/GridCacheIdMessage.java    |  117 +
 .../processors/cache/GridCacheIoManager.java    |  843 +--
 .../cache/GridCacheLocalConcurrentMap.java      |   82 +
 .../processors/cache/GridCacheLogger.java       |    2 +-
 .../processors/cache/GridCacheManager.java      |    3 +-
 .../cache/GridCacheManagerAdapter.java          |    9 +-
 .../processors/cache/GridCacheMapEntry.java     | 3210 ++++++------
 .../cache/GridCacheMapEntryFactory.java         |    6 +-
 .../processors/cache/GridCacheMessage.java      |  114 +-
 .../cache/GridCacheMultiTxFuture.java           |    3 -
 .../processors/cache/GridCacheMvcc.java         |   23 +-
 .../processors/cache/GridCacheMvccManager.java  |  111 +-
 .../cache/GridCacheOffheapSwapEntry.java        |  191 -
 .../GridCachePartitionExchangeManager.java      | 1070 ++--
 .../processors/cache/GridCachePreloader.java    |   30 +-
 .../cache/GridCachePreloaderAdapter.java        |   53 +-
 .../processors/cache/GridCacheProcessor.java    | 2878 ++++++-----
 .../processors/cache/GridCacheProxyImpl.java    |  165 +-
 .../processors/cache/GridCacheReturn.java       |    8 +-
 .../cache/GridCacheSharedContext.java           |  186 +-
 .../cache/GridCacheSharedManagerAdapter.java    |   15 +
 .../cache/GridCacheSharedTtlCleanupManager.java |   11 +-
 .../processors/cache/GridCacheSwapEntry.java    |   82 -
 .../cache/GridCacheSwapEntryImpl.java           |  339 --
 .../processors/cache/GridCacheSwapListener.java |   33 -
 .../processors/cache/GridCacheSwapManager.java  | 2616 ----------
 .../cache/GridCacheTryPutFailedException.java   |   28 -
 .../processors/cache/GridCacheTtlManager.java   |  140 +-
 .../cache/GridCacheUpdateAtomicResult.java      |   96 +-
 .../processors/cache/GridCacheUtils.java        |  486 +-
 .../GridChangeGlobalStateMessageResponse.java   |  187 +
 .../cache/GridDeferredAckMessageSender.java     |   28 +-
 .../processors/cache/GridNoStorageCacheMap.java |   50 +-
 .../processors/cache/IgniteCacheFutureImpl.java |    8 +-
 .../cache/IgniteCacheOffheapManager.java        |  544 ++
 .../cache/IgniteCacheOffheapManagerImpl.java    | 2790 ++++++++++
 .../processors/cache/IgniteCacheProxy.java      |  858 +++-
 .../cache/IgniteFinishedCacheFutureImpl.java    |   32 +
 .../processors/cache/IgniteInternalCache.java   |  168 +-
 .../cache/IgniteRebalanceIterator.java          |   31 +
 .../processors/cache/IncompleteCacheObject.java |   84 +
 .../processors/cache/IncompleteObject.java      |   89 +
 .../processors/cache/KeyCacheObjectImpl.java    |   86 +-
 .../processors/cache/QueryCursorImpl.java       |   19 +-
 .../processors/cache/ReaderArguments.java       |   74 +
 .../processors/cache/StoredCacheData.java       |   93 +
 .../cache/affinity/GridCacheAffinityImpl.java   |    9 +-
 .../cache/binary/BinaryMetadataFileStore.java   |  114 +
 .../cache/binary/BinaryMetadataHolder.java      |   80 +
 .../cache/binary/BinaryMetadataTransport.java   |  683 +++
 .../binary/BinaryMetadataUpdatedListener.java   |   29 +
 .../CacheDefaultBinaryAffinityKeyMapper.java    |   51 -
 .../cache/binary/CacheObjectBinaryContext.java  |   16 +-
 .../binary/CacheObjectBinaryProcessor.java      |   31 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  702 +--
 .../binary/ClientMetadataRequestFuture.java     |  166 +
 .../cache/binary/IgniteBinaryImpl.java          |   26 +
 .../cache/binary/MetadataRequestMessage.java    |  124 +
 .../cache/binary/MetadataResponseMessage.java   |  215 +
 .../binary/MetadataUpdateAcceptedMessage.java   |   96 +
 .../binary/MetadataUpdateProposedMessage.java   |  224 +
 .../cache/binary/MetadataUpdateResult.java      |   96 +
 .../CacheDataStructuresManager.java             |   38 +-
 .../distributed/GridCacheCommittedTxInfo.java   |  117 -
 .../distributed/GridCacheTtlUpdateRequest.java  |    6 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |    9 +-
 .../distributed/GridCacheTxRecoveryRequest.java |    2 +-
 .../GridCacheTxRecoveryResponse.java            |    2 +-
 .../distributed/GridDistributedBaseMessage.java |    6 +-
 .../GridDistributedCacheAdapter.java            |   83 +-
 .../distributed/GridDistributedCacheEntry.java  |    8 +-
 .../distributed/GridDistributedLockRequest.java |   62 +-
 .../GridDistributedLockResponse.java            |    2 +-
 .../GridDistributedTxFinishRequest.java         |  219 +-
 .../GridDistributedTxFinishResponse.java        |   83 +-
 .../distributed/GridDistributedTxMapping.java   |  113 +-
 .../GridDistributedTxPrepareRequest.java        |  231 +-
 .../GridDistributedTxPrepareResponse.java       |   78 +-
 .../GridDistributedTxRemoteAdapter.java         |  504 +-
 .../GridDistributedUnlockRequest.java           |   20 +-
 .../dht/CacheDistributedGetFutureAdapter.java   |   11 +-
 .../dht/ClientCacheDhtTopologyFuture.java       |   78 +
 .../dht/GridCachePartitionedConcurrentMap.java  |  109 +-
 .../dht/GridClientPartitionTopology.java        |  386 +-
 .../dht/GridDhtAffinityAssignmentRequest.java   |   78 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |  173 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |   86 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  653 ++-
 .../distributed/dht/GridDhtCacheEntry.java      |  103 +-
 .../distributed/dht/GridDhtEmbeddedFuture.java  |    3 -
 .../cache/distributed/dht/GridDhtGetFuture.java |  171 +-
 .../distributed/dht/GridDhtGetSingleFuture.java |  181 +-
 .../distributed/dht/GridDhtLocalPartition.java  | 1024 ++--
 .../distributed/dht/GridDhtLockFuture.java      |   74 +-
 .../distributed/dht/GridDhtLockRequest.java     |   22 +-
 .../distributed/dht/GridDhtLockResponse.java    |    4 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |   71 -
 .../distributed/dht/GridDhtPartitionState.java  |    5 +-
 .../dht/GridDhtPartitionTopology.java           |  118 +-
 .../dht/GridDhtPartitionTopologyImpl.java       | 1568 ++++--
 .../distributed/dht/GridDhtTopologyFuture.java  |   13 +-
 .../dht/GridDhtTopologyFutureAdapter.java       |  195 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |  103 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   61 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |  253 +-
 .../dht/GridDhtTxFinishResponse.java            |   58 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |  164 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   66 +-
 .../dht/GridDhtTxOnePhaseCommitAckRequest.java  |   18 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  321 +-
 .../dht/GridDhtTxPrepareRequest.java            |  176 +-
 .../dht/GridDhtTxPrepareResponse.java           |  121 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   24 +-
 .../distributed/dht/GridDhtUnlockRequest.java   |   17 +-
 .../dht/GridPartitionedGetFuture.java           |  198 +-
 .../dht/GridPartitionedSingleGetFuture.java     |  282 +-
 .../GridDhtAtomicAbstractUpdateFuture.java      |  352 +-
 .../GridDhtAtomicAbstractUpdateRequest.java     |  409 +-
 .../dht/atomic/GridDhtAtomicCache.java          | 1651 +++---
 .../dht/atomic/GridDhtAtomicCacheEntry.java     |   58 -
 .../GridDhtAtomicDeferredUpdateResponse.java    |   74 +-
 .../dht/atomic/GridDhtAtomicNearResponse.java   |  314 ++
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |   71 -
 .../atomic/GridDhtAtomicSingleUpdateFuture.java |  104 +-
 .../GridDhtAtomicSingleUpdateRequest.java       |  317 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   86 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  362 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |  121 +-
 ...idNearAtomicAbstractSingleUpdateRequest.java |  487 +-
 .../GridNearAtomicAbstractUpdateFuture.java     |  651 ++-
 .../GridNearAtomicAbstractUpdateRequest.java    |  554 +-
 .../GridNearAtomicCheckUpdateRequest.java       |  175 +
 .../atomic/GridNearAtomicFullUpdateRequest.java |  496 +-
 ...GridNearAtomicSingleUpdateFilterRequest.java |   38 +-
 .../GridNearAtomicSingleUpdateFuture.java       |  639 +--
 ...GridNearAtomicSingleUpdateInvokeRequest.java |   52 +-
 .../GridNearAtomicSingleUpdateRequest.java      |   77 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 1003 ++--
 .../atomic/GridNearAtomicUpdateResponse.java    |  366 +-
 .../dht/atomic/NearCacheUpdates.java            |  335 ++
 .../distributed/dht/atomic/UpdateErrors.java    |  222 +
 .../dht/colocated/GridDhtColocatedCache.java    |  285 +-
 .../colocated/GridDhtColocatedCacheEntry.java   |   57 -
 .../colocated/GridDhtColocatedLockFuture.java   |  184 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |   71 -
 .../colocated/GridDhtDetachedCacheEntry.java    |   28 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   32 +-
 .../dht/preloader/GridDhtForceKeysRequest.java  |    6 +-
 .../dht/preloader/GridDhtForceKeysResponse.java |   10 +-
 .../GridDhtPartitionDemandMessage.java          |  110 +-
 .../dht/preloader/GridDhtPartitionDemander.java |  953 ++--
 .../preloader/GridDhtPartitionExchangeId.java   |    2 +-
 .../dht/preloader/GridDhtPartitionFullMap.java  |   45 +-
 .../dht/preloader/GridDhtPartitionMap.java      |  227 +-
 .../dht/preloader/GridDhtPartitionMap2.java     |  329 --
 .../dht/preloader/GridDhtPartitionSupplier.java |  694 +--
 .../GridDhtPartitionSupplyMessage.java          |  264 +-
 .../GridDhtPartitionSupplyMessageV2.java        |  384 --
 .../GridDhtPartitionsAbstractMessage.java       |   44 +-
 .../GridDhtPartitionsExchangeFuture.java        | 1160 ++++-
 .../preloader/GridDhtPartitionsFullMessage.java |  222 +-
 .../GridDhtPartitionsSingleMessage.java         |  178 +-
 .../GridDhtPartitionsSingleRequest.java         |   17 +-
 .../dht/preloader/GridDhtPreloader.java         |  574 +--
 .../IgniteDhtPartitionCountersMap.java          |   64 +
 .../IgniteDhtPartitionHistorySuppliersMap.java  |  110 +
 .../IgniteDhtPartitionsToReloadMap.java         |   88 +
 .../distributed/near/CacheVersionedValue.java   |    2 +-
 .../distributed/near/GridNearAtomicCache.java   |   50 +-
 .../distributed/near/GridNearCacheAdapter.java  |  111 +-
 .../distributed/near/GridNearCacheEntry.java    |   34 +-
 .../distributed/near/GridNearGetFuture.java     |   62 +-
 .../distributed/near/GridNearGetRequest.java    |  170 +-
 .../distributed/near/GridNearGetResponse.java   |    6 +-
 .../distributed/near/GridNearLockFuture.java    |   70 +-
 .../distributed/near/GridNearLockRequest.java   |  235 +-
 .../distributed/near/GridNearLockResponse.java  |   14 +-
 .../near/GridNearOffHeapCacheEntry.java         |   60 -
 ...arOptimisticSerializableTxPrepareFuture.java |  280 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |  160 +-
 ...ridNearOptimisticTxPrepareFutureAdapter.java |   20 +-
 .../GridNearPessimisticTxPrepareFuture.java     |  242 +-
 .../near/GridNearSingleGetRequest.java          |   83 +-
 .../near/GridNearSingleGetResponse.java         |    8 +-
 .../near/GridNearTransactionalCache.java        |   23 +-
 .../near/GridNearTxFinishFuture.java            |  196 +-
 .../near/GridNearTxFinishRequest.java           |  176 +-
 .../near/GridNearTxFinishResponse.java          |   38 +-
 .../cache/distributed/near/GridNearTxLocal.java | 2773 +++++++++-
 .../near/GridNearTxPrepareFutureAdapter.java    |   55 +-
 .../near/GridNearTxPrepareRequest.java          |  256 +-
 .../near/GridNearTxPrepareResponse.java         |  102 +-
 .../distributed/near/GridNearTxRemote.java      |   12 +-
 .../distributed/near/GridNearUnlockRequest.java |   22 +-
 .../distributed/near/IgniteTxMappingsImpl.java  |    4 +-
 .../near/IgniteTxMappingsSingleImpl.java        |    6 +-
 .../cache/dr/GridOsCacheDrManager.java          |    2 +-
 .../extras/GridCacheAttributesEntryExtras.java  |    7 +-
 .../GridCacheAttributesMvccEntryExtras.java     |    5 +-
 ...dCacheAttributesMvccObsoleteEntryExtras.java |    5 +-
 ...cheAttributesMvccObsoleteTtlEntryExtras.java |    7 +-
 .../GridCacheAttributesMvccTtlEntryExtras.java  |    7 +-
 .../GridCacheAttributesObsoleteEntryExtras.java |    5 +-
 ...idCacheAttributesObsoleteTtlEntryExtras.java |    7 +-
 .../GridCacheAttributesTtlEntryExtras.java      |    7 +-
 .../cache/extras/GridCacheMvccEntryExtras.java  |    3 +-
 .../GridCacheMvccObsoleteEntryExtras.java       |    5 +-
 .../GridCacheMvccObsoleteTtlEntryExtras.java    |    7 +-
 .../extras/GridCacheMvccTtlEntryExtras.java     |    7 +-
 .../extras/GridCacheObsoleteEntryExtras.java    |    3 +-
 .../extras/GridCacheObsoleteTtlEntryExtras.java |    7 +-
 .../cache/extras/GridCacheTtlEntryExtras.java   |    5 +-
 .../processors/cache/local/GridLocalCache.java  |   77 +-
 .../cache/local/GridLocalCacheEntry.java        |   26 +-
 .../cache/local/GridLocalLockFuture.java        |   55 +-
 .../local/atomic/GridLocalAtomicCache.java      |  285 +-
 .../cache/persistence/CacheDataRow.java         |   57 +
 .../cache/persistence/CacheDataRowAdapter.java  |  593 +++
 .../cache/persistence/CacheSearchRow.java       |   45 +
 .../persistence/CheckpointLockStateChecker.java |   28 +
 .../cache/persistence/DataStructure.java        |  384 ++
 .../cache/persistence/DbCheckpointListener.java |   38 +
 .../FullPageIdIterableComparator.java           |   51 +
 .../GridCacheDatabaseSharedManager.java         | 3097 +++++++++++
 .../persistence/GridCacheOffheapManager.java    | 1321 +++++
 .../IgniteCacheDatabaseSharedManager.java       |  964 ++++
 .../persistence/IgniteCacheSnapshotManager.java |  140 +
 .../cache/persistence/MemoryMetricsImpl.java    |  284 ++
 .../persistence/MemoryMetricsMXBeanImpl.java    |  131 +
 .../persistence/MemoryMetricsSnapshot.java      |  112 +
 .../cache/persistence/MemoryPolicy.java         |   83 +
 .../processors/cache/persistence/MetaStore.java |   51 +
 .../cache/persistence/MetadataStorage.java      |  417 ++
 .../persistence/PersistenceMetricsImpl.java     |  297 ++
 .../persistence/PersistenceMetricsSnapshot.java |  144 +
 .../processors/cache/persistence/RootPage.java  |   62 +
 .../processors/cache/persistence/RowStore.java  |  107 +
 .../evict/FairFifoPageEvictionTracker.java      |   76 +
 .../evict/NoOpPageEvictionTracker.java          |   50 +
 .../evict/PageAbstractEvictionTracker.java      |  171 +
 .../persistence/evict/PageEvictionTracker.java  |   52 +
 .../evict/Random2LruPageEvictionTracker.java    |  180 +
 .../evict/RandomLruPageEvictionTracker.java     |  159 +
 .../cache/persistence/file/FilePageStore.java   |  529 ++
 .../persistence/file/FilePageStoreManager.java  |  671 +++
 .../cache/persistence/freelist/FreeList.java    |   51 +
 .../persistence/freelist/FreeListImpl.java      |  599 +++
 .../cache/persistence/freelist/PagesList.java   | 1482 ++++++
 .../freelist/io/PagesListMetaIO.java            |  186 +
 .../freelist/io/PagesListNodeIO.java            |  231 +
 .../pagemem/CheckpointMetricsTracker.java       |  183 +
 .../persistence/pagemem/EvictCandidate.java     |   77 +
 .../persistence/pagemem/FullPageIdTable.java    |  556 ++
 .../cache/persistence/pagemem/PageMemoryEx.java |  149 +
 .../persistence/pagemem/PageMemoryImpl.java     | 2322 +++++++++
 .../cache/persistence/tree/BPlusTree.java       | 4808 ++++++++++++++++++
 .../cache/persistence/tree/io/BPlusIO.java      |  403 ++
 .../cache/persistence/tree/io/BPlusInnerIO.java |  180 +
 .../cache/persistence/tree/io/BPlusLeafIO.java  |   56 +
 .../cache/persistence/tree/io/BPlusMetaIO.java  |  182 +
 .../persistence/tree/io/CacheVersionIO.java     |  255 +
 .../cache/persistence/tree/io/DataPageIO.java   | 1450 ++++++
 .../persistence/tree/io/DataPagePayload.java    |   64 +
 .../cache/persistence/tree/io/IOVersions.java   |   99 +
 .../cache/persistence/tree/io/PageIO.java       |  548 ++
 .../cache/persistence/tree/io/PageMetaIO.java   |  222 +
 .../tree/io/PagePartitionCountersIO.java        |  175 +
 .../tree/io/PagePartitionMetaIO.java            |  163 +
 .../persistence/tree/io/TrackingPageIO.java     |  334 ++
 .../cache/persistence/tree/reuse/ReuseBag.java  |   33 +
 .../cache/persistence/tree/reuse/ReuseList.java |   43 +
 .../persistence/tree/reuse/ReuseListImpl.java   |   95 +
 .../persistence/tree/util/PageHandler.java      |  509 ++
 .../persistence/tree/util/PageLockListener.java |   69 +
 .../wal/ByteBufferBackedDataInput.java          |   40 +
 .../wal/ByteBufferBackedDataInputImpl.java      |  173 +
 .../cache/persistence/wal/FileInput.java        |  453 ++
 .../cache/persistence/wal/FileWALPointer.java   |  139 +
 .../wal/FileWriteAheadLogManager.java           | 2540 +++++++++
 .../cache/persistence/wal/RecordSerializer.java |   52 +
 .../persistence/wal/SegmentEofException.java    |   37 +
 .../IgniteDataIntegrityViolationException.java  |   41 +
 .../persistence/wal/crc/PureJavaCrc32.java      |  655 +++
 .../persistence/wal/record/HeaderRecord.java    |   50 +
 .../wal/serializer/RecordV1Serializer.java      | 1654 ++++++
 .../query/GridCacheDistributedQueryFuture.java  |   44 +-
 .../query/GridCacheDistributedQueryManager.java |   33 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |    3 +-
 .../cache/query/GridCacheQueryAdapter.java      |   13 +-
 .../query/GridCacheQueryFutureAdapter.java      |   49 +-
 .../cache/query/GridCacheQueryManager.java      |  972 +---
 .../cache/query/GridCacheQueryRequest.java      |   19 +-
 .../cache/query/GridCacheQueryResponse.java     |   55 +-
 .../cache/query/GridCacheSqlIndexMetadata.java  |    3 +-
 .../cache/query/GridCacheSqlMetadata.java       |    3 +-
 .../cache/query/GridCacheSqlQuery.java          |  224 +-
 .../cache/query/IgniteQueryErrorCode.java       |   27 +-
 .../CacheContinuousQueryAcknowledgeBuffer.java  |  120 +
 .../CacheContinuousQueryBatchAck.java           |   10 +-
 .../CacheContinuousQueryDeployableObject.java   |  110 +
 .../continuous/CacheContinuousQueryEntry.java   |  119 +-
 .../continuous/CacheContinuousQueryEvent.java   |    3 +-
 .../CacheContinuousQueryEventBuffer.java        |  486 ++
 .../continuous/CacheContinuousQueryHandler.java |  852 +---
 .../CacheContinuousQueryHandlerV2.java          |    6 +-
 .../CacheContinuousQueryListener.java           |   20 +
 .../continuous/CacheContinuousQueryManager.java |  178 +-
 .../CacheContinuousQueryPartitionRecovery.java  |  267 +
 .../query/continuous/CounterSkipContext.java    |   78 +
 .../query/jdbc/GridCacheQueryJdbcTask.java      |   10 +-
 .../cache/ratemetrics/HitRateMetrics.java       |  179 +
 .../ratemetrics/HitRateMetricsSandbox.java      |   95 +
 .../cache/store/CacheOsStoreManager.java        |    3 +
 .../cache/store/CacheStoreManager.java          |   15 +-
 .../store/GridCacheStoreManagerAdapter.java     |  247 +-
 .../cache/store/GridCacheWriteBehindStore.java  |  639 ++-
 .../cache/transactions/IgniteInternalTx.java    |   80 +-
 .../transactions/IgniteTransactionsImpl.java    |   16 +-
 .../cache/transactions/IgniteTxAdapter.java     |  230 +-
 .../cache/transactions/IgniteTxEntry.java       |  134 +-
 .../cache/transactions/IgniteTxHandler.java     |  376 +-
 .../IgniteTxImplicitSingleStateImpl.java        |   31 +-
 .../cache/transactions/IgniteTxKey.java         |    2 +-
 .../transactions/IgniteTxLocalAdapter.java      | 2828 +---------
 .../cache/transactions/IgniteTxLocalEx.java     |  145 +-
 .../cache/transactions/IgniteTxManager.java     |  364 +-
 .../cache/transactions/IgniteTxRemoteEx.java    |   11 +
 .../IgniteTxRemoteSingleStateImpl.java          |    2 +-
 .../IgniteTxRemoteStateAdapter.java             |   17 +-
 .../cache/transactions/IgniteTxState.java       |   19 +-
 .../cache/transactions/IgniteTxStateImpl.java   |  101 +-
 .../transactions/TransactionProxyImpl.java      |   52 +-
 .../cache/transactions/TxDeadlock.java          |   21 +-
 .../cache/transactions/TxDeadlockDetection.java |    5 +-
 .../cache/transactions/TxEntryValueHolder.java  |    2 +-
 .../processors/cache/transactions/TxLock.java   |    4 +-
 .../cache/transactions/TxLockList.java          |    2 +-
 .../cache/transactions/TxLocksRequest.java      |   22 +-
 .../cache/transactions/TxLocksResponse.java     |   67 +-
 .../version/CacheVersionConflictResolver.java   |    2 +
 .../GridCacheLazyPlainVersionedEntry.java       |    5 +-
 .../version/GridCachePlainVersionedEntry.java   |    8 +-
 .../version/GridCacheRawVersionedEntry.java     |   12 +-
 .../cache/version/GridCacheVersion.java         |   51 +-
 .../GridCacheVersionConflictContext.java        |   16 +-
 .../cache/version/GridCacheVersionEx.java       |   21 +-
 .../cache/version/GridCacheVersionManager.java  |   14 +-
 .../cache/version/GridCacheVersionedEntry.java  |   11 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |   56 +-
 .../IgniteCacheObjectProcessorImpl.java         |  171 +-
 .../processors/cacheobject/NoOpBinary.java      |   12 +
 .../clock/GridClockDeltaSnapshot.java           |  235 -
 .../clock/GridClockDeltaSnapshotMessage.java    |  154 -
 .../processors/clock/GridClockDeltaVersion.java |  194 -
 .../processors/clock/GridClockMessage.java      |  171 -
 .../processors/clock/GridClockServer.java       |  222 -
 .../processors/clock/GridClockSource.java       |   30 -
 .../clock/GridClockSyncProcessor.java           |  481 --
 .../processors/clock/GridJvmClockSource.java    |   28 -
 .../processors/closure/AffinityTask.java        |    6 -
 .../closure/GridClosureProcessor.java           |  602 +--
 .../processors/cluster/ClusterProcessor.java    |  315 +-
 .../cluster/GridClusterStateProcessor.java      | 1223 +++++
 .../processors/cluster/GridUpdateNotifier.java  |   66 +-
 .../cluster/HttpIgniteUpdatesChecker.java       |   82 +
 .../cluster/IgniteChangeGlobalStateSupport.java |   42 +
 .../continuous/GridContinuousBatchAdapter.java  |    2 +-
 .../continuous/GridContinuousHandler.java       |   10 +-
 .../continuous/GridContinuousMessage.java       |    2 +-
 .../continuous/GridContinuousProcessor.java     |   46 +-
 .../continuous/GridContinuousQueryBatch.java    |   16 +-
 .../StartRoutineAckDiscoveryMessage.java        |   13 +-
 .../StartRoutineDiscoveryMessage.java           |   19 +-
 .../datastreamer/DataStreamProcessor.java       |    8 +-
 .../datastreamer/DataStreamerEntry.java         |    2 +-
 .../datastreamer/DataStreamerFuture.java        |    3 -
 .../datastreamer/DataStreamerImpl.java          |  129 +-
 .../datastreamer/DataStreamerRequest.java       |    2 +-
 .../datastreamer/DataStreamerResponse.java      |    2 +-
 .../AtomicDataStructureValue.java               |   35 +
 .../CacheDataStructuresCacheKey.java            |   64 -
 .../CacheDataStructuresConfigurationKey.java    |   64 -
 .../datastructures/DataStructureInfoKey.java    |   58 +
 .../datastructures/DataStructureType.java       |  105 +
 .../datastructures/DataStructuresCacheKey.java  |   44 +
 .../datastructures/DataStructuresProcessor.java | 2214 +++-----
 .../DistributedCollectionMetadata.java          |   90 +
 .../GridAtomicCacheQueueImpl.java               |    7 +-
 .../datastructures/GridCacheAtomicLongImpl.java |  631 ++-
 .../GridCacheAtomicLongValue.java               |    8 +-
 .../GridCacheAtomicReferenceImpl.java           |  369 +-
 .../GridCacheAtomicReferenceValue.java          |    9 +-
 .../GridCacheAtomicSequenceImpl.java            |  240 +-
 .../GridCacheAtomicSequenceValue.java           |    8 +-
 .../GridCacheAtomicStampedImpl.java             |  362 +-
 .../GridCacheAtomicStampedValue.java            |    9 +-
 .../GridCacheCountDownLatchImpl.java            |   87 +-
 .../GridCacheCountDownLatchValue.java           |   23 +-
 .../datastructures/GridCacheInternalKey.java    |    6 +-
 .../GridCacheInternalKeyImpl.java               |   36 +-
 .../datastructures/GridCacheLockImpl.java       |  128 +-
 .../datastructures/GridCacheLockState.java      |   23 +-
 .../datastructures/GridCacheQueueAdapter.java   |   49 +-
 .../datastructures/GridCacheQueueProxy.java     |  310 +-
 .../datastructures/GridCacheSemaphoreImpl.java  |  352 +-
 .../datastructures/GridCacheSemaphoreState.java |   44 +-
 .../datastructures/GridCacheSetImpl.java        |   54 +-
 .../datastructures/GridCacheSetProxy.java       |  170 +-
 .../GridTransactionalCacheQueueImpl.java        |   18 +-
 .../VolatileAtomicDataStructureValue.java       |   32 +
 .../processors/hadoop/HadoopClasspathUtils.java |    3 +-
 .../internal/processors/hadoop/HadoopJobId.java |   12 +-
 .../processors/hadoop/HadoopPayloadAware.java   |   28 -
 .../shuffle/HadoopDirectShuffleMessage.java     |   24 +-
 .../hadoop/shuffle/HadoopShuffleAck.java        |   12 +-
 .../shuffle/HadoopShuffleFinishRequest.java     |    2 +-
 .../shuffle/HadoopShuffleFinishResponse.java    |    2 +-
 .../hadoop/shuffle/HadoopShuffleMessage.java    |   20 +-
 .../processors/igfs/IgfsAckMessage.java         |    2 +-
 .../internal/processors/igfs/IgfsAsyncImpl.java |   56 +-
 .../internal/processors/igfs/IgfsBlockKey.java  |    3 +-
 .../processors/igfs/IgfsBlocksMessage.java      |    2 +-
 .../internal/processors/igfs/IgfsContext.java   |   21 +-
 .../processors/igfs/IgfsDataManager.java        |  115 +-
 .../processors/igfs/IgfsDeleteMessage.java      |    2 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |    7 -
 .../processors/igfs/IgfsFileAffinityRange.java  |    2 +-
 .../igfs/IgfsFragmentizerManager.java           |    8 +-
 .../igfs/IgfsFragmentizerRequest.java           |    2 +-
 .../igfs/IgfsFragmentizerResponse.java          |    2 +-
 .../processors/igfs/IgfsHandshakeResponse.java  |   22 +-
 .../internal/processors/igfs/IgfsImpl.java      |  161 +-
 .../processors/igfs/IgfsIpcHandler.java         |   24 +-
 .../processors/igfs/IgfsMetaManager.java        |  106 +-
 .../processors/igfs/IgfsModeResolver.java       |   91 +-
 .../processors/igfs/IgfsNoopProcessor.java      |    6 +-
 .../internal/processors/igfs/IgfsPaths.java     |  152 -
 .../internal/processors/igfs/IgfsProcessor.java |  180 +-
 .../processors/igfs/IgfsProcessorAdapter.java   |    4 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |    4 +-
 .../internal/processors/igfs/IgfsServer.java    |    6 +-
 .../processors/igfs/IgfsServerManager.java      |    2 +-
 .../processors/igfs/IgfsSyncMessage.java        |    2 +-
 .../processors/igfs/IgfsThreadFactory.java      |    6 +-
 .../internal/processors/igfs/IgfsUtils.java     |  216 +-
 .../igfs/client/IgfsClientAbstractCallable.java |   23 +-
 .../igfs/client/IgfsClientAffinityCallable.java |    6 +-
 .../igfs/client/IgfsClientDeleteCallable.java   |    6 +-
 .../igfs/client/IgfsClientExistsCallable.java   |    5 +-
 .../igfs/client/IgfsClientInfoCallable.java     |    5 +-
 .../client/IgfsClientListFilesCallable.java     |    5 +-
 .../client/IgfsClientListPathsCallable.java     |    5 +-
 .../igfs/client/IgfsClientMkdirsCallable.java   |    6 +-
 .../igfs/client/IgfsClientRenameCallable.java   |    6 +-
 .../igfs/client/IgfsClientSetTimesCallable.java |    8 +-
 .../igfs/client/IgfsClientSizeCallable.java     |    5 +-
 .../igfs/client/IgfsClientSummaryCallable.java  |    5 +-
 .../igfs/client/IgfsClientUpdateCallable.java   |    6 +-
 .../meta/IgfsClientMetaIdsForPathCallable.java  |    5 +-
 .../meta/IgfsClientMetaInfoForPathCallable.java |    5 +-
 .../meta/IgfsClientMetaUnlockCallable.java      |    8 +-
 .../IgfsMetaDirectoryListingAddProcessor.java   |    5 +-
 .../secondary/local/LocalFileSystemUtils.java   |   25 +
 .../processors/job/GridJobProcessor.java        |   58 +-
 .../internal/processors/job/GridJobWorker.java  |   25 +-
 .../jobmetrics/GridJobMetricsProcessor.java     |    7 +-
 .../marshaller/ClientRequestFuture.java         |    4 +-
 .../GridMarshallerMappingProcessor.java         |   50 +-
 .../marshaller/MappingUpdatedListener.java      |   29 +
 .../marshaller/MarshallerMappingTransport.java  |   17 +-
 .../MissingMappingRequestMessage.java           |    8 +-
 .../MissingMappingResponseMessage.java          |   19 +-
 .../internal/processors/odbc/IgniteTypes.java   |   69 -
 .../processors/odbc/OdbcBufferedParser.java     |   81 -
 .../processors/odbc/OdbcColumnMeta.java         |  110 -
 .../processors/odbc/OdbcHandshakeRequest.java   |   83 -
 .../processors/odbc/OdbcHandshakeResult.java    |   73 -
 .../processors/odbc/OdbcMessageParser.java      |  311 --
 .../processors/odbc/OdbcNioListener.java        |  179 -
 .../processors/odbc/OdbcNioServerBuffer.java    |  114 -
 .../internal/processors/odbc/OdbcProcessor.java |  200 -
 .../processors/odbc/OdbcProtocolVersion.java    |  125 -
 .../processors/odbc/OdbcQueryCloseRequest.java  |   49 -
 .../processors/odbc/OdbcQueryCloseResult.java   |   40 -
 .../odbc/OdbcQueryExecuteRequest.java           |   78 -
 .../processors/odbc/OdbcQueryExecuteResult.java |   54 -
 .../processors/odbc/OdbcQueryFetchRequest.java  |   61 -
 .../processors/odbc/OdbcQueryFetchResult.java   |   66 -
 .../odbc/OdbcQueryGetColumnsMetaRequest.java    |   74 -
 .../odbc/OdbcQueryGetColumnsMetaResult.java     |   42 -
 .../odbc/OdbcQueryGetParamsMetaRequest.java     |   60 -
 .../odbc/OdbcQueryGetParamsMetaResult.java      |   40 -
 .../odbc/OdbcQueryGetTablesMetaRequest.java     |   85 -
 .../odbc/OdbcQueryGetTablesMetaResult.java      |   42 -
 .../internal/processors/odbc/OdbcRequest.java   |   61 -
 .../processors/odbc/OdbcRequestHandler.java     |  530 --
 .../internal/processors/odbc/OdbcResponse.java  |   96 -
 .../internal/processors/odbc/OdbcTableMeta.java |   88 -
 .../internal/processors/odbc/OdbcTypes.java     |  131 -
 .../internal/processors/odbc/OdbcUtils.java     |  141 -
 .../odbc/SqlListenerBufferedParser.java         |   81 +
 .../odbc/SqlListenerConnectionContext.java      |   56 +
 .../processors/odbc/SqlListenerDataTypes.java   |   69 +
 .../odbc/SqlListenerMessageParser.java          |   39 +
 .../processors/odbc/SqlListenerNioListener.java |  269 +
 .../odbc/SqlListenerNioServerBuffer.java        |  114 +
 .../processors/odbc/SqlListenerProcessor.java   |  277 +
 .../odbc/SqlListenerProtocolVersion.java        |  116 +
 .../processors/odbc/SqlListenerRequest.java     |   43 +
 .../odbc/SqlListenerRequestHandler.java         |   39 +
 .../processors/odbc/SqlListenerResponse.java    |   76 +
 .../processors/odbc/SqlListenerUtils.java       |  250 +
 .../odbc/escape/OdbcEscapeParseResult.java      |   73 -
 .../processors/odbc/escape/OdbcEscapeToken.java |   61 -
 .../processors/odbc/escape/OdbcEscapeType.java  |  112 -
 .../processors/odbc/escape/OdbcEscapeUtils.java |  440 --
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |  129 +
 .../processors/odbc/jdbc/JdbcMessageParser.java |   87 +
 .../odbc/jdbc/JdbcQueryCloseRequest.java        |   72 +
 .../processors/odbc/jdbc/JdbcQueryCursor.java   |  135 +
 .../odbc/jdbc/JdbcQueryExecuteRequest.java      |  147 +
 .../odbc/jdbc/JdbcQueryExecuteResult.java       |  150 +
 .../odbc/jdbc/JdbcQueryFetchRequest.java        |   87 +
 .../odbc/jdbc/JdbcQueryFetchResult.java         |   84 +
 .../odbc/jdbc/JdbcQueryMetadataRequest.java     |   73 +
 .../odbc/jdbc/JdbcQueryMetadataResult.java      |   93 +
 .../odbc/jdbc/JdbcRawBinarylizable.java         |   44 +
 .../processors/odbc/jdbc/JdbcRequest.java       |  108 +
 .../odbc/jdbc/JdbcRequestHandler.java           |  310 ++
 .../processors/odbc/jdbc/JdbcResponse.java      |  104 +
 .../processors/odbc/jdbc/JdbcResult.java        |   91 +
 .../processors/odbc/jdbc/JdbcUtils.java         |   73 +
 .../processors/odbc/odbc/OdbcColumnMeta.java    |  110 +
 .../processors/odbc/odbc/OdbcMessageParser.java |  308 ++
 .../odbc/odbc/OdbcQueryCloseRequest.java        |   49 +
 .../odbc/odbc/OdbcQueryCloseResult.java         |   40 +
 .../odbc/odbc/OdbcQueryExecuteBatchRequest.java |   95 +
 .../odbc/odbc/OdbcQueryExecuteBatchResult.java  |   75 +
 .../odbc/odbc/OdbcQueryExecuteRequest.java      |   80 +
 .../odbc/odbc/OdbcQueryExecuteResult.java       |   54 +
 .../odbc/odbc/OdbcQueryFetchRequest.java        |   61 +
 .../odbc/odbc/OdbcQueryFetchResult.java         |   66 +
 .../odbc/OdbcQueryGetColumnsMetaRequest.java    |   74 +
 .../odbc/OdbcQueryGetColumnsMetaResult.java     |   42 +
 .../odbc/OdbcQueryGetParamsMetaRequest.java     |   61 +
 .../odbc/odbc/OdbcQueryGetParamsMetaResult.java |   40 +
 .../odbc/OdbcQueryGetTablesMetaRequest.java     |   85 +
 .../odbc/odbc/OdbcQueryGetTablesMetaResult.java |   42 +
 .../processors/odbc/odbc/OdbcRequest.java       |   63 +
 .../odbc/odbc/OdbcRequestHandler.java           |  634 +++
 .../processors/odbc/odbc/OdbcResponse.java      |   69 +
 .../processors/odbc/odbc/OdbcTableMeta.java     |   88 +
 .../processors/odbc/odbc/OdbcTypes.java         |  131 +
 .../processors/odbc/odbc/OdbcUtils.java         |  142 +
 .../odbc/odbc/escape/OdbcEscapeParseResult.java |   73 +
 .../odbc/odbc/escape/OdbcEscapeToken.java       |   61 +
 .../odbc/odbc/escape/OdbcEscapeType.java        |  112 +
 .../odbc/odbc/escape/OdbcEscapeUtils.java       |  441 ++
 .../platform/PlatformAbstractTarget.java        |   32 +-
 .../platform/PlatformAsyncResult.java           |   41 +
 .../platform/PlatformAsyncTarget.java           |   44 -
 .../platform/PlatformContextImpl.java           |  140 +-
 .../processors/platform/PlatformIgnition.java   |   34 +-
 .../platform/PlatformProcessorImpl.java         |   11 +-
 .../processors/platform/PlatformTarget.java     |   10 +
 .../platform/PlatformTargetProxy.java           |   23 +-
 .../platform/PlatformTargetProxyImpl.java       |   67 +-
 .../binary/PlatformBinaryProcessor.java         |   65 +-
 .../platform/cache/PlatformCache.java           |  292 +-
 .../cache/PlatformCacheEntryProcessorImpl.java  |    6 +-
 .../callback/PlatformCallbackGateway.java       |   65 +-
 .../platform/callback/PlatformCallbackOp.java   |    3 +
 .../platform/cluster/PlatformClusterGroup.java  |   86 +
 .../platform/compute/PlatformAbstractJob.java   |    2 +-
 .../platform/compute/PlatformCompute.java       |   24 +-
 .../dotnet/PlatformDotNetCacheStore.java        |   12 +-
 .../PlatformDotNetConfigurationClosure.java     |   72 +-
 .../dotnet/PlatformDotNetConfigurationEx.java   |   21 +-
 ...formDotNetEntityFrameworkCacheExtension.java |    8 +-
 .../platform/events/PlatformEvents.java         |   70 +-
 .../memory/PlatformOutputStreamImpl.java        |    8 +-
 .../platform/messaging/PlatformMessaging.java   |   35 +-
 .../cache/PlatformCachePluginConfiguration.java |    7 -
 .../cache/PlatformCachePluginProvider.java      |  123 -
 .../platform/services/PlatformServices.java     |   95 +-
 .../transactions/PlatformTransactions.java      |    9 +-
 .../utils/PlatformConfigurationUtils.java       |  686 ++-
 .../processors/plugin/CachePluginManager.java   |   20 +-
 .../plugin/IgnitePluginProcessor.java           |    3 +-
 .../internal/processors/pool/PoolProcessor.java |   40 +-
 .../processors/port/GridPortProcessor.java      |    2 +-
 .../query/CacheQueryObjectValueContext.java     |   64 +
 .../query/GridQueryCacheObjectsIterator.java    |   16 +-
 .../query/GridQueryIndexDescriptor.java         |   16 +-
 .../processors/query/GridQueryIndexType.java    |   32 -
 .../processors/query/GridQueryIndexing.java     |  198 +-
 .../processors/query/GridQueryProcessor.java    | 4054 +++++++--------
 .../query/GridQueryTypeDescriptor.java          |   46 +
 .../processors/query/GridRunningQueryInfo.java  |  132 +
 .../processors/query/IgniteSQLException.java    |    7 +
 .../query/QueryIndexDescriptorImpl.java         |  147 +
 .../processors/query/QueryIndexKey.java         |   71 +
 .../internal/processors/query/QuerySchema.java  |  166 +
 .../processors/query/QueryTypeCandidate.java    |   66 +
 .../query/QueryTypeDescriptorImpl.java          |  459 ++
 .../processors/query/QueryTypeIdKey.java        |   94 +
 .../processors/query/QueryTypeNameKey.java      |   68 +
 .../internal/processors/query/QueryUtils.java   | 1218 +++++
 .../messages/GridQueryCancelRequest.java        |    2 +-
 .../twostep/messages/GridQueryFailResponse.java |   12 +-
 .../messages/GridQueryNextPageRequest.java      |   31 +-
 .../messages/GridQueryNextPageResponse.java     |   31 +-
 .../h2/twostep/messages/GridQueryRequest.java   |  368 --
 .../query/property/QueryBinaryProperty.java     |  269 +
 .../query/property/QueryClassProperty.java      |  130 +
 .../query/property/QueryFieldAccessor.java      |   67 +
 .../query/property/QueryMethodsAccessor.java    |   82 +
 .../query/property/QueryPropertyAccessor.java   |   53 +
 .../property/QueryReadOnlyMethodsAccessor.java  |   71 +
 .../query/schema/SchemaExchangeWorkerTask.java  |   53 +
 .../query/schema/SchemaIndexCacheVisitor.java   |   33 +
 .../schema/SchemaIndexCacheVisitorClosure.java  |   42 +
 .../schema/SchemaIndexCacheVisitorImpl.java     |  197 +
 .../SchemaIndexOperationCancellationToken.java  |   53 +
 .../SchemaNodeLeaveExchangeWorkerTask.java      |   53 +
 .../schema/SchemaOperationClientFuture.java     |   52 +
 .../query/schema/SchemaOperationException.java  |  138 +
 .../query/schema/SchemaOperationManager.java    |  292 ++
 .../query/schema/SchemaOperationWorker.java     |  205 +
 .../message/SchemaAbstractDiscoveryMessage.java |   70 +
 .../message/SchemaFinishDiscoveryMessage.java   |   98 +
 .../message/SchemaOperationStatusMessage.java   |  168 +
 .../message/SchemaProposeDiscoveryMessage.java  |  132 +
 .../operation/SchemaAbstractOperation.java      |   78 +
 .../operation/SchemaIndexAbstractOperation.java |   41 +
 .../operation/SchemaIndexCreateOperation.java   |   93 +
 .../operation/SchemaIndexDropOperation.java     |   69 +
 .../resource/GridResourceProcessor.java         |   12 +-
 .../processors/rest/GridRestCommand.java        |   11 +-
 .../processors/rest/GridRestProcessor.java      |   30 +-
 .../rest/client/message/GridClientNodeBean.java |   27 +-
 .../client/message/GridClientStateRequest.java  |   80 +
 .../handlers/GridRestCommandHandlerAdapter.java |    3 +
 .../handlers/cache/GridCacheCommandHandler.java |   47 +-
 .../cluster/GridChangeStateCommandHandler.java  |   90 +
 .../handlers/query/QueryCommandHandler.java     |   16 +-
 .../handlers/redis/GridRedisCommandHandler.java |    4 +-
 .../GridRedisConnectionCommandHandler.java      |   47 +-
 .../redis/GridRedisRestCommandHandler.java      |   40 +-
 .../redis/key/GridRedisDelCommandHandler.java   |    1 +
 .../key/GridRedisExistsCommandHandler.java      |    1 +
 .../server/GridRedisDbSizeCommandHandler.java   |    1 +
 .../string/GridRedisAppendCommandHandler.java   |    3 +
 .../string/GridRedisGetCommandHandler.java      |    1 +
 .../string/GridRedisGetRangeCommandHandler.java |    1 +
 .../string/GridRedisGetSetCommandHandler.java   |    1 +
 .../string/GridRedisIncrDecrCommandHandler.java |    2 +
 .../string/GridRedisMGetCommandHandler.java     |    1 +
 .../string/GridRedisMSetCommandHandler.java     |    1 +
 .../string/GridRedisSetCommandHandler.java      |   44 +-
 .../string/GridRedisSetRangeCommandHandler.java |    2 +
 .../string/GridRedisStrlenCommandHandler.java   |    1 +
 .../handlers/task/GridTaskCommandHandler.java   |    8 +-
 .../handlers/task/GridTaskResultRequest.java    |    2 +-
 .../handlers/task/GridTaskResultResponse.java   |    2 +-
 .../top/GridTopologyCommandHandler.java         |   19 +-
 .../version/GridVersionCommandHandler.java      |    2 +-
 .../tcp/GridTcpMemcachedNioListener.java        |   25 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |   20 +
 .../rest/protocols/tcp/GridTcpRestProtocol.java |    2 +-
 .../protocols/tcp/redis/GridRedisCommand.java   |    2 +
 .../protocols/tcp/redis/GridRedisMessage.java   |    8 +
 .../tcp/redis/GridRedisNioListener.java         |   26 +-
 .../request/GridRestChangeStateRequest.java     |   57 +
 .../processors/security/SecurityContext.java    |    9 +
 .../processors/security/SecurityUtils.java      |   92 +
 .../service/GridServiceDeploymentFuture.java    |    3 -
 .../service/GridServiceProcessor.java           |  524 +-
 .../processors/service/GridServiceProxy.java    |   14 +-
 .../session/GridTaskSessionProcessor.java       |   12 +-
 .../processors/task/GridTaskProcessor.java      |  100 +-
 .../task/GridTaskThreadContextKey.java          |   10 +-
 .../processors/task/GridTaskWorker.java         |   40 +-
 .../timeout/GridTimeoutProcessor.java           |    6 +-
 .../suggestions/GridPerformanceSuggestions.java |  105 +
 .../JvmConfigurationSuggestions.java            |  104 +
 .../suggestions/OsConfigurationSuggestions.java |  127 +
 .../internal/suggestions/package-info.java      |   21 +
 .../org/apache/ignite/internal/util/F0.java     |  196 +-
 .../apache/ignite/internal/util/GridArrays.java |  116 +
 .../ignite/internal/util/GridBusyLock.java      |    2 +-
 .../ignite/internal/util/GridByteArrayList.java |    2 +-
 .../util/GridCursorIteratorWrapper.java         |   52 +
 .../apache/ignite/internal/util/GridDebug.java  |    2 +-
 .../ignite/internal/util/GridIntIterator.java   |   33 +
 .../ignite/internal/util/GridIntList.java       |  603 +++
 .../ignite/internal/util/GridLongList.java      |    2 +-
 .../internal/util/GridMessageCollection.java    |    2 +-
 .../util/GridMultiCollectionWrapper.java        |  186 +
 .../internal/util/GridPartitionStateMap.java    |  222 +
 .../ignite/internal/util/GridStringBuilder.java |   19 +
 .../apache/ignite/internal/util/GridUnsafe.java |   67 +-
 .../internal/util/IgniteExceptionRegistry.java  |    3 +-
 .../apache/ignite/internal/util/IgniteTree.java |  134 +
 .../ignite/internal/util/IgniteUtils.java       |  861 ++--
 .../internal/util/OffheapReadWriteLock.java     |  665 +++
 .../util/StripedCompositeReadWriteLock.java     |   50 +-
 .../ignite/internal/util/StripedExecutor.java   |  101 +-
 .../internal/util/UUIDCollectionMessage.java    |    2 +-
 .../util/future/AsyncFutureListener.java        |   57 +
 .../internal/util/future/CountDownFuture.java   |   81 +
 .../util/future/GridCompoundFuture.java         |   56 +-
 .../util/future/GridEmbeddedFuture.java         |    3 -
 .../util/future/GridFinishedFuture.java         |   13 -
 .../internal/util/future/GridFutureAdapter.java |  479 +-
 .../internal/util/future/IgniteFutureImpl.java  |   43 +-
 .../util/future/IgniteRemoteMapTask.java        |  142 +
 .../internal/util/gridify/GridifyUtils.java     |    6 +-
 .../internal/util/ipc/IpcToNioAdapter.java      |    7 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   22 +-
 .../ignite/internal/util/lang/GridCursor.java   |   41 +
 .../ignite/internal/util/lang/GridFunc.java     | 1826 +------
 .../internal/util/lang/GridTreePrinter.java     |   81 +
 .../ignite/internal/util/lang/GridTupleV.java   |  195 -
 .../lang/gridfunc/AlwaysFalsePredicate.java     |   46 +
 .../util/lang/gridfunc/AlwaysTruePredicate.java |   46 +
 .../util/lang/gridfunc/AlwaysTrueReducer.java   |   56 +
 .../gridfunc/AtomicIntegerFactoryCallable.java  |   40 +
 .../gridfunc/CacheEntryGetValueClosure.java     |   42 +
 .../gridfunc/CacheEntryHasPeekPredicate.java    |   41 +
 .../lang/gridfunc/ClusterNodeGetIdClosure.java  |   41 +
 .../ConcurrentDequeFactoryCallable.java         |   40 +
 .../ConcurrentHashSetFactoryCallable.java       |   40 +
 .../gridfunc/ConcurrentMapFactoryCallable.java  |   41 +
 .../lang/gridfunc/ContainsNodeIdsPredicate.java |   52 +
 .../util/lang/gridfunc/ContainsPredicate.java   |   55 +
 .../gridfunc/EntryByKeyEvaluationPredicate.java |   53 +
 .../gridfunc/EqualsClusterNodeIdPredicate.java  |   51 +
 .../util/lang/gridfunc/EqualsUuidPredicate.java |   50 +
 .../lang/gridfunc/FlatCollectionWrapper.java    |   66 +
 .../util/lang/gridfunc/FlatIterator.java        |  104 +
 .../util/lang/gridfunc/HasEqualIdPredicate.java |   51 +
 .../lang/gridfunc/HasNotEqualIdPredicate.java   |   51 +
 .../util/lang/gridfunc/IdentityClosure.java     |   39 +
 .../util/lang/gridfunc/IntSumReducer.java       |   51 +
 .../util/lang/gridfunc/IsAllPredicate.java      |   52 +
 .../util/lang/gridfunc/IsNotAllPredicate.java   |   52 +
 .../util/lang/gridfunc/IsNotNullPredicate.java  |   44 +
 .../util/lang/gridfunc/LongSumReducer.java      |   51 +
 .../util/lang/gridfunc/MapFactoryCallable.java  |   41 +
 .../util/lang/gridfunc/MultipleIterator.java    |  106 +
 .../util/lang/gridfunc/NoOpClosure.java         |   39 +
 .../lang/gridfunc/NotContainsPredicate.java     |   54 +
 .../util/lang/gridfunc/NotEqualPredicate.java   |   53 +
 .../lang/gridfunc/PredicateCollectionView.java  |   73 +
 .../util/lang/gridfunc/PredicateMapView.java    |  115 +
 .../util/lang/gridfunc/PredicateSetView.java    |  147 +
 .../lang/gridfunc/ReadOnlyCollectionView.java   |   89 +
 .../lang/gridfunc/ReadOnlyCollectionView2X.java |   94 +
 .../lang/gridfunc/RunnableWrapperClosure.java   |   51 +
 .../util/lang/gridfunc/SetFactoryCallable.java  |   41 +
 .../util/lang/gridfunc/StringConcatReducer.java |   79 +
 .../util/lang/gridfunc/ToStringClosure.java     |   42 +
 .../lang/gridfunc/TransformCollectionView.java  |   74 +
 .../gridfunc/TransformFilteringIterator.java    |  138 +
 .../util/lang/gridfunc/TransformMapView.java    |  162 +
 .../util/lang/gridfunc/TransformMapView2.java   |  165 +
 .../util/lang/gridfunc/package-info.java        |   22 +
 .../nio/GridAbstractCommunicationClient.java    |   37 +-
 .../util/nio/GridCommunicationClient.java       |    5 -
 .../nio/GridConnectionBytesVerifyFilter.java    |    7 +-
 .../internal/util/nio/GridDirectParser.java     |   12 +-
 .../util/nio/GridNioAsyncNotifyFilter.java      |   23 +-
 .../util/nio/GridNioBackPressureControl.java    |   39 +-
 .../internal/util/nio/GridNioCodecFilter.java   |    9 +-
 .../util/nio/GridNioEmbeddedFuture.java         |    8 +-
 .../ignite/internal/util/nio/GridNioFilter.java |   12 +-
 .../internal/util/nio/GridNioFilterAdapter.java |    7 +-
 .../internal/util/nio/GridNioFilterChain.java   |   15 +-
 .../util/nio/GridNioFinishedFuture.java         |    5 -
 .../ignite/internal/util/nio/GridNioFuture.java |    7 -
 .../internal/util/nio/GridNioFutureImpl.java    |   19 +-
 .../util/nio/GridNioMessageTracker.java         |    7 +
 .../util/nio/GridNioRecoveryDescriptor.java     |    1 +
 .../ignite/internal/util/nio/GridNioServer.java |  415 +-
 .../internal/util/nio/GridNioSession.java       |    7 +-
 .../internal/util/nio/GridNioSessionImpl.java   |    9 +-
 .../util/nio/GridNioSessionMetaKey.java         |    5 +-
 .../util/nio/GridShmemCommunicationClient.java  |    6 +-
 .../util/nio/GridTcpNioCommunicationClient.java |   13 +-
 .../internal/util/nio/SessionWriteRequest.java  |    7 -
 .../internal/util/nio/ssl/GridNioSslFilter.java |   12 +-
 .../util/nio/ssl/GridNioSslHandler.java         |   29 +-
 .../offheap/unsafe/GridOffHeapSnapTreeMap.java  |   14 +-
 .../internal/util/snaptree/SnapTreeMap.java     |   10 +-
 .../util/tostring/GridToStringBuilder.java      |  421 +-
 .../util/tostring/GridToStringThreadLocal.java  |    6 +-
 .../ignite/internal/util/worker/GridWorker.java |   24 +-
 .../internal/util/worker/GridWorkerFuture.java  |    3 -
 .../internal/visor/VisorDataTransferObject.java |  118 +
 .../visor/VisorDataTransferObjectInput.java     |  156 +
 .../visor/VisorDataTransferObjectOutput.java    |  141 +
 .../ignite/internal/visor/VisorEither.java      |  103 +
 .../internal/visor/VisorMultiNodeTask.java      |   15 +-
 .../ignite/internal/visor/VisorOneNodeTask.java |    5 +-
 .../internal/visor/VisorTaskArgument.java       |   53 +-
 .../visor/binary/VisorBinaryMetadata.java       |  138 +
 .../VisorBinaryMetadataCollectorTask.java       |   72 +
 .../VisorBinaryMetadataCollectorTaskArg.java    |   71 +
 .../VisorBinaryMetadataCollectorTaskResult.java |   87 +
 .../visor/binary/VisorBinaryMetadataField.java  |  101 +
 .../ignite/internal/visor/cache/VisorCache.java |  326 +-
 .../cache/VisorCacheAffinityConfiguration.java  |   93 +-
 .../visor/cache/VisorCacheAffinityNodeTask.java |   13 +-
 .../cache/VisorCacheAffinityNodeTaskArg.java    |   86 +
 .../cache/VisorCacheAggregatedMetrics.java      |  207 +-
 .../visor/cache/VisorCacheClearTask.java        |  128 +-
 .../visor/cache/VisorCacheClearTaskArg.java     |   72 +
 .../visor/cache/VisorCacheClearTaskResult.java  |   85 +
 .../visor/cache/VisorCacheConfiguration.java    |  493 +-
 .../VisorCacheConfigurationCollectorJob.java    |   27 +-
 .../VisorCacheConfigurationCollectorTask.java   |    7 +-
 ...VisorCacheConfigurationCollectorTaskArg.java |   73 +
 .../cache/VisorCacheDefaultConfiguration.java   |   58 -
 .../cache/VisorCacheEvictionConfiguration.java  |   97 +-
 .../visor/cache/VisorCacheJdbcType.java         |  189 +
 .../visor/cache/VisorCacheJdbcTypeField.java    |  117 +
 .../visor/cache/VisorCacheLoadTask.java         |   23 +-
 .../visor/cache/VisorCacheLoadTaskArg.java      |  101 +
 .../visor/cache/VisorCacheMetadataTask.java     |   26 +-
 .../visor/cache/VisorCacheMetadataTaskArg.java  |   72 +
 .../internal/visor/cache/VisorCacheMetrics.java |  349 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |   45 +-
 .../VisorCacheMetricsCollectorTaskArg.java      |   87 +
 .../visor/cache/VisorCacheMetricsV2.java        |   66 -
 .../cache/VisorCacheNearConfiguration.java      |   59 +-
 .../visor/cache/VisorCacheNodesTask.java        |   12 +-
 .../visor/cache/VisorCacheNodesTaskArg.java     |   72 +
 .../visor/cache/VisorCachePartition.java        |   90 -
 .../visor/cache/VisorCachePartitions.java       |   57 +-
 .../visor/cache/VisorCachePartitionsTask.java   |   42 +-
 .../cache/VisorCachePartitionsTaskArg.java      |   72 +
 .../cache/VisorCacheQueryConfiguration.java     |  112 -
 .../cache/VisorCacheQueryConfigurationV2.java   |   47 -
 .../cache/VisorCacheQueryDetailMetrics.java     |  167 -
 ...sorCacheQueryDetailMetricsCollectorTask.java |  146 -
 .../visor/cache/VisorCacheQueryMetrics.java     |  102 -
 .../cache/VisorCacheRebalanceConfiguration.java |   97 +-
 .../visor/cache/VisorCacheRebalanceTask.java    |   13 +-
 .../visor/cache/VisorCacheRebalanceTaskArg.java |   73 +
 .../visor/cache/VisorCacheResetMetricsTask.java |   14 +-
 .../cache/VisorCacheResetMetricsTaskArg.java    |   72 +
 .../VisorCacheResetQueryDetailMetricsTask.java  |   71 -
 .../cache/VisorCacheResetQueryMetricsTask.java  |   69 -
 .../visor/cache/VisorCacheSqlIndexMetadata.java |  115 +
 .../visor/cache/VisorCacheSqlMetadata.java      |  162 +
 .../visor/cache/VisorCacheStartTask.java        |   68 +-
 .../visor/cache/VisorCacheStartTaskArg.java     |  100 +
 .../visor/cache/VisorCacheStopTask.java         |   17 +-
 .../visor/cache/VisorCacheStopTaskArg.java      |   72 +
 .../cache/VisorCacheStoreConfiguration.java     |   99 +-
 .../cache/VisorCacheStoreConfigurationV2.java   |   48 -
 .../visor/cache/VisorCacheSwapBackupsTask.java  |  102 -
 .../cache/VisorCacheTypeFieldMetadata.java      |  102 -
 .../visor/cache/VisorCacheTypeMetadata.java     |  375 --
 .../internal/visor/cache/VisorCacheV2.java      |   73 -
 .../internal/visor/cache/VisorCacheV3.java      |   52 -
 .../internal/visor/cache/VisorCacheV4.java      |  124 -
 .../visor/cache/VisorMemoryMetrics.java         |  177 +
 .../internal/visor/cache/VisorPartitionMap.java |  110 +
 .../compute/VisorComputeCancelSessionsTask.java |   15 +-
 .../VisorComputeCancelSessionsTaskArg.java      |   76 +
 .../compute/VisorComputeMonitoringHolder.java   |    8 +-
 .../VisorComputeToggleMonitoringTask.java       |   52 +-
 .../VisorComputeToggleMonitoringTaskArg.java    |   86 +
 .../visor/compute/VisorGatewayTask.java         |  199 +-
 .../visor/debug/VisorThreadDumpTask.java        |   13 +-
 .../visor/debug/VisorThreadDumpTaskResult.java  |   88 +
 .../internal/visor/debug/VisorThreadInfo.java   |  234 +-
 .../visor/debug/VisorThreadLockInfo.java        |   51 +-
 .../visor/debug/VisorThreadMonitorInfo.java     |   77 +-
 .../visor/event/VisorGridDeploymentEvent.java   |   44 +-
 .../visor/event/VisorGridDiscoveryEvent.java    |   74 +-
 .../visor/event/VisorGridDiscoveryEventV2.java  |   80 -
 .../internal/visor/event/VisorGridEvent.java    |   66 +-
 .../visor/event/VisorGridEventsLost.java        |   15 +-
 .../internal/visor/event/VisorGridJobEvent.java |   62 +-
 .../visor/event/VisorGridTaskEvent.java         |   62 +-
 .../internal/visor/file/VisorFileBlock.java     |   60 +-
 .../internal/visor/file/VisorFileBlockTask.java |   68 +-
 .../visor/file/VisorFileBlockTaskArg.java       |  114 +
 .../visor/file/VisorFileBlockTaskResult.java    |   87 +
 .../visor/file/VisorLatestTextFilesTask.java    |   17 +-
 .../visor/file/VisorLatestTextFilesTaskArg.java |   86 +
 .../ignite/internal/visor/igfs/VisorIgfs.java   |   78 +-
 .../internal/visor/igfs/VisorIgfsEndpoint.java  |   50 +-
 .../visor/igfs/VisorIgfsFormatTask.java         |   16 +-
 .../visor/igfs/VisorIgfsFormatTaskArg.java      |   72 +
 .../internal/visor/igfs/VisorIgfsMetrics.java   |  119 +-
 .../internal/visor/igfs/VisorIgfsProfiler.java  |   24 +-
 .../visor/igfs/VisorIgfsProfilerClearTask.java  |   29 +-
 .../igfs/VisorIgfsProfilerClearTaskArg.java     |   72 +
 .../igfs/VisorIgfsProfilerClearTaskResult.java  |   85 +
 .../visor/igfs/VisorIgfsProfilerEntry.java      |  106 +-
 .../visor/igfs/VisorIgfsProfilerTask.java       |   51 +-
 .../visor/igfs/VisorIgfsProfilerTaskArg.java    |   72 +
 .../VisorIgfsProfilerUniformityCounters.java    |   38 +-
 .../visor/igfs/VisorIgfsResetMetricsTask.java   |   13 +-
 .../igfs/VisorIgfsResetMetricsTaskArg.java      |   73 +
 .../visor/igfs/VisorIgfsSamplingStateTask.java  |   17 +-
 .../igfs/VisorIgfsSamplingStateTaskArg.java     |   86 +
 .../ignite/internal/visor/log/VisorLogFile.java |   43 +-
 .../visor/log/VisorLogSearchResult.java         |   80 +-
 .../internal/visor/log/VisorLogSearchTask.java  |   81 +-
 .../visor/log/VisorLogSearchTaskArg.java        |  114 +
 .../visor/log/VisorLogSearchTaskResult.java     |   92 +
 .../internal/visor/misc/VisorAckTask.java       |   14 +-
 .../internal/visor/misc/VisorAckTaskArg.java    |   72 +
 .../misc/VisorChangeGridActiveStateTask.java    |   65 +
 .../misc/VisorChangeGridActiveStateTaskArg.java |   71 +
 .../node/VisorAffinityTopologyVersion.java      |   87 +
 .../visor/node/VisorAtomicConfiguration.java    |   49 +-
 .../visor/node/VisorBasicConfiguration.java     |  387 +-
 .../visor/node/VisorBinaryConfiguration.java    |  131 +
 .../node/VisorBinaryTypeConfiguration.java      |  150 +
 .../visor/node/VisorCacheKeyConfiguration.java  |  108 +
 .../visor/node/VisorExecutorConfiguration.java  |  108 +
 .../node/VisorExecutorServiceConfiguration.java |  194 +-
 .../visor/node/VisorGridConfiguration.java      |  258 +-
 .../visor/node/VisorHadoopConfiguration.java    |  145 +
 .../visor/node/VisorIgfsConfiguration.java      |  293 +-
 .../visor/node/VisorLifecycleConfiguration.java |   39 +-
 .../visor/node/VisorMemoryConfiguration.java    |  167 +
 .../node/VisorMemoryPolicyConfiguration.java    |  166 +
 .../visor/node/VisorMetricsConfiguration.java   |   50 +-
 .../VisorNodeConfigurationCollectorJob.java     |    4 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |  120 +-
 .../node/VisorNodeDataCollectorJobResult.java   |  173 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |   50 +-
 .../node/VisorNodeDataCollectorTaskArg.java     |   68 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |  176 +-
 .../node/VisorNodeEventsCollectorTask.java      |  133 +-
 .../node/VisorNodeEventsCollectorTaskArg.java   |  163 +
 .../internal/visor/node/VisorNodeGcTask.java    |   18 +-
 .../visor/node/VisorNodeGcTaskResult.java       |   85 +
 .../internal/visor/node/VisorNodePingTask.java  |   20 +-
 .../visor/node/VisorNodePingTaskArg.java        |   73 +
 .../visor/node/VisorNodePingTaskResult.java     |   99 +
 .../visor/node/VisorNodeSuppressedErrors.java   |   89 +
 .../node/VisorNodeSuppressedErrorsTask.java     |   31 +-
 .../node/VisorNodeSuppressedErrorsTaskArg.java  |   74 +
 .../node/VisorPeerToPeerConfiguration.java      |   57 +-
 .../node/VisorPersistentStoreConfiguration.java |  307 ++
 .../visor/node/VisorRestConfiguration.java      |  282 +-
 .../node/VisorSegmentationConfiguration.java    |   80 +-
 .../visor/node/VisorServiceConfiguration.java   |  176 +
 .../visor/node/VisorSpiDescription.java         |   89 +
 .../visor/node/VisorSpisConfiguration.java      |  139 +-
 .../node/VisorSqlConnectorConfiguration.java    |  166 +
 .../visor/node/VisorSuppressedError.java        |  147 +
 .../node/VisorTransactionConfiguration.java     |  104 +-
 .../internal/visor/query/VisorQueryArg.java     |   81 -
 .../internal/visor/query/VisorQueryArgV2.java   |   49 -
 .../visor/query/VisorQueryCancelTask.java       |   72 +
 .../visor/query/VisorQueryCancelTaskArg.java    |   71 +
 .../visor/query/VisorQueryCleanupTask.java      |   16 +-
 .../visor/query/VisorQueryCleanupTaskArg.java   |   75 +
 .../visor/query/VisorQueryConfiguration.java    |  130 +
 .../visor/query/VisorQueryDetailMetrics.java    |  205 +
 .../VisorQueryDetailMetricsCollectorTask.java   |  149 +
 ...VisorQueryDetailMetricsCollectorTaskArg.java |   71 +
 .../internal/visor/query/VisorQueryEntity.java  |  188 +
 .../internal/visor/query/VisorQueryField.java   |   42 +-
 .../internal/visor/query/VisorQueryIndex.java   |  105 +
 .../visor/query/VisorQueryIndexField.java       |  106 +
 .../internal/visor/query/VisorQueryJob.java     |  274 -
 .../internal/visor/query/VisorQueryMetrics.java |  125 +
 .../visor/query/VisorQueryNextPageTask.java     |   33 +-
 .../visor/query/VisorQueryNextPageTaskArg.java  |   86 +
 .../query/VisorQueryResetDetailMetricsTask.java |   73 +
 .../visor/query/VisorQueryResetMetricsTask.java |   73 +
 .../query/VisorQueryResetMetricsTaskArg.java    |   72 +
 .../internal/visor/query/VisorQueryResult.java  |   97 +-
 .../visor/query/VisorQueryResultEx.java         |   89 -
 .../visor/query/VisorQueryScanRegexFilter.java  |   59 +
 .../query/VisorQueryScanSubstringFilter.java    |   64 -
 .../internal/visor/query/VisorQueryTask.java    |  103 +-
 .../internal/visor/query/VisorQueryTaskArg.java |  155 +
 .../internal/visor/query/VisorQueryUtils.java   |   43 +-
 .../query/VisorRunningQueriesCollectorTask.java |  100 +
 .../VisorRunningQueriesCollectorTaskArg.java    |   71 +
 .../internal/visor/query/VisorRunningQuery.java |  175 +
 .../visor/query/VisorScanQueryTask.java         |  185 +
 .../visor/query/VisorScanQueryTaskArg.java      |  157 +
 .../visor/service/VisorCancelServiceTask.java   |   12 +-
 .../service/VisorCancelServiceTaskArg.java      |   72 +
 .../visor/service/VisorServiceDescriptor.java   |   40 +-
 .../internal/visor/util/VisorEventMapper.java   |    4 +-
 .../visor/util/VisorExceptionWrapper.java       |   11 +-
 .../internal/visor/util/VisorMimeTypes.java     |   12 +-
 .../internal/visor/util/VisorTaskUtils.java     |   87 +-
 .../apache/ignite/lang/IgniteAsyncSupport.java  |   52 +-
 .../ignite/lang/IgniteAsyncSupported.java       |    4 +-
 .../org/apache/ignite/lang/IgniteFuture.java    |   29 +-
 .../java/org/apache/ignite/lang/IgniteUuid.java |   25 +-
 .../apache/ignite/marshaller/Marshaller.java    |    2 -
 .../ignite/marshaller/MarshallerContext.java    |   27 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |    4 +-
 .../optimized/OptimizedClassDescriptor.java     | 1141 -----
 .../optimized/OptimizedFieldType.java           |   50 -
 .../optimized/OptimizedMarshaller.java          |  304 --
 .../optimized/OptimizedMarshallerIdMapper.java  |   33 -
 .../optimized/OptimizedMarshallerUtils.java     |  551 --
 .../optimized/OptimizedObjectInputStream.java   | 1231 -----
 .../optimized/OptimizedObjectOutputStream.java  |  875 ----
 .../OptimizedObjectStreamRegistry.java          |  244 -
 .../marshaller/optimized/package-info.java      |   22 -
 .../ignite/mxbean/CacheMetricsMXBean.java       |   48 +-
 .../apache/ignite/mxbean/IgniteMBeanAware.java  |   28 +
 .../org/apache/ignite/mxbean/IgniteMXBean.java  |   73 +-
 .../apache/ignite/mxbean/IgnitionMXBean.java    |   14 +-
 .../ignite/mxbean/MemoryMetricsMXBean.java      |  139 +
 .../ignite/mxbean/PersistenceMetricsMXBean.java |  121 +
 .../ignite/mxbean/StripedExecutorMXBean.java    |   90 +
 .../dotnet/PlatformDotNetAffinityFunction.java  |   10 +-
 .../PlatformDotNetBinaryConfiguration.java      |   31 +-
 .../PlatformDotNetBinaryTypeConfiguration.java  |   35 +-
 .../dotnet/PlatformDotNetConfiguration.java     |   10 +-
 .../ignite/plugin/CachePluginConfiguration.java |    7 -
 .../ignite/plugin/CachePluginContext.java       |    5 -
 .../ignite/plugin/CachePluginProvider.java      |    4 +-
 .../apache/ignite/plugin/PluginProvider.java    |   11 +-
 .../extensions/communication/Message.java       |    5 +-
 .../communication/MessageFactory.java           |    2 +-
 .../extensions/communication/MessageWriter.java |    5 +-
 ...PlatformCachePluginConfigurationClosure.java |   31 +
 ...mCachePluginConfigurationClosureFactory.java |   37 +
 .../PlatformPluginConfigurationClosure.java     |   30 +
 ...atformPluginConfigurationClosureFactory.java |   37 +
 .../ignite/plugin/platform/package-info.java    |   22 +
 .../security/SecurityBasicPermissionSet.java    |  135 +-
 .../plugin/security/SecurityPermission.java     |   13 +-
 .../plugin/security/SecurityPermissionSet.java  |   11 +-
 .../security/SecurityPermissionSetBuilder.java  |   36 +-
 .../ignite/plugin/security/SecuritySubject.java |    3 +-
 .../plugin/segmentation/SegmentationPolicy.java |   18 +-
 .../ignite/services/ServiceConfiguration.java   |   35 +-
 .../java/org/apache/ignite/spi/IgniteSpi.java   |    6 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  115 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |    6 +
 .../ignite/spi/IgniteSpiMBeanAdapter.java       |   78 +
 .../spi/IgniteSpiOperationTimeoutHelper.java    |    8 +-
 .../org/apache/ignite/spi/IgniteSpiThread.java  |    6 +-
 .../ignite/spi/IgniteSpiThreadFactory.java      |   10 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |   42 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |  221 +-
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |    9 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   64 +-
 .../sharedfs/SharedFsTimeoutTask.java           |    6 +-
 .../ignite/spi/collision/CollisionSpi.java      |   11 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |  136 +-
 .../fifoqueue/FifoQueueCollisionSpiMBean.java   |    2 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  287 +-
 .../JobStealingCollisionSpiMBean.java           |    6 +-
 .../jobstealing/JobStealingRequest.java         |    2 +-
 .../spi/collision/noop/NoopCollisionSpi.java    |    9 +-
 .../PriorityQueueCollisionSpi.java              |  280 +-
 .../PriorityQueueCollisionSpiMBean.java         |    2 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 1056 ++--
 .../tcp/TcpCommunicationSpiMBean.java           |   54 -
 .../deployment/local/LocalDeploymentSpi.java    |   43 +-
 .../ignite/spi/discovery/DiscoveryDataBag.java  |   18 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../spi/discovery/DiscoverySpiListener.java     |    9 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  417 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  527 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   27 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  642 ++-
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |   48 +-
 .../tcp/internal/TcpDiscoveryNode.java          |   52 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    3 +-
 .../ipfinder/TcpDiscoveryIpFinderAdapter.java   |    5 +-
 .../ipfinder/jdbc/TcpDiscoveryJdbcIpFinder.java |   17 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   47 +-
 .../sharedfs/TcpDiscoverySharedFsIpFinder.java  |   12 +-
 .../tcp/ipfinder/vm/TcpDiscoveryVmIpFinder.java |   14 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   29 +-
 .../messages/TcpDiscoveryClientAckResponse.java |    4 -
 .../TcpDiscoveryClientHeartbeatMessage.java     |   72 -
 .../TcpDiscoveryClientMetricsUpdateMessage.java |   72 +
 .../messages/TcpDiscoveryHeartbeatMessage.java  |  338 --
 .../TcpDiscoveryMetricsUpdateMessage.java       |  338 ++
 .../TcpDiscoveryRingLatencyCheckMessage.java    |   77 +
 .../spi/eventstorage/NoopEventStorageSpi.java   |   53 +
 .../memory/MemoryEventStorageSpi.java           |  100 +-
 .../ignite/spi/failover/FailoverContext.java    |   10 -
 .../spi/failover/always/AlwaysFailoverSpi.java  |   55 +-
 .../jobstealing/JobStealingFailoverSpi.java     |   70 +-
 .../spi/failover/never/NeverFailoverSpi.java    |   32 +-
 .../spi/indexing/IndexingQueryFilter.java       |    8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   36 +-
 .../spi/indexing/noop/NoopIndexingSpi.java      |   23 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |   57 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |   46 +-
 .../WeightedRandomLoadBalancingSpi.java         |   63 +-
 .../ignite/spi/swapspace/SwapContext.java       |   47 -
 .../apache/ignite/spi/swapspace/SwapKey.java    |  128 -
 .../ignite/spi/swapspace/SwapSpaceSpi.java      |  225 -
 .../spi/swapspace/SwapSpaceSpiListener.java     |   35 -
 .../spi/swapspace/file/FileSwapArray.java       |  188 -
 .../spi/swapspace/file/FileSwapSpaceSpi.java    | 1947 -------
 .../swapspace/file/FileSwapSpaceSpiMBean.java   |   67 -
 .../ignite/spi/swapspace/file/package-info.java |   22 -
 .../spi/swapspace/noop/NoopSwapSpaceSpi.java    |  152 -
 .../ignite/spi/swapspace/noop/package-info.java |   22 -
 .../ignite/spi/swapspace/package-info.java      |   22 -
 .../ignite/startup/BasicWarmupClosure.java      |    6 +-
 .../startup/cmdline/CommandLineStartup.java     |    6 +-
 .../org/apache/ignite/stream/StreamVisitor.java |    2 +-
 .../ignite/stream/socket/SocketStreamer.java    |    6 +-
 .../thread/IgniteStripedThreadPoolExecutor.java |    6 +-
 .../org/apache/ignite/thread/IgniteThread.java  |   92 +-
 .../ignite/thread/IgniteThreadFactory.java      |   18 +-
 .../ignite/thread/IgniteThreadPoolExecutor.java |   12 +-
 .../apache/ignite/transactions/Transaction.java |   22 +
 .../TransactionDeadlockException.java           |    4 +-
 .../transactions/TransactionException.java      |   80 +
 .../TransactionHeuristicException.java          |    4 +-
 .../TransactionOptimisticException.java         |    4 +-
 .../TransactionRollbackException.java           |    4 +-
 .../TransactionTimeoutException.java            |    4 +-
 .../org/jsr166/ConcurrentLinkedHashMap.java     |    2 +-
 .../resources/META-INF/classnames.properties    |  676 +--
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../spring-cache-client-benchmark-1.xml         |    3 -
 .../spring-cache-client-benchmark-2.xml         |    3 -
 .../spring-cache-client-benchmark-3.xml         |    3 -
 .../test/config/default-spring-url-testing.xml  |    2 +-
 .../core/src/test/config/discovery-stress.xml   |    2 +-
 modules/core/src/test/config/example-cache.xml  |    8 +-
 .../core/src/test/config/examples.properties    |    1 +
 modules/core/src/test/config/igfs-loopback.xml  |   45 -
 modules/core/src/test/config/igfs-shmem.xml     |   45 -
 .../src/test/config/load/cache-benchmark.xml    |    4 -
 .../test/config/load/cache-client-benchmark.xml |    2 -
 .../config/load/dsi-49-server-production.xml    |    2 -
 .../core/src/test/config/load/dsi-load-base.xml |    5 +-
 .../src/test/config/load/dsi-load-client.xml    |    2 -
 .../src/test/config/load/dsi-load-server.xml    |    2 -
 .../src/test/config/load/merge-sort-base.xml    |    7 +-
 .../test/config/loaders/grid-cfg-2-grids.xml    |   10 +-
 .../core/src/test/config/loaders/grid-cfg.xml   |    2 +-
 modules/core/src/test/config/log4j-test.xml     |    8 +-
 modules/core/src/test/config/log4j2-test.xml    |    6 +-
 .../src/test/config/log4j2-verbose-test.xml     |    4 +-
 .../core/src/test/config/spring-cache-load.xml  |    1 +
 .../core/src/test/config/spring-cache-swap.xml  |    2 +
 .../src/test/config/spring-cache-teststore.xml  |    2 +
 .../core/src/test/config/spring-multicache.xml  |   33 +-
 .../src/test/config/spring-start-nodes-attr.xml |    4 +-
 .../core/src/test/config/spring-start-nodes.xml |    4 +-
 .../test/config/store/jdbc/ignite-jdbc-type.xml |  285 ++
 .../config/store/jdbc/ignite-type-metadata.xml  |  276 -
 .../config/streamer/spring-streamer-base.xml    |    5 +-
 .../config/websession/example-cache-base.xml    |    9 +-
 .../config/websession/example-cache-client.xml  |    6 +-
 .../test/config/websession/example-cache2.xml   |    2 +-
 .../test/config/websession/spring-cache-1.xml   |    8 +-
 .../test/config/websession/spring-cache-2.xml   |    8 +-
 .../test/config/websession/spring-cache-3.xml   |    8 +-
 .../org.apache.ignite.plugin.PluginProvider     |    3 +-
 ...atformCachePluginConfigurationClosureFactory |    1 +
 ...rm.PlatformPluginConfigurationClosureFactory |    1 +
 .../GridCacheAffinityBackupsSelfTest.java       |   26 +-
 .../java/org/apache/ignite/GridTestJob.java     |   19 +
 .../apache/ignite/GridTestStoreNodeStartup.java |    2 +-
 .../java/org/apache/ignite/GridTestTask.java    |   18 +-
 .../ignite/IgniteCacheAffinitySelfTest.java     |   20 +-
 .../IgniteExternalizableAbstractTest.java       |   15 +-
 .../cache/IgniteWarmupClosureSelfTest.java      |    6 +-
 .../ignite/cache/LargeEntryUpdateTest.java      |  176 +
 .../affinity/AffinityClientNodeSelfTest.java    |   35 +-
 ...ityFunctionBackupFilterAbstractSelfTest.java |   25 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    8 +-
 .../affinity/AffinityHistoryCleanupTest.java    |    9 +-
 .../fair/FairAffinityDynamicCacheSelfTest.java  |   86 -
 ...airAffinityFunctionBackupFilterSelfTest.java |   44 -
 ...ffinityFunctionExcludeNeighborsSelfTest.java |   31 -
 .../fair/FairAffinityFunctionNodesSelfTest.java |  247 -
 .../fair/FairAffinityFunctionSelfTest.java      |   31 -
 .../local/LocalAffinityFunctionTest.java        |   16 +-
 ...inityFunctionFastPowerOfTwoHashSelfTest.java |   33 +
 ...ndezvousAffinityFunctionSimpleBenchmark.java | 1054 ++++
 ...ousAffinityFunctionStandardHashSelfTest.java |   33 +
 ...cheStoreSessionListenerAbstractSelfTest.java |   10 +-
 ...heStoreSessionListenerLifecycleSelfTest.java |    4 +-
 .../store/GridCacheBalancingStoreSelfTest.java  |   40 +-
 .../GridCacheLoadOnlyStoreAdapterSelfTest.java  |    4 +-
 .../IgniteCacheExpiryStoreLoadSelfTest.java     |   38 +-
 .../store/StoreResourceInjectionSelfTest.java   |    6 +-
 .../CacheJdbcPojoStoreAbstractSelfTest.java     |   73 +-
 ...BinaryMarshallerStoreKeepBinarySelfTest.java |   28 +
 ...lerStoreKeepBinaryWithSqlEscapeSelfTest.java |   28 +
 ...dbcPojoStoreOptimizedMarshallerSelfTest.java |   31 -
 ...ptimizedMarshallerWithSqlEscapeSelfTest.java |   28 -
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |  136 +-
 ...eJdbcStoreAbstractMultithreadedSelfTest.java |   76 +-
 ...CacheJdbcBlobStoreMultithreadedSelfTest.java |   13 +-
 .../cache/store/jdbc/model/Organization.java    |    2 -
 .../cache/store/jdbc/model/OrganizationKey.java |    2 -
 .../ignite/cache/store/jdbc/model/Person.java   |    2 -
 .../store/jdbc/model/PersonComplexKey.java      |    4 +-
 .../cache/store/jdbc/model/PersonKey.java       |    2 -
 .../ignite/custom/DummyEventFilterFactory.java  |   47 +
 .../ignite/igfs/IgfsEventsAbstractSelfTest.java |   66 +-
 .../igfs/IgfsFragmentizerAbstractSelfTest.java  |   23 +-
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |    5 +-
 .../apache/ignite/igfs/IgfsPathSelfTest.java    |    9 +-
 .../internal/ClusterGroupAbstractTest.java      |   77 +-
 .../internal/ClusterGroupHostsSelfTest.java     |    4 +-
 .../ignite/internal/ClusterGroupSelfTest.java   |  100 +
 .../ignite/internal/ClusterMetricsSelfTest.java |    4 +-
 .../internal/ClusterNodeMetricsSelfTest.java    |   88 +-
 .../ComputeJobCancelWithServiceSelfTest.java    |   14 +-
 .../ignite/internal/GridAffinityMappedTest.java |   22 +-
 .../internal/GridAffinityNoCacheSelfTest.java   |   33 +-
 .../internal/GridAffinityP2PSelfTest.java       |   20 +-
 .../ignite/internal/GridAffinitySelfTest.java   |   20 +-
 .../GridAlwaysFailoverSpiFailSelfTest.java      |    4 +-
 ...chePartitionExchangeManagerHistSizeTest.java |   76 +
 .../internal/GridCancelOnGridStopSelfTest.java  |    2 +-
 .../internal/GridCancelUnusedJobSelfTest.java   |    6 +-
 .../GridCancelledJobsMetricsSelfTest.java       |   19 +-
 .../GridCollisionJobsContextSelfTest.java       |   10 +-
 .../internal/GridCommunicationSelfTest.java     |    2 +-
 ...omputationBinarylizableClosuresSelfTest.java |    4 +-
 .../GridContinuousJobAnnotationSelfTest.java    |    4 +-
 .../internal/GridContinuousTaskSelfTest.java    |  100 +-
 .../ignite/internal/GridDeploymentSelfTest.java |   16 +-
 .../internal/GridDiscoveryEventSelfTest.java    |    4 +-
 .../ignite/internal/GridDiscoverySelfTest.java  |    8 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |   14 +-
 .../GridEventStorageDefaultExceptionTest.java   |   94 +
 ...ventStorageRuntimeConfigurationSelfTest.java |    4 +-
 .../internal/GridFactoryVmShutdownTest.java     |    2 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |   10 +-
 .../GridFailedInputParametersSelfTest.java      |    2 +-
 .../GridFailoverCustomTopologySelfTest.java     |   11 +-
 .../ignite/internal/GridFailoverSelfTest.java   |    4 +-
 .../GridFailoverTaskWithPredicateSelfTest.java  |    6 +-
 .../internal/GridFailoverTopologySelfTest.java  |    4 +-
 .../ignite/internal/GridGetOrStartSelfTest.java |    8 +-
 .../ignite/internal/GridHomePathSelfTest.java   |    8 +-
 .../GridJobCheckpointCleanupSelfTest.java       |    6 +-
 .../GridJobCollisionCancelSelfTest.java         |    6 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |  101 +-
 .../internal/GridJobStealingSelfTest.java       |    4 +-
 .../GridJobStealingZeroActiveJobsSelfTest.java  |    6 +-
 .../internal/GridLifecycleAwareSelfTest.java    |    9 +-
 .../internal/GridLifecycleBeanSelfTest.java     |   16 +-
 .../GridLocalEventListenerSelfTest.java         |    6 +-
 .../internal/GridMultipleJobsSelfTest.java      |   26 +-
 .../internal/GridMultipleSpisSelfTest.java      |    4 +-
 .../GridMultipleVersionsDeploymentSelfTest.java |    4 +-
 .../GridMultithreadedJobStealingSelfTest.java   |    4 +-
 .../ignite/internal/GridNodeLocalSelfTest.java  |    2 +-
 .../internal/GridNodeMetricsLogSelfTest.java    |   17 +-
 .../GridNodeVisorAttributesSelfTest.java        |    4 +-
 .../internal/GridNonHistoryMetricsSelfTest.java |    4 +-
 .../GridProjectionForCachesSelfTest.java        |   51 +-
 ...ectionLocalJobMultipleArgumentsSelfTest.java |    8 +-
 .../ignite/internal/GridReduceSelfTest.java     |    9 +-
 .../internal/GridReleaseTypeSelfTest.java       |    4 +-
 .../internal/GridRuntimeExceptionSelfTest.java  |    8 +-
 .../internal/GridSpiExceptionSelfTest.java      |    8 +-
 .../ignite/internal/GridStartStopSelfTest.java  |   14 +-
 .../internal/GridStopWithWaitSelfTest.java      |   12 +-
 .../GridTaskCancelSingleNodeSelfTest.java       |    7 +-
 .../internal/GridTaskExecutionSelfTest.java     |   58 +-
 ...xecutionWithoutPeerClassLoadingSelfTest.java |   31 +
 .../GridTaskFailoverAffinityRunTest.java        |   15 +-
 .../GridTaskFutureImplStopGridSelfTest.java     |    8 +-
 .../GridTaskInstanceExecutionSelfTest.java      |    9 +-
 .../internal/GridTaskJobRejectSelfTest.java     |   11 +-
 .../internal/GridTaskListenerSelfTest.java      |    2 +-
 .../internal/GridTaskMapAsyncSelfTest.java      |    6 +-
 .../internal/GridTaskResultCacheSelfTest.java   |    4 +-
 .../internal/GridTaskTimeoutSelfTest.java       |    8 +-
 .../IgniteClientReconnectAbstractTest.java      |    4 +-
 .../IgniteClientReconnectApiExceptionTest.java  |   32 +-
 .../IgniteClientReconnectAtomicsTest.java       |   15 +-
 .../IgniteClientReconnectCacheTest.java         |  230 +-
 ...eClientReconnectContinuousProcessorTest.java |   10 +-
 ...niteClientReconnectFailoverAbstractTest.java |    4 +-
 .../IgniteClientReconnectFailoverTest.java      |    8 +-
 .../internal/IgniteClientReconnectStopTest.java |    2 +-
 .../IgniteClientReconnectStreamerTest.java      |    6 +-
 .../ignite/internal/IgniteClientRejoinTest.java |  378 ++
 .../IgniteComputeEmptyClusterGroupTest.java     |   40 +-
 .../IgniteComputeTopologyExceptionTest.java     |    2 +-
 ...eConcurrentEntryProcessorAccessStopTest.java |    2 +-
 .../internal/IgniteExecutorServiceTest.java     |   14 +-
 ...gniteExplicitImplicitDeploymentSelfTest.java |    4 +-
 .../IgniteLocalNodeMapBeforeStartTest.java      |    2 +-
 ...RoundRobinErrorAfterClientReconnectTest.java |    6 +-
 .../IgniteSlowClientDetectionSelfTest.java      |   13 +-
 ...UpdateNotifierPerClusterSettingSelfTest.java |    4 +-
 .../MarshallerContextLockingSelfTest.java       |   38 +-
 .../internal/SensitiveInfoTestLoggerProxy.java  |  493 ++
 .../SensitiveInfoTestLoggerProxy_Exclude.txt    |   20 +
 .../SensitiveInfoTestLoggerProxy_Include.txt    |   18 +
 .../ignite/internal/TaskNodeRestartTest.java    |    4 +-
 .../internal/TestRecordingCommunicationSpi.java |   81 +-
 .../BinaryArrayIdentityResolverSelfTest.java    |    8 +-
 .../BinaryConfigurationConsistencySelfTest.java |    4 +-
 ...ryConfigurationCustomSerializerSelfTest.java |  149 +
 .../internal/binary/BinaryEnumsSelfTest.java    |  154 +-
 .../binary/BinaryFieldExtractionSelfTest.java   |  213 +
 .../BinaryFieldIdentityResolverSelfTest.java    |  333 --
 ...ryIdentityResolverConfigurationSelfTest.java |  138 -
 .../binary/BinaryMarshallerSelfTest.java        |  139 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   14 +-
 ...naryObjectBuilderDefaultMappersSelfTest.java |  130 +-
 ...ilderSimpleNameLowerCaseMappersSelfTest.java |    4 +-
 .../binary/BinaryObjectExceptionSelfTest.java   |  209 +
 .../binary/BinaryObjectToStringSelfTest.java    |    4 +-
 .../BinarySerialiedFieldComparatorSelfTest.java |    4 +-
 .../BinarySimpleNameTestPropertySelfTest.java   |   17 +
 .../internal/binary/BinaryTreeSelfTest.java     |    6 +-
 .../binary/GridBinaryAffinityKeySelfTest.java   |   36 +-
 ...GridBinaryMarshallerCtxDisabledSelfTest.java |    5 +
 ...aultBinaryMappersBinaryMetaDataSelfTest.java |   22 +-
 ...CaseBinaryMappersBinaryMetaDataSelfTest.java |    4 +-
 .../binary/TestCachingMetadataHandler.java      |   10 +
 .../mutabletest/GridBinaryTestClasses.java      |    9 +
 ...ByteBufferStreamImplV2ByteOrderSelfTest.java |    2 +-
 ...GridManagerLocalMessageListenerSelfTest.java |    8 +-
 ...dManagerMxBeanIllegalArgumentHandleTest.java |  125 +
 .../managers/GridManagerStopSelfTest.java       |   23 +-
 .../managers/GridNoopManagerSelfTest.java       |    2 +-
 ...agnosticMessagesMultipleConnectionsTest.java |   35 +
 .../managers/IgniteDiagnosticMessagesTest.java  |  542 ++
 .../GridCheckpointManagerAbstractSelfTest.java  |   33 +-
 .../checkpoint/GridCheckpointTaskSelfTest.java  |    4 +-
 ...idCommunicationManagerListenersSelfTest.java |    8 +-
 .../GridCommunicationSendMessageSelfTest.java   |   92 +-
 .../communication/GridIoManagerSelfTest.java    |   36 +-
 .../IgniteCommunicationBalanceTest.java         |    4 +-
 .../communication/IgniteIoTestMessagesTest.java |    4 +-
 .../IgniteVariousConnectionNumberTest.java      |   10 +-
 .../GridDeploymentManagerStopSelfTest.java      |    2 +-
 .../GridDeploymentMessageCountSelfTest.java     |   22 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   69 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |   84 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |  214 -
 .../IgniteTopologyPrintFormatSelfTest.java      |    8 +-
 .../events/GridEventStorageManagerSelfTest.java |    4 +-
 .../GridSwapSpaceCustomLocalValue.java          |   85 -
 .../swapspace/GridSwapSpaceManagerSelfTest.java |  170 -
 .../managers/swapspace/package-info.java        |   22 -
 .../OptimizedMarshallerEnumSelfTest.java        |   87 +
 .../OptimizedMarshallerNodeFailoverTest.java    |  358 ++
 .../OptimizedMarshallerPooledSelfTest.java      |   46 +
 .../optimized/OptimizedMarshallerSelfTest.java  |  284 ++
 ...arshallerSerialPersistentFieldsSelfTest.java |  116 +
 .../optimized/OptimizedMarshallerTest.java      |  798 +++
 .../OptimizedObjectStreamSelfTest.java          | 2162 ++++++++
 .../TestTcpDiscoveryIpFinderAdapter.java        |   43 +
 .../marshaller/optimized/package-info.java      |   22 +
 .../pagemem/impl/PageIdUtilsSelfTest.java       |  146 +
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |  380 ++
 .../GridCacheTxLoadFromStoreOnLockSelfTest.java |    6 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    6 +-
 .../AtomicCacheAffinityConfigurationTest.java   |  185 +
 .../BinaryObjectOffHeapUnswapTemporaryTest.java |  367 --
 .../cache/CacheAffinityCallSelfTest.java        |    6 +-
 .../CacheAtomicSingleMessageCountSelfTest.java  |   12 +-
 .../cache/CacheClientStoreSelfTest.java         |   43 +-
 .../cache/CacheConcurrentReadThroughTest.java   |  180 +
 .../cache/CacheConfigurationLeakTest.java       |   27 +-
 .../cache/CacheDeferredDeleteQueueTest.java     |    6 +-
 .../CacheDeferredDeleteSanitySelfTest.java      |    4 +-
 ...cheDhtLocalPartitionAfterRemoveSelfTest.java |   12 +-
 .../cache/CacheEntryProcessorCopySelfTest.java  |    4 +-
 .../cache/CacheEnumOperationsAbstractTest.java  |   60 +-
 ...CacheExchangeMessageDuplicatedStateTest.java |  162 +-
 .../cache/CacheFutureExceptionSelfTest.java     |   14 +-
 .../cache/CacheGetEntryAbstractTest.java        |   44 +-
 .../processors/cache/CacheGetFromJobTest.java   |    7 +-
 .../cache/CacheGroupsMetricsRebalanceTest.java  |  140 +
 ...erceptorPartitionCounterLocalSanityTest.java |   31 +-
 ...torPartitionCounterRandomOperationsTest.java |  119 +-
 .../cache/CacheKeepBinaryTransactionTest.java   |  121 +
 .../CacheMemoryPolicyConfigurationTest.java     |  172 +
 .../CacheMetricsForClusterGroupSelfTest.java    |    4 +-
 .../processors/cache/CacheNamesSelfTest.java    |   12 +-
 .../CacheNamesWithSpecialCharactersTest.java    |    8 +-
 .../cache/CacheNearReaderUpdateTest.java        |   30 +-
 ...cheNearUpdateTopologyChangeAbstractTest.java |    8 +-
 .../cache/CacheOffheapMapEntrySelfTest.java     |   62 +-
 ...cTransactionsWithFilterSingleServerTest.java |   28 +
 ...cheOptimisticTransactionsWithFilterTest.java |  493 ++
 .../CachePutEventListenerErrorSelfTest.java     |   52 +-
 .../processors/cache/CachePutIfAbsentTest.java  |    6 +-
 .../cache/CacheReadThroughRestartSelfTest.java  |   14 +-
 .../cache/CacheRebalancingSelfTest.java         |   27 +-
 .../cache/CacheRemoveAllSelfTest.java           |    5 +-
 .../CacheSerializableTransactionsTest.java      |  244 +-
 .../CacheStartupInDeploymentModesTest.java      |   47 +-
 .../cache/CacheStopAndDestroySelfTest.java      |   88 +-
 .../CacheStoreUsageMultinodeAbstractTest.java   |   20 +-
 ...eUsageMultinodeDynamicStartAbstractTest.java |    4 +-
 .../cache/CacheSwapUnswapGetTest.java           |  313 --
 .../CacheSwapUnswapGetTestSmallQueueSize.java   |   35 -
 .../processors/cache/CacheTxFastFinishTest.java |   10 +-
 .../CacheTxNotAllowReadFromBackupTest.java      |    4 +-
 .../cache/ClusterStateAbstractTest.java         |  439 ++
 .../cache/ClusterStatePartitionedSelfTest.java  |   41 +
 .../cache/ClusterStateReplicatedSelfTest.java   |   42 +
 .../processors/cache/CrossCacheLockTest.java    |   10 +-
 ...sCacheTxNearEnabledRandomOperationsTest.java |   28 +
 .../cache/CrossCacheTxRandomOperationsTest.java |   55 +-
 .../EntryVersionConsistencyReadThroughTest.java |   22 +-
 ...idAbstractCacheInterceptorRebalanceTest.java |   40 +-
 ...ridCacheAbstractByteArrayValuesSelfTest.java |   10 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   32 +-
 ...cheAbstractFullApiMultithreadedSelfTest.java |  105 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java | 1195 +++--
 .../GridCacheAbstractLocalStoreSelfTest.java    |  119 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |  192 +-
 .../GridCacheAbstractRemoveFailureTest.java     |   50 +-
 .../cache/GridCacheAbstractSelfTest.java        |   53 +-
 .../cache/GridCacheAbstractTxReadTest.java      |   10 +-
 ...acheAbstractUsersAffinityMapperSelfTest.java |   11 +-
 .../cache/GridCacheAffinityApiSelfTest.java     |   30 +-
 .../cache/GridCacheAffinityRoutingSelfTest.java |   16 +-
 .../GridCacheAsyncOperationsLimitSelfTest.java  |   13 +-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |    6 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |   88 +-
 .../cache/GridCacheBasicApiAbstractTest.java    |   34 +-
 .../cache/GridCacheBasicStoreAbstractTest.java  |   10 +-
 ...acheBasicStoreMultithreadedAbstractTest.java |    5 +-
 .../cache/GridCacheClearAllSelfTest.java        |    4 +-
 .../cache/GridCacheClearLocallySelfTest.java    |   18 +-
 .../cache/GridCacheClearSelfTest.java           |  136 +-
 ...GridCacheConcurrentGetCacheOnClientTest.java |  129 +
 .../cache/GridCacheConcurrentMapSelfTest.java   |   15 +-
 .../cache/GridCacheConcurrentMapTest.java       |  138 -
 .../GridCacheConcurrentTxMultiNodeTest.java     |   50 +-
 .../GridCacheConditionalDeploymentSelfTest.java |   49 +-
 ...idCacheConfigurationConsistencySelfTest.java |   67 +-
 ...ridCacheConfigurationValidationSelfTest.java |   73 +-
 .../GridCacheDaemonNodeAbstractSelfTest.java    |   12 +-
 .../GridCacheDeploymentOffHeapSelfTest.java     |   41 -
 ...ridCacheDeploymentOffHeapValuesSelfTest.java |   41 -
 .../cache/GridCacheDeploymentSelfTest.java      |   68 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java |  114 +-
 .../cache/GridCacheEntryVersionSelfTest.java    |   22 +-
 .../GridCacheEvictionEventAbstractTest.java     |   10 +-
 .../GridCacheFinishPartitionsSelfTest.java      |   20 +-
 ...CacheFullTextQueryMultithreadedSelfTest.java |    6 +-
 ...idCacheGetAndTransformStoreAbstractTest.java |    5 +-
 .../cache/GridCacheIncrementTransformTest.java  |   10 +-
 .../GridCacheInterceptorAbstractSelfTest.java   |   71 +-
 ...heInterceptorAtomicOffheapRebalanceTest.java |   30 -
 ...erceptorAtomicPrimaryWriteOrderSelfTest.java |   47 -
 ...GridCacheInterceptorAtomicRebalanceTest.java |    6 -
 ...omicReplicatedPrimaryWriteOrderSelfTest.java |   48 -
 ...acheInterceptorAtomicReplicatedSelfTest.java |    8 -
 .../GridCacheInterceptorAtomicSelfTest.java     |    8 -
 ...GridCacheInterceptorLocalAtomicSelfTest.java |    8 -
 ...ceptorTransactionalOffheapRebalanceTest.java |   35 -
 ...heInterceptorTransactionalRebalanceTest.java |    6 -
 .../cache/GridCacheIteratorPerformanceTest.java |   10 +-
 .../cache/GridCacheKeyCheckSelfTest.java        |   10 +-
 .../processors/cache/GridCacheLeakTest.java     |   11 +-
 .../cache/GridCacheLifecycleAwareSelfTest.java  |   38 +-
 .../cache/GridCacheLuceneQueryIndexTest.java    |    4 +-
 .../GridCacheMarshallerTxAbstractTest.java      |   14 +-
 .../GridCacheMarshallingNodeJoinSelfTest.java   |    8 +-
 .../cache/GridCacheMemoryModeSelfTest.java      |  289 --
 .../GridCacheMissingCommitVersionSelfTest.java  |    8 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |    8 +-
 .../cache/GridCacheMultiUpdateLockSelfTest.java |   10 +-
 ...ridCacheMultinodeUpdateAbstractSelfTest.java |   10 +-
 ...inodeUpdateNearEnabledNoBackupsSelfTest.java |    4 +-
 .../cache/GridCacheMvccFlagsTest.java           |    8 +-
 .../cache/GridCacheMvccManagerSelfTest.java     |   13 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java |   58 +-
 .../processors/cache/GridCacheMvccSelfTest.java |   66 +-
 .../cache/GridCacheNestedTxAbstractTest.java    |   16 +-
 .../cache/GridCacheObjectToStringSelfTest.java  |    9 +-
 .../cache/GridCacheOffHeapCleanupTest.java      |  169 -
 ...HeapMultiThreadedUpdateAbstractSelfTest.java |   31 +-
 ...CacheOffHeapMultiThreadedUpdateSelfTest.java |   14 +-
 .../processors/cache/GridCacheOffHeapTest.java  |  274 -
 .../GridCacheOffHeapTieredAbstractSelfTest.java |  675 ---
 .../GridCacheOffHeapTieredAtomicSelfTest.java   |   32 -
 ...heOffHeapTieredEvictionAbstractSelfTest.java |  364 --
 ...acheOffHeapTieredEvictionAtomicSelfTest.java |   32 -
 .../GridCacheOffHeapTieredEvictionSelfTest.java |   33 -
 .../cache/GridCacheOffHeapTieredSelfTest.java   |   33 -
 .../GridCacheOffHeapValuesEvictionSelfTest.java |  210 -
 .../cache/GridCacheOffheapUpdateSelfTest.java   |   21 +-
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |    4 +-
 .../GridCacheOrderedPreloadingSelfTest.java     |   58 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |   45 +-
 .../GridCachePartitionedAffinitySpreadTest.java |  169 -
 .../cache/GridCachePartitionedGetSelfTest.java  |   24 +-
 ...chePartitionedOffHeapLocalStoreSelfTest.java |   44 -
 ...hePartitionedProjectionAffinitySelfTest.java |   12 +-
 .../cache/GridCachePartitionedWritesTest.java   |    5 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |   23 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |   85 +-
 .../cache/GridCacheQueryEmbeddedValue.java      |   37 +
 .../GridCacheQueryIndexingDisabledSelfTest.java |    6 +-
 .../GridCacheQueryInternalKeysSelfTest.java     |   10 +-
 .../GridCacheReferenceCleanupSelfTest.java      |   29 +-
 .../cache/GridCacheReloadSelfTest.java          |    8 +-
 ...ridCacheReplicatedSynchronousCommitTest.java |   14 +-
 .../GridCacheReturnValueTransferSelfTest.java   |   23 +-
 .../cache/GridCacheSlowTxWarnTest.java          |    4 +-
 .../processors/cache/GridCacheStopSelfTest.java |   20 +-
 ...ridCacheStoreManagerDeserializationTest.java |   35 +-
 .../cache/GridCacheStorePutxSelfTest.java       |    6 +-
 .../cache/GridCacheStoreValueBytesSelfTest.java |   13 +-
 .../cache/GridCacheSwapCleanupTest.java         |   99 -
 .../cache/GridCacheSwapPreloadSelfTest.java     |   16 +-
 .../cache/GridCacheSwapReloadSelfTest.java      |  260 -
 ...ridCacheSwapSpaceSpiConsistencySelfTest.java |  146 -
 ...acheTcpClientDiscoveryMultiThreadedTest.java |    8 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   80 +-
 .../processors/cache/GridCacheTestStore.java    |    2 -
 ...cheTransactionalAbstractMetricsSelfTest.java |    8 +-
 .../GridCacheTtlManagerEvictionSelfTest.java    |   62 +-
 .../cache/GridCacheTtlManagerLoadTest.java      |    4 +-
 .../GridCacheTtlManagerNotificationTest.java    |   12 +-
 .../cache/GridCacheTtlManagerSelfTest.java      |   17 +-
 .../cache/GridCacheUtilsSelfTest.java           |    4 -
 .../GridCacheValueBytesPreloadingSelfTest.java  |   79 +-
 ...idCacheValueConsistencyAbstractSelfTest.java |   75 +-
 .../GridCacheVariableTopologySelfTest.java      |    8 +-
 .../cache/GridCacheVersionMultinodeTest.java    |   52 +-
 .../cache/GridCacheVersionSelfTest.java         |   11 +-
 .../GridCacheVersionTopologyChangeTest.java     |   13 +-
 ...calCacheStoreManagerDeserializationTest.java |    2 +-
 .../cache/GridLocalIgniteSerializationTest.java |   10 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java |   20 +-
 .../processors/cache/H2CacheStoreStrategy.java  |   37 +-
 .../cache/IgniteActiveClusterTest.java          |  182 +
 .../IgniteCacheAbstractStopBusySelfTest.java    |   16 +-
 .../cache/IgniteCacheAbstractTest.java          |   43 +-
 .../cache/IgniteCacheAtomicInvokeTest.java      |    7 -
 .../cache/IgniteCacheAtomicPeekModesTest.java   |    7 -
 ...eCacheAtomicPrimaryWriteOrderInvokeTest.java |   57 -
 ...maryWriteOrderNearEnabledStoreValueTest.java |   31 -
 ...heAtomicPrimaryWriteOrderStoreValueTest.java |   32 -
 ...micPrimaryWriteOrderWithStoreInvokeTest.java |   32 -
 ...IgniteCacheAtomicPutAllFailoverSelfTest.java |    5 -
 .../cache/IgniteCacheAtomicStoreValueTest.java  |    7 -
 .../IgniteCacheAtomicWithStoreInvokeTest.java   |   32 +
 ...IgniteCacheBinaryEntryProcessorSelfTest.java |   12 +-
 .../IgniteCacheBinaryObjectsScanSelfTest.java   |   17 +-
 .../IgniteCacheConfigVariationsFullApiTest.java | 1362 +++--
 ...teCacheConfigurationDefaultTemplateTest.java |   10 +-
 .../IgniteCacheConfigurationTemplateTest.java   |   33 +-
 .../IgniteCacheContainsKeyAbstractSelfTest.java |    8 +-
 .../cache/IgniteCacheContainsKeyAtomicTest.java |  103 +
 ...niteCacheCopyOnReadDisabledAbstractTest.java |    6 +-
 .../cache/IgniteCacheCreateRestartSelfTest.java |    4 +-
 .../cache/IgniteCacheDynamicStopSelfTest.java   |   10 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   28 +-
 ...cheEntryListenerAtomicOffheapTieredTest.java |   32 -
 ...cheEntryListenerAtomicOffheapValuesTest.java |   32 -
 .../IgniteCacheEntryListenerAtomicTest.java     |    7 -
 ...niteCacheEntryListenerExpiredEventsTest.java |   34 +-
 ...teCacheEntryListenerTxOffheapTieredTest.java |   32 -
 ...teCacheEntryListenerTxOffheapValuesTest.java |   32 -
 .../IgniteCacheEntryProcessorCallTest.java      |   19 +-
 .../IgniteCacheEntryProcessorNodeJoinTest.java  |   20 +-
 ...niteCacheExpireAndUpdateConsistencyTest.java |   43 +-
 ...IgniteCacheGetCustomCollectionsSelfTest.java |    6 +-
 .../processors/cache/IgniteCacheGroupsTest.java | 4319 ++++++++++++++++
 .../cache/IgniteCacheIncrementTxTest.java       |   16 +-
 .../IgniteCacheInterceptorSelfTestSuite.java    |   14 +-
 .../cache/IgniteCacheInvokeAbstractTest.java    |   24 +-
 ...gniteCacheInvokeReadThroughAbstractTest.java |  104 +-
 ...iteCacheInvokeReadThroughSingleNodeTest.java |   32 +-
 .../cache/IgniteCacheInvokeReadThroughTest.java |   92 +-
 ...gniteCacheLoadRebalanceEvictionSelfTest.java |   13 +-
 .../IgniteCacheManyAsyncOperationsTest.java     |   12 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    6 +-
 .../cache/IgniteCacheNoSyncForGetTest.java      |  395 ++
 .../cache/IgniteCacheObjectPutSelfTest.java     |  179 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   14 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |   14 +-
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |    4 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |    6 +-
 .../IgniteCachePartitionMapUpdateTest.java      |    8 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |  312 +-
 .../cache/IgniteCachePutAllRestartTest.java     |    4 +-
 .../IgniteCachePutStackOverflowSelfTest.java    |    4 +-
 .../IgniteCacheReadThroughEvictionSelfTest.java |   14 +-
 .../IgniteCacheReadThroughStoreCallTest.java    |    8 +-
 ...iteCacheScanPredicateDeploymentSelfTest.java |    8 +-
 .../cache/IgniteCacheSerializationSelfTest.java |   10 +-
 .../cache/IgniteCacheStartStopLoadTest.java     |    6 +-
 .../processors/cache/IgniteCacheStartTest.java  |  192 +
 .../cache/IgniteCacheStoreCollectionTest.java   |    8 +-
 .../IgniteCacheStoreValueAbstractTest.java      |   51 +-
 .../IgniteCacheTopologySafeGetSelfTest.java     |    6 +-
 .../cache/IgniteCacheTxNearPeekModesTest.java   |    5 +-
 .../cache/IgniteCacheTxPeekModesTest.java       |   12 +-
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |   16 +-
 .../cache/IgniteCachingProviderSelfTest.java    |    8 +-
 .../IgniteClientAffinityAssignmentSelfTest.java |   46 +-
 ...IgniteClientCacheInitializationFailTest.java |  366 ++
 .../IgniteClientCacheStartFailoverTest.java     |  585 +++
 .../IgniteDaemonNodeMarshallerCacheTest.java    |    6 +-
 .../cache/IgniteDynamicCacheAndNodeStop.java    |    8 +-
 .../cache/IgniteDynamicCacheFilterTest.java     |   14 +-
 ...eDynamicCacheStartNoExchangeTimeoutTest.java |   51 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   46 +-
 ...niteDynamicCacheStartStopConcurrentTest.java |   10 +-
 ...niteDynamicCacheWithConfigStartSelfTest.java |    4 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |  296 +-
 .../cache/IgniteExchangeFutureHistoryTest.java  |    8 +-
 .../IgniteIncompleteCacheObjectSelfTest.java    |  186 +
 .../cache/IgniteInternalCacheTypesTest.java     |   16 +-
 ...iteMarshallerCacheClassNameConflictTest.java |  113 +-
 ...lerCacheClientRequestsMappingOnMissTest.java |  136 +-
 ...eMarshallerCacheConcurrentReadWriteTest.java |   12 +-
 .../cache/IgniteNearClientCacheCloseTest.java   |  264 +
 .../cache/IgniteOnePhaseCommitInvokeTest.java   |  213 +
 .../IgniteOnePhaseCommitNearReadersTest.java    |  270 +
 .../cache/IgniteOnePhaseCommitNearSelfTest.java |   18 +-
 .../cache/IgnitePutAllLargeBatchSelfTest.java   |   22 +-
 ...tAllUpdateNonPreloadedPartitionSelfTest.java |   16 +-
 .../IgniteStartCacheInTransactionSelfTest.java  |   22 +-
 .../cache/IgniteStaticCacheStartSelfTest.java   |    6 +-
 .../cache/IgniteSystemCacheOnClientTest.java    |    6 +-
 ...gniteTopologyValidatorAbstractCacheTest.java |  163 +-
 ...ologyValidatorAbstractTxCacheGroupsTest.java |  126 +
 ...iteTopologyValidatorAbstractTxCacheTest.java |   48 +-
 ...opologyValidatorCacheGroupsAbstractTest.java |  131 +
 ...niteTopologyValidatorGridSplitCacheTest.java |  344 ++
 ...torNearPartitionedAtomicCacheGroupsTest.java |   31 +
 ...lidatorNearPartitionedTxCacheGroupsTest.java |   31 +
 ...lidatorPartitionedAtomicCacheGroupsTest.java |   45 +
 ...logyValidatorPartitionedAtomicCacheTest.java |    7 -
 ...gyValidatorPartitionedTxCacheGroupsTest.java |   31 +
 ...alidatorReplicatedAtomicCacheGroupsTest.java |   45 +
 ...ologyValidatorReplicatedAtomicCacheTest.java |    7 -
 ...ogyValidatorReplicatedTxCacheGroupsTest.java |   31 +
 .../processors/cache/IgniteTxAbstractTest.java  |   10 +-
 .../IgniteTxConcurrentGetAbstractTest.java      |   10 +-
 .../cache/IgniteTxConfigCacheSelfTest.java      |   15 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |   62 +-
 .../cache/IgniteTxMultiNodeAbstractTest.java    |   62 +-
 .../IgniteTxMultiThreadedAbstractTest.java      |    4 +-
 .../cache/IgniteTxReentryAbstractSelfTest.java  |    6 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |   50 +-
 .../MarshallerCacheJobRunNodeRestartTest.java   |    4 +-
 .../cache/MemoryPolicyConfigValidationTest.java |  398 ++
 ...AffinityCoordinatorDynamicStartStopTest.java |  139 +
 .../cache/OffHeapTieredTransactionSelfTest.java |  135 -
 ...heapCacheMetricsForClusterGroupSelfTest.java |    7 +-
 .../cache/OffheapCacheOnClientsTest.java        |  143 -
 ...sExchangeOnDiscoveryHistoryOverflowTest.java |  183 +
 .../cache/WithKeepBinaryCacheFullApiTest.java   |  228 +-
 .../BinaryAtomicCacheLocalEntriesSelfTest.java  |   30 +
 .../binary/BinaryMetadataUpdatesFlowTest.java   |  592 +++
 .../BinaryTxCacheLocalEntriesSelfTest.java      |   84 +
 .../CacheKeepBinaryWithInterceptorTest.java     |   26 +-
 ...ryDuplicateIndexObjectsAbstractSelfTest.java |  161 -
 ...yAtomicEntryProcessorDeploymentSelfTest.java |    6 +-
 ...naryObjectMetadataExchangeMultinodeTest.java |  463 ++
 ...acheBinaryObjectUserClassloaderSelfTest.java |   14 +-
 ...naryObjectsAbstractDataStreamerSelfTest.java |    8 +-
 ...aryObjectsAbstractMultiThreadedSelfTest.java |    6 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  212 +-
 .../GridCacheBinaryStoreAbstractSelfTest.java   |    6 +-
 ...yStoreBinariesSimpleNameMappersSelfTest.java |    4 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |   14 +-
 ...CacheClientNodeBinaryObjectMetadataTest.java |   10 +-
 .../DataStreamProcessorBinarySelfTest.java      |    4 +-
 .../GridDataStreamerImplSelfTest.java           |   30 +-
 .../GridCacheAffinityRoutingBinarySelfTest.java |    4 +-
 ...sNearPartitionedByteArrayValuesSelfTest.java |    4 +-
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |    4 +-
 ...ateIndexObjectPartitionedAtomicSelfTest.java |   38 -
 ...xObjectPartitionedTransactionalSelfTest.java |   41 -
 ...AtomicNearDisabledOffheapTieredSelfTest.java |   29 -
 ...ObjectsAtomicNearDisabledOnheapSelfTest.java |   25 +
 ...inaryObjectsAtomicOffheapTieredSelfTest.java |   29 -
 ...dCacheBinaryObjectsAtomicOnheapSelfTest.java |   25 +
 ...tionedNearDisabledOffheapTieredSelfTest.java |   30 -
 ...tsPartitionedNearDisabledOnheapSelfTest.java |   25 +
 ...ObjectsPartitionedOffheapTieredSelfTest.java |   30 -
 ...eBinaryObjectsPartitionedOnheapSelfTest.java |   25 +
 .../dht/GridCacheMemoryModeBinarySelfTest.java  |   36 -
 ...dCacheOffHeapTieredAtomicBinarySelfTest.java |   48 -
 .../GridCacheOffHeapTieredBinarySelfTest.java   |   48 -
 ...fHeapTieredEvictionAtomicBinarySelfTest.java |   96 -
 ...acheOffHeapTieredEvictionBinarySelfTest.java |   96 -
 ...BinaryObjectsLocalOffheapTieredSelfTest.java |   29 -
 ...idCacheBinaryObjectsLocalOnheapSelfTest.java |   26 +
 ...IgniteCacheAbstractExecutionContextTest.java |   12 +-
 .../IgniteCacheAtomicExecutionContextTest.java  |    7 -
 ...niteCacheContinuousExecutionContextTest.java |    4 +-
 ...IgniteCacheIsolatedExecutionContextTest.java |    4 +-
 ...niteCacheP2PDisableExecutionContextTest.java |    4 +-
 ...iteCachePartitionedExecutionContextTest.java |    7 -
 .../IgniteCachePrivateExecutionContextTest.java |    4 +-
 ...niteCacheReplicatedExecutionContextTest.java |    7 -
 .../IgniteCacheSharedExecutionContextTest.java  |    4 +-
 .../IgniteCacheTxExecutionContextTest.java      |    7 -
 ...eAbstractDataStructuresFailoverSelfTest.java |  140 +-
 ...actQueueFailoverDataConsistencySelfTest.java |   21 +-
 ...CacheAtomicReferenceApiSelfAbstractTest.java |  132 +-
 ...idCacheAtomicStampedApiSelfAbstractTest.java |  106 +
 .../GridCacheQueueApiSelfAbstractTest.java      |  192 +-
 .../GridCacheQueueCleanupSelfTest.java          |   34 +-
 ...ridCacheQueueJoinedNodeSelfAbstractTest.java |   12 +-
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |    6 +-
 ...dCacheQueueMultiNodeConsistencySelfTest.java |    7 -
 .../GridCacheSequenceApiSelfAbstractTest.java   |   72 +-
 .../GridCacheSetAbstractSelfTest.java           |  184 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |    6 +-
 .../IgniteAtomicLongApiAbstractSelfTest.java    |   80 +-
 .../IgniteAtomicsAbstractTest.java              |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |    6 +-
 .../IgniteCollectionAbstractTest.java           |   34 +-
 .../IgniteCountDownLatchAbstractSelfTest.java   |   55 +-
 .../IgniteDataStructureUniqueNameTest.java      |   73 +-
 .../IgniteDataStructureWithJobTest.java         |    4 +-
 ...IgniteDataStructuresNoClassOnServerTest.java |   30 +
 .../IgniteLockAbstractSelfTest.java             |   63 +-
 .../IgniteSemaphoreAbstractSelfTest.java        |   71 +-
 ...SemaphoreFailoverSafeReleasePermitsTest.java |   13 +-
 .../GridCacheLocalAtomicOffheapSetSelfTest.java |   32 -
 .../GridCacheLocalAtomicQueueApiSelfTest.java   |    6 -
 .../local/GridCacheLocalAtomicSetSelfTest.java  |    7 -
 .../GridCacheLocalOffheapQueueApiSelfTest.java  |   31 -
 .../local/GridCacheLocalQueueApiSelfTest.java   |    7 -
 .../local/GridCacheLocalSetSelfTest.java        |    7 -
 ...artitionedAtomicOffheapQueueApiSelfTest.java |   32 -
 ...omicOffheapQueueCreateMultiNodeSelfTest.java |   32 -
 ...onedAtomicOffheapQueueMultiNodeSelfTest.java |   32 -
 ...dCachePartitionedAtomicQueueApiSelfTest.java |    7 -
 ...ionedAtomicQueueCreateMultiNodeSelfTest.java |    7 -
 ...micQueueFailoverDataConsistencySelfTest.java |    7 -
 ...PartitionedAtomicQueueMultiNodeSelfTest.java |    7 -
 ...itionedAtomicQueueRotativeMultiNodeTest.java |    7 -
 ...titionedAtomicSequenceMultiThreadedTest.java |   32 +
 ...achePartitionedAtomicSequenceTxSelfTest.java |    6 +-
 ...chePartitionedAtomicSetFailoverSelfTest.java |    7 -
 ...rtitionedDataStructuresFailoverSelfTest.java |    7 -
 ...idCachePartitionedNodeRestartTxSelfTest.java |   31 +-
 ...artitionedOffHeapValuesQueueApiSelfTest.java |   35 -
 ...achePartitionedOffHeapValuesSetSelfTest.java |   32 -
 ...edOffheapDataStructuresFailoverSelfTest.java |   33 -
 ...hePartitionedOffheapSetFailoverSelfTest.java |   32 -
 .../GridCachePartitionedQueueApiSelfTest.java   |    7 -
 ...PartitionedQueueCreateMultiNodeSelfTest.java |   15 +-
 ...dCachePartitionedQueueEntryMoveSelfTest.java |    9 +-
 ...nedQueueFailoverDataConsistencySelfTest.java |    7 -
 ...CachePartitionedQueueJoinedNodeSelfTest.java |    7 -
 ...dCachePartitionedQueueMultiNodeSelfTest.java |    7 -
 ...hePartitionedQueueRotativeMultiNodeTest.java |    7 -
 ...GridCachePartitionedSetFailoverSelfTest.java |    7 -
 .../GridCachePartitionedSetSelfTest.java        |    7 -
 .../IgnitePartitionedQueueNoBackupsTest.java    |   13 +-
 .../IgnitePartitionedSetNoBackupsSelfTest.java  |    6 +-
 ...eplicatedDataStructuresFailoverSelfTest.java |    7 -
 .../GridCacheReplicatedQueueApiSelfTest.java    |    7 -
 ...idCacheReplicatedQueueMultiNodeSelfTest.java |    7 -
 ...cheReplicatedQueueRotativeMultiNodeTest.java |    7 -
 .../GridCacheReplicatedSetSelfTest.java         |    7 -
 .../CacheAbstractRestartSelfTest.java           |    6 +-
 .../distributed/CacheAffinityEarlyTest.java     |   11 +-
 ...acheAsyncOperationsFailoverAbstractTest.java |   20 +-
 .../distributed/CacheAsyncOperationsTest.java   |   40 +-
 ...CacheAtomicNearUpdateTopologyChangeTest.java |    7 -
 .../CacheAtomicPrimarySyncBackPressureTest.java |  165 +
 .../CacheClientsConcurrentStartTest.java        |  250 +
 .../CacheDiscoveryDataConcurrentJoinTest.java   |  215 +
 .../CacheGetFutureHangsSelfTest.java            |   16 +-
 .../CacheGetInsideLockChangingTopologyTest.java |    6 +-
 .../distributed/CacheGroupsPreloadTest.java     |  194 +
 ...eLateAffinityAssignmentFairAffinityTest.java |   32 -
 ...ffinityAssignmentNodeJoinValidationTest.java |    4 +-
 .../CacheLateAffinityAssignmentTest.java        |  225 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java |   52 +-
 .../CacheLockReleaseNodeLeaveTest.java          |   94 +-
 ...NearDisabledAtomicInvokeRestartSelfTest.java |    6 -
 .../CacheNoValueClassOnServerNodeTest.java      |  112 +-
 .../CachePutAllFailoverAbstractTest.java        |   16 +-
 .../cache/distributed/CacheStartOnJoinTest.java |  260 +
 .../CacheTryLockMultithreadedTest.java          |    8 +-
 ...tractDistributedByteArrayValuesSelfTest.java |  167 +-
 .../GridCacheAbstractJobExecutionTest.java      |   37 +-
 .../GridCacheAbstractNodeRestartSelfTest.java   |  122 +-
 ...tractPartitionedByteArrayValuesSelfTest.java |   41 +-
 .../GridCacheAbstractPrimarySyncSelfTest.java   |   10 +-
 .../GridCacheBasicOpAbstractTest.java           |   68 +-
 .../GridCacheClientModesAbstractSelfTest.java   |   40 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |    4 +-
 .../GridCacheEntrySetAbstractSelfTest.java      |    2 +-
 ...acheEntrySetIterationPreloadingSelfTest.java |   10 +-
 .../distributed/GridCacheEventAbstractTest.java |  173 +-
 ...heExpiredEntriesPreloadAbstractSelfTest.java |  122 -
 .../distributed/GridCacheLockAbstractTest.java  |   10 +-
 .../distributed/GridCacheMixedModeSelfTest.java |   16 +-
 .../GridCacheMultiNodeAbstractTest.java         |   39 +-
 .../GridCacheMultiNodeLockAbstractTest.java     |   31 +-
 ...dCacheMultithreadedFailoverAbstractTest.java |   29 +-
 .../GridCacheNodeFailureAbstractTest.java       |   13 +-
 ...ridCachePartitionNotLoadedEventSelfTest.java |   33 +-
 ...chePartitionedReloadAllAbstractSelfTest.java |   23 +-
 .../GridCachePreloadEventsAbstractSelfTest.java |    8 +-
 .../GridCachePreloadLifecycleAbstractTest.java  |    4 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |   17 +-
 .../GridCacheTransformEventSelfTest.java        |    8 +-
 ...yMetadataUpdateChangingTopologySelfTest.java |   24 +-
 ...niteBinaryMetadataUpdateNodeRestartTest.java |    6 +-
 .../distributed/IgniteCache150ClientsTest.java  |   13 +-
 ...micMessageRecoveryPairedConnectionsTest.java |    4 +-
 .../IgniteCacheAtomicNodeJoinTest.java          |    6 -
 .../IgniteCacheAtomicNodeRestartTest.java       |    5 -
 ...heClientMultiNodeUpdateTopologyLockTest.java |  201 +
 ...niteCacheClientNodeChangingTopologyTest.java |  296 +-
 .../IgniteCacheClientNodeConcurrentStart.java   |    8 +-
 ...teCacheClientNodePartitionsExchangeTest.java |   63 +-
 .../IgniteCacheClientReconnectTest.java         |    6 +-
 ...acheConnectionRecovery10ConnectionsTest.java |    4 +-
 .../IgniteCacheConnectionRecoveryTest.java      |   16 +-
 .../IgniteCacheCreatePutMultiNodeSelfTest.java  |   11 +-
 .../distributed/IgniteCacheCreatePutTest.java   |   17 +-
 .../distributed/IgniteCacheGetRestartTest.java  |   14 +-
 ...eCacheGroupsPartitionLossPolicySelfTest.java |  355 ++
 .../distributed/IgniteCacheManyClientsTest.java |   10 +-
 .../IgniteCacheMessageRecoveryAbstractTest.java |   12 +-
 ...eCacheMessageRecoveryIdleConnectionTest.java |   12 +-
 .../IgniteCacheMessageWriteTimeoutTest.java     |    4 +-
 .../IgniteCacheNearOffheapGetSelfTest.java      |  136 -
 .../IgniteCacheNearRestartRollbackSelfTest.java |   18 +-
 .../IgniteCacheNodeJoinAbstractTest.java        |    4 +-
 .../IgniteCachePartitionLossPolicySelfTest.java |  337 ++
 .../distributed/IgniteCachePrimarySyncTest.java |    8 +-
 .../IgniteCachePutGetRestartAbstractTest.java   |   10 +-
 .../IgniteCacheReadFromBackupTest.java          |   26 +-
 .../IgniteCacheServerNodeConcurrentStart.java   |   10 +-
 .../IgniteCacheSingleGetMessageTest.java        |    6 +-
 .../IgniteCacheSizeFailoverTest.java            |   10 +-
 .../IgniteCacheSystemTransactionsSelfTest.java  |   17 +-
 .../IgniteCacheTxFairAffinityNodeJoinTest.java  |   35 -
 .../IgniteCacheTxIteratorSelfTest.java          |   59 +-
 ...arDisabledFairAffinityPutGetRestartTest.java |   35 -
 .../IgniteCrossCacheTxStoreSelfTest.java        |   24 +-
 .../IgniteNoClassOnServerAbstractTest.java      |  135 +
 .../IgniteRejectConnectOnNodeStopTest.java      |  188 +
 .../IgniteTxCachePrimarySyncTest.java           |  107 +-
 ...teSynchronizationModesMultithreadedTest.java |   24 +-
 ...iteTxConsistencyRestartAbstractSelfTest.java |   22 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   27 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   45 +-
 .../IgniteTxPreloadAbstractTest.java            |    4 +-
 .../IgniteTxTimeoutAbstractTest.java            |    2 +-
 ...tPartitionedOnlyByteArrayValuesSelfTest.java |   64 +-
 ...heAbstractTransformWriteThroughSelfTest.java |    6 +-
 ...acheAtomicExpiredEntriesPreloadSelfTest.java |   46 -
 .../dht/GridCacheAtomicFullApiSelfTest.java     |   19 -
 .../dht/GridCacheAtomicNearCacheSelfTest.java   |  193 +-
 ...EnabledPrimaryWriteOrderFullApiSelfTest.java |   31 -
 ...eAtomicPrimaryWriteOrderFullApiSelfTest.java |   32 -
 ...tomicPrimaryWriteOrderReloadAllSelfTest.java |   32 -
 .../dht/GridCacheColocatedDebugTest.java        |  117 +-
 ...eColocatedOptimisticTransactionSelfTest.java |   10 +-
 ...dCacheColocatedTxSingleThreadedSelfTest.java |    6 +-
 .../GridCacheDhtAtomicRemoveFailureTest.java    |    7 -
 .../dht/GridCacheDhtEntrySelfTest.java          |   24 +-
 .../dht/GridCacheDhtEntrySetSelfTest.java       |    4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   19 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |  357 --
 .../GridCacheDhtEvictionsDisabledSelfTest.java  |   10 +-
 ...idCacheDhtExpiredEntriesPreloadSelfTest.java |   39 -
 .../dht/GridCacheDhtMappingSelfTest.java        |    8 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |   14 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   49 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   27 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |   12 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   10 +-
 .../dht/GridCacheDhtPreloadOffHeapSelfTest.java |   38 -
 .../dht/GridCacheDhtPreloadOnheapSelfTest.java  |   26 +
 .../dht/GridCacheDhtPreloadPerformanceTest.java |    4 +-
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |    8 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   79 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |   12 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |   28 +-
 .../dht/GridCacheDhtTxPreloadSelfTest.java      |    4 +-
 .../dht/GridCacheGlobalLoadTest.java            |   25 +-
 ...icOffHeapTieredMultiNodeFullApiSelfTest.java |   43 -
 ...NearDisabledAtomicOnheapFullApiSelfTest.java |   38 +
 ...ledAtomicOnheapMultiNodeFullApiSelfTest.java |   38 +
 ...ledFairAffinityMultiNodeFullApiSelfTest.java |   36 -
 ...ePartitionedNearDisabledMetricsSelfTest.java |   20 +-
 ...sabledMultiNodeWithGroupFullApiSelfTest.java |   35 +
 ...ionedNearDisabledOffHeapFullApiSelfTest.java |   33 -
 ...DisabledOffHeapMultiNodeFullApiSelfTest.java |   33 -
 ...abledOffHeapTieredAtomicFullApiSelfTest.java |   57 -
 ...earDisabledOffHeapTieredFullApiSelfTest.java |   33 -
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   33 -
 ...tionedNearDisabledOnheapFullApiSelfTest.java |   27 +
 ...rDisabledOnheapMultiNodeFullApiSelfTest.java |   27 +
 ...idCachePartitionedPreloadEventsSelfTest.java |   10 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   24 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   14 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |   23 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   47 +-
 .../dht/GridNearCacheTxNodeFailureSelfTest.java |   31 -
 ...gniteAtomicLongChangingTopologySelfTest.java |    4 +-
 .../IgniteCacheClearDuringRebalanceTest.java    |  126 +
 .../IgniteCacheCommitDelayTxRecoveryTest.java   |   26 +-
 .../dht/IgniteCacheConcurrentPutGetRemove.java  |   35 +-
 .../IgniteCacheCrossCacheTxFailoverTest.java    |   60 +-
 .../dht/IgniteCacheLockFailoverSelfTest.java    |   16 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   10 +-
 ...artitionedBackupNodeFailureRecoveryTest.java |   31 +-
 ...woBackupsPrimaryNodeFailureRecoveryTest.java |    4 +-
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |   25 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |  191 +-
 .../dht/IgniteCachePutRetryAtomicSelfTest.java  |    6 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |   38 +-
 .../dht/IgniteCacheTxRecoveryRollbackTest.java  |  502 ++
 .../dht/IgniteCrossCacheTxSelfTest.java         |   12 +-
 .../dht/IgniteTxReentryColocatedSelfTest.java   |    2 +-
 .../dht/TxRecoveryStoreEnabledTest.java         |  231 +
 .../AtomicPutAllChangingTopologyTest.java       |    6 +-
 .../atomic/GridCacheAtomicFailoverSelfTest.java |    7 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |  128 +-
 .../atomic/GridCacheAtomicPreloadSelfTest.java  |   14 +-
 ...AtomicPrimaryWriteOrderFailoverSelfTest.java |   36 -
 ...tomicPrimaryWriteOrderRemoveFailureTest.java |   53 -
 .../GridCacheAtomicRemoveFailureTest.java       |    7 -
 ...micPrimaryWriteOrderNearEnabledSelfTest.java |   39 -
 ...sistencyAtomicPrimaryWriteOrderSelfTest.java |   32 -
 .../atomic/IgniteCacheAtomicProtocolTest.java   |  965 ++++
 ...PutRetryAtomicPrimaryWriteOrderSelfTest.java |   32 -
 ...tionedMultiNodeLongTxTimeoutFullApiTest.java |    4 +-
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |    4 +-
 ...nlyFairAffinityMultiNodeFullApiSelfTest.java |   36 -
 ...tomicClientOnlyMultiNodeFullApiSelfTest.java |  177 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 ...yOnReadDisabledMultiNodeFullApiSelfTest.java |    4 +-
 ...micFairAffinityMultiNodeFullApiSelfTest.java |   35 -
 ...LateAffDisabledMultiNodeFullApiSelfTest.java |   35 +
 ...imaryWriteOrderMultiNodeFullApiSelfTest.java |   35 -
 ...GridCacheAtomicMultiNodeFullApiSelfTest.java |   14 +-
 ...omicMultiNodeP2PDisabledFullApiSelfTest.java |   19 -
 ...AtomicMultiNodeWithGroupFullApiSelfTest.java |   34 +
 ...ledFairAffinityMultiNodeFullApiSelfTest.java |   36 -
 ...nabledMultiNodeWithGroupFullApiSelfTest.java |   35 +
 ...imaryWriteOrderMultiNodeFullApiSelfTest.java |   31 -
 ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java |   58 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCacheAtomicNearRemoveFailureTest.java   |    7 -
 .../GridCacheAtomicOffHeapFullApiSelfTest.java  |   67 -
 ...heAtomicOffHeapMultiNodeFullApiSelfTest.java |   68 -
 ...CacheAtomicOffHeapTieredFullApiSelfTest.java |   32 -
 ...icOffHeapTieredMultiNodeFullApiSelfTest.java |   33 -
 .../GridCacheAtomicOnheapFullApiSelfTest.java   |   28 +
 ...cheAtomicOnheapMultiNodeFullApiSelfTest.java |   26 +
 ...idCacheAtomicPartitionedMetricsSelfTest.java |    6 +-
 ...AtomicPartitionedTckMetricsSelfTestImpl.java |    8 +-
 ...derFairAffinityMultiNodeFullApiSelfTest.java |   36 -
 ...imaryWriteOrderMultiNodeFullApiSelfTest.java |   32 -
 ...rderMultiNodeP2PDisabledFullApiSelfTest.java |   33 -
 ...cPrimaryWriteOrderNearRemoveFailureTest.java |   52 -
 ...erNoStripedPoolMultiNodeFullApiSelfTest.java |   35 -
 ...PrimaryWriteOrderOffHeapFullApiSelfTest.java |   32 -
 ...yWriteOrderOffHeapTieredFullApiSelfTest.java |   33 -
 ...ityOrderOffHeapMultiNodeFullApiSelfTest.java |   33 -
 ...erOffHeapTieredMultiNodeFullApiSelfTest.java |   33 -
 .../near/GridCacheGetStoreErrorSelfTest.java    |    8 +-
 .../near/GridCacheNearEvictionSelfTest.java     |   17 +-
 ...dCacheNearExpiredEntriesPreloadSelfTest.java |   33 -
 .../near/GridCacheNearJobExecutionSelfTest.java |    4 +-
 .../near/GridCacheNearMetricsSelfTest.java      |   65 +-
 .../near/GridCacheNearMultiGetSelfTest.java     |   14 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |   54 +-
 .../near/GridCacheNearOneNodeSelfTest.java      |    2 +-
 ...nlyFairAffinityMultiNodeFullApiSelfTest.java |   35 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   43 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |   20 +-
 .../GridCacheNearPartitionedClearSelfTest.java  |    6 +-
 .../GridCacheNearReaderPreloadSelfTest.java     |    8 +-
 .../near/GridCacheNearReadersSelfTest.java      |   89 +-
 .../near/GridCacheNearTxForceKeyTest.java       |   22 +-
 .../near/GridCacheNearTxMultiNodeSelfTest.java  |   22 +-
 .../near/GridCacheNearTxPreloadSelfTest.java    |    4 +-
 ...AffinityExcludeNeighborsPerformanceTest.java |    6 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |  102 -
 .../GridCachePartitionedAffinitySelfTest.java   |   28 +-
 ...ePartitionedAtomicOnheapFullApiSelfTest.java |   38 +
 ...nedAtomicOnheapMultiNodeFullApiSelfTest.java |   38 +
 .../near/GridCachePartitionedBasicApiTest.java  |    4 +-
 .../GridCachePartitionedBasicOpSelfTest.java    |    4 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java |   27 +-
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    8 +-
 ...yOnReadDisabledMultiNodeFullApiSelfTest.java |    4 +-
 .../near/GridCachePartitionedEventSelfTest.java |    2 +-
 .../GridCachePartitionedEvictionSelfTest.java   |    9 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |    8 +-
 ...GridCachePartitionedFilteredPutSelfTest.java |    6 +-
 .../GridCachePartitionedFullApiSelfTest.java    |   35 +-
 ...idCachePartitionedHitsAndMissesSelfTest.java |    9 +-
 ...LateAffDisabledMultiNodeFullApiSelfTest.java |    4 +-
 .../GridCachePartitionedLoadCacheSelfTest.java  |   15 +-
 .../near/GridCachePartitionedLockSelfTest.java  |    4 +-
 .../GridCachePartitionedMetricsSelfTest.java    |    8 +-
 ...achePartitionedMultiNodeCounterSelfTest.java |  115 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |  142 +-
 ...onedMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCachePartitionedMultiNodeSelfTest.java  |    4 +-
 ...tionedMultiNodeWithGroupFullApiSelfTest.java |   34 +
 ...ePartitionedMultiThreadedPutGetSelfTest.java |   15 +-
 .../near/GridCachePartitionedNestedTxTest.java  |    4 +-
 ...edNoStripedPoolMultiNodeFullApiSelfTest.java |   35 -
 ...GridCachePartitionedNodeFailureSelfTest.java |    4 +-
 .../GridCachePartitionedNodeRestartTest.java    |    5 +-
 ...dCachePartitionedOffHeapFullApiSelfTest.java |   32 -
 ...titionedOffHeapMultiNodeFullApiSelfTest.java |   32 -
 ...PartitionedOffHeapTieredFullApiSelfTest.java |   32 -
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   73 -
 ...idCachePartitionedOnheapFullApiSelfTest.java |   26 +
 ...rtitionedOnheapMultiNodeFullApiSelfTest.java |   26 +
 ...ePartitionedOptimisticTxNodeRestartTest.java |    5 +-
 ...achePartitionedPreloadLifecycleSelfTest.java |    5 +-
 .../GridCachePartitionedStorePutSelfTest.java   |   10 +-
 ...GridCachePartitionedTxConcurrentGetTest.java |    4 +-
 ...GridCachePartitionedTxMultiNodeSelfTest.java |    4 +-
 ...CachePartitionedTxMultiThreadedSelfTest.java |    4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   15 +-
 ...achePartitionedTxSingleThreadedSelfTest.java |    6 +-
 .../GridCachePartitionedTxTimeoutSelfTest.java  |    4 +-
 .../near/GridCachePutArrayValueSelfTest.java    |    6 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    6 +-
 .../near/GridNearCacheStoreUpdateTest.java      |  466 ++
 .../near/GridPartitionedBackupLoadSelfTest.java |   12 +-
 .../near/IgniteCacheNearOnlyTxTest.java         |   42 +-
 .../near/IgniteCacheNearReadCommittedTest.java  |    4 +-
 .../near/IgniteCacheNearTxRollbackTest.java     |    4 +-
 .../near/IgniteTxReentryNearSelfTest.java       |    2 +-
 .../near/NearCacheMultithreadedUpdateTest.java  |   12 +-
 .../near/NearCachePutAllMultinodeTest.java      |    9 +-
 .../near/NearCacheSyncUpdateTest.java           |   35 +-
 .../near/NoneRebalanceModeSelfTest.java         |   12 +-
 .../rebalancing/CacheManualRebalancingTest.java |  178 +
 ...cingDelayedPartitionMapExchangeSelfTest.java |   12 +-
 .../GridCacheRebalancingAsyncSelfTest.java      |    4 +-
 .../GridCacheRebalancingOrderingTest.java       |   10 +-
 ...cheRebalancingPartitionDistributionTest.java |    4 +-
 .../GridCacheRebalancingSyncCheckDataTest.java  |   10 +-
 .../GridCacheRebalancingSyncSelfTest.java       |  118 +-
 ...eRebalancingUnmarshallingFailedSelfTest.java |   12 +-
 ...xcludeNeighborsMultiNodeFullApiSelfTest.java |   36 -
 ...tedFairAffinityMultiNodeFullApiSelfTest.java |   35 -
 ...xcludeNeighborsMultiNodeFullApiSelfTest.java |    4 +-
 ...dezvousAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...stractReplicatedByteArrayValuesSelfTest.java |   36 +-
 ...ridCacheAtomicReplicatedMetricsSelfTest.java |    6 +-
 ...nedFairAffinityMultiNodeFullApiSelfTest.java |   37 -
 ...eplicatedAtomicMultiNodeFullApiSelfTest.java |   19 -
 ...imaryWriteOrderMultiNodeFullApiSelfTest.java |   33 -
 .../GridCacheReplicatedBasicApiTest.java        |    4 +-
 .../GridCacheReplicatedBasicOpSelfTest.java     |    4 +-
 .../GridCacheReplicatedEvictionSelfTest.java    |  135 -
 .../GridCacheReplicatedFullApiSelfTest.java     |   13 +-
 .../GridCacheReplicatedJobExecutionTest.java    |    4 +-
 .../GridCacheReplicatedMarshallerTxTest.java    |    4 +-
 .../GridCacheReplicatedMetricsSelfTest.java     |    4 +-
 ...atedMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCacheReplicatedMultiNodeSelfTest.java   |    4 +-
 .../GridCacheReplicatedNodeFailureSelfTest.java |    4 +-
 .../GridCacheReplicatedNodeRestartSelfTest.java |    7 +-
 ...idCacheReplicatedOffHeapFullApiSelfTest.java |   32 -
 ...plicatedOffHeapMultiNodeFullApiSelfTest.java |   32 -
 ...eReplicatedOffHeapTieredFullApiSelfTest.java |   33 -
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   33 -
 ...ridCacheReplicatedOnheapFullApiSelfTest.java |   26 +
 ...eplicatedOnheapMultiNodeFullApiSelfTest.java |   26 +
 .../GridCacheReplicatedTxConcurrentGetTest.java |    4 +-
 ...GridCacheReplicatedTxMultiNodeBasicTest.java |    4 +-
 ...dCacheReplicatedTxMultiThreadedSelfTest.java |    4 +-
 ...CacheReplicatedTxSingleThreadedSelfTest.java |    4 +-
 .../GridCacheReplicatedTxTimeoutSelfTest.java   |    4 +-
 ...idCacheReplicatedUnswapAdvancedSelfTest.java |  153 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |   18 +-
 .../replicated/GridReplicatedTxPreloadTest.java |    4 +-
 .../IgniteCacheSyncRebalanceModeSelfTest.java   |    4 +-
 ...CacheReplicatedPreloadLifecycleSelfTest.java |    5 +-
 ...idCacheReplicatedPreloadOffHeapSelfTest.java |   37 -
 .../GridCacheReplicatedPreloadSelfTest.java     |   92 +-
 ...eplicatedPreloadStartStopEventsSelfTest.java |    6 +-
 .../cache/eviction/EvictionAbstractTest.java    |   70 +-
 .../GridCacheBatchEvictUnswapSelfTest.java      |  200 -
 ...heConcurrentEvictionConsistencySelfTest.java |   14 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |    9 +-
 .../GridCacheDistributedEvictionsSelfTest.java  |  237 -
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |   33 +-
 .../GridCacheEvictableEntryEqualsSelfTest.java  |    1 +
 .../GridCacheEvictionFilterSelfTest.java        |   11 +-
 .../GridCacheEvictionLockUnlockSelfTest.java    |    5 +-
 .../GridCacheEvictionTouchSelfTest.java         |   26 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |  167 -
 .../lru/LruNearEvictionPolicySelfTest.java      |   39 +-
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |   68 +-
 .../paged/PageEvictionAbstractTest.java         |  138 +
 .../paged/PageEvictionMultinodeTest.java        |  128 +
 .../paged/PageEvictionReadThroughTest.java      |  140 +
 .../paged/PageEvictionTouchOrderTest.java       |  109 +
 .../paged/PageEvictionWithRebalanceTest.java    |   81 +
 ...LruNearEnabledPageEvictionMultinodeTest.java |   28 +
 .../Random2LruPageEvictionMultinodeTest.java    |   30 +
 ...Random2LruPageEvictionWithRebalanceTest.java |   30 +
 ...LruNearEnabledPageEvictionMultinodeTest.java |   28 +
 .../RandomLruPageEvictionMultinodeTest.java     |   30 +
 .../RandomLruPageEvictionWithRebalanceTest.java |   30 +
 .../cache/eviction/paged/TestObject.java        |   78 +
 .../SortedEvictionPolicyPerformanceTest.java    |    8 +-
 .../IgniteCacheAtomicExpiryPolicyTest.java      |    7 -
 ...iteCacheAtomicExpiryPolicyWithStoreTest.java |    7 -
 ...CacheAtomicLocalOffheapExpiryPolicyTest.java |   30 -
 ...eCacheAtomicLocalOnheapExpiryPolicyTest.java |   26 +
 ...gniteCacheAtomicOffheapExpiryPolicyTest.java |   30 -
 ...IgniteCacheAtomicOnheapExpiryPolicyTest.java |   26 +
 ...AtomicPrimaryWriteOrderExpiryPolicyTest.java |   32 -
 ...rimaryWriteOrderOffheapExpiryPolicyTest.java |   31 -
 ...maryWriteOrderWithStoreExpiryPolicyTest.java |   32 -
 ...teOrderWithStoreOffheapExpiryPolicyTest.java |   31 -
 ...AtomicReplicatedOffheapExpiryPolicyTest.java |   30 -
 ...eAtomicWithStoreOffheapExpiryPolicyTest.java |   30 -
 .../IgniteCacheClientNearCacheExpiryTest.java   |   36 +-
 .../IgniteCacheExpiryPolicyAbstractTest.java    |  187 +-
 .../IgniteCacheExpiryPolicyTestSuite.java       |   17 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |   32 +-
 .../expiry/IgniteCacheLargeValueExpireTest.java |  175 +
 ...eCacheOnlyOneTtlCleanupThreadExistsTest.java |    5 +-
 .../expiry/IgniteCacheTtlCleanupSelfTest.java   |   14 +-
 ...niteCacheTxLocalOffheapExpiryPolicyTest.java |   30 -
 .../IgniteCacheTxOffheapExpiryPolicyTest.java   |   30 -
 ...acheTxReplicatedOffheapExpiryPolicyTest.java |   30 -
 ...CacheTxWithStoreOffheapExpiryPolicyTest.java |   30 -
 .../IgniteCacheAtomicLoadAllTest.java           |    7 -
 .../IgniteCacheAtomicLoaderWriterTest.java      |    7 -
 ...gniteCacheAtomicNoLoadPreviousValueTest.java |    7 -
 .../IgniteCacheAtomicNoReadThroughTest.java     |    7 -
 .../IgniteCacheAtomicNoWriteThroughTest.java    |    7 -
 .../IgniteCacheAtomicStoreSessionTest.java      |    7 -
 ...eCacheAtomicStoreSessionWriteBehindTest.java |    7 -
 ...IgniteCacheJdbcBlobStoreNodeRestartTest.java |   11 +-
 .../IgniteCacheLoadAllAbstractTest.java         |   15 +-
 ...iteCacheNoLoadPreviousValueAbstractTest.java |    8 +-
 .../IgniteCacheNoReadThroughAbstractTest.java   |    8 +-
 .../IgniteCacheNoWriteThroughAbstractTest.java  |    8 +-
 ...IgniteCacheStoreNodeRestartAbstractTest.java |    6 +-
 .../IgniteCacheStoreSessionAbstractTest.java    |   11 +-
 ...acheStoreSessionWriteBehindAbstractTest.java |    8 +-
 .../IgniteCacheTxStoreSessionTest.java          |   22 +-
 ...dCacheAtomicLocalMetricsNoStoreSelfTest.java |    4 +-
 .../GridCacheAtomicLocalMetricsSelfTest.java    |    6 +-
 ...dCacheAtomicLocalTckMetricsSelfTestImpl.java |    8 +-
 ...LocalAtomicMetricsNoReadThroughSelfTest.java |   48 +
 ...dCacheLocalAtomicOffHeapFullApiSelfTest.java |   42 -
 ...LocalAtomicOffHeapTieredFullApiSelfTest.java |   32 -
 ...acheLocalAtomicWithGroupFullApiSelfTest.java |   34 +
 .../local/GridCacheLocalBasicApiSelfTest.java   |    4 +-
 ...cheLocalBasicStoreMultithreadedSelfTest.java |    1 +
 .../GridCacheLocalByteArrayValuesSelfTest.java  |   75 +-
 .../local/GridCacheLocalFullApiSelfTest.java    |   18 +-
 .../local/GridCacheLocalLoadAllSelfTest.java    |    4 +-
 .../cache/local/GridCacheLocalLockSelfTest.java |    6 +-
 .../local/GridCacheLocalMetricsSelfTest.java    |    8 +-
 .../GridCacheLocalMultithreadedSelfTest.java    |    2 +-
 .../GridCacheLocalOffHeapFullApiSelfTest.java   |   30 -
 ...dCacheLocalOffHeapTieredFullApiSelfTest.java |   32 -
 .../GridCacheLocalTxMultiThreadedSelfTest.java  |    4 +-
 .../GridCacheLocalTxSingleThreadedSelfTest.java |    4 +-
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |    2 +-
 .../GridCacheLocalWithGroupFullApiSelfTest.java |   34 +
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java |  621 ---
 ...OnlyFairAffinityMultiJvmFullApiSelfTest.java |   31 -
 ...omicFairAffinityMultiJvmFullApiSelfTest.java |   31 -
 ...bledFairAffinityMultiJvmFullApiSelfTest.java |   36 -
 ...rimaryWriteOrderMultiJvmFullApiSelfTest.java |   31 -
 ...cheAtomicOffHeapMultiJvmFullApiSelfTest.java |   31 -
 ...micOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...acheAtomicOnheapMultiJvmFullApiSelfTest.java |   28 +
 ...rderFairAffinityMultiJvmFullApiSelfTest.java |   31 -
 ...rimaryWriteOrderMultiJvmFullApiSelfTest.java |   31 -
 ...OrderMultiJvmP2PDisabledFullApiSelfTest.java |   31 -
 ...rityOrderOffHeapMultiJvmFullApiSelfTest.java |   31 -
 ...derOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...OnlyFairAffinityMultiJvmFullApiSelfTest.java |   31 -
 ...onedFairAffinityMultiJvmFullApiSelfTest.java |   31 -
 ...micOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...bledAtomicOnheapMultiJvmFullApiSelfTest.java |   28 +
 ...bledFairAffinityMultiJvmFullApiSelfTest.java |   31 -
 ...rDisabledOffHeapMultiJvmFullApiSelfTest.java |   31 -
 ...ledOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...arDisabledOnheapMultiJvmFullApiSelfTest.java |   28 +
 ...rtitionedOffHeapMultiJvmFullApiSelfTest.java |   31 -
 ...nedOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...artitionedOnheapMultiJvmFullApiSelfTest.java |   28 +
 ...rimaryWriteOrderMultiJvmFullApiSelfTest.java |   31 -
 ...eplicatedOffHeapMultiJvmFullApiSelfTest.java |   31 -
 ...tedOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...ReplicatedOnheapMultiJvmFullApiSelfTest.java |   28 +
 .../IgnitePdsAtomicCacheRebalancingTest.java    |   40 +
 ...tePdsBinaryMetadataOnClusterRestartTest.java |  514 ++
 .../IgnitePdsCacheRebalancingAbstractTest.java  |  539 ++
 .../IgnitePdsClientNearCachePutGetTest.java     |   55 +
 .../IgnitePdsContinuousRestartTest.java         |  261 +
 .../persistence/IgnitePdsDynamicCacheTest.java  |  181 +
 .../IgnitePdsMultiNodePutGetRestartTest.java    |  256 +
 .../persistence/IgnitePdsPageSizesTest.java     |  154 +
 ...gnitePdsRecoveryAfterFileCorruptionTest.java |  365 ++
 .../IgnitePdsRemoveDuringRebalancingTest.java   |  145 +
 ...gnitePdsSingleNodePutGetPersistenceTest.java |   55 +
 .../IgnitePdsTxCacheRebalancingTest.java        |   61 +
 .../IgnitePersistenceMetricsSelfTest.java       |  225 +
 .../IgnitePersistentStoreCacheGroupsTest.java   |  514 ++
 ...IgnitePersistentStoreDataStructuresTest.java |  269 +
 .../MemoryPolicyInitializationTest.java         |  313 ++
 .../db/IgnitePdsMultiNodePutGetRestartTest.java |  246 +
 ...PdsPageEvictionDuringPartitionClearTest.java |  176 +
 .../db/IgnitePdsPageEvictionTest.java           |  241 +
 ...tePdsRebalancingOnNotStableTopologyTest.java |  217 +
 .../db/IgnitePdsTransactionsHangTest.java       |  306 ++
 .../db/IgnitePdsWholeClusterRestartTest.java    |  165 +
 .../db/file/IgnitePdsCacheIntegrationTest.java  |  291 ++
 ...ckpointSimulationWithRealCpDisabledTest.java |  984 ++++
 .../db/file/IgnitePdsEvictionTest.java          |  299 ++
 .../file/IgnitePdsNoActualWalHistoryTest.java   |  219 +
 .../persistence/db/wal/IgnitePdsWalTlbTest.java |  136 +
 .../wal/IgniteWalHistoryReservationsTest.java   |  403 ++
 .../IgniteWalRecoverySeveralRestartsTest.java   |  359 ++
 .../db/wal/IgniteWalRecoveryTest.java           | 1334 +++++
 .../db/wal/WalRecoveryTxLogicalRecordsTest.java |  981 ++++
 .../db/wal/crc/IgniteDataIntegrityTests.java    |  137 +
 .../pagemem/BPlusTreePageMemoryImplTest.java    |   96 +
 .../BPlusTreeReuseListPageMemoryImplTest.java   |   97 +
 .../pagemem/FullPageIdTableTest.java            |  115 +
 .../MetadataStoragePageMemoryImplTest.java      |  102 +
 .../pagemem/NoOpPageStoreManager.java           |  199 +
 .../persistence/pagemem/NoOpWALManager.java     |  128 +
 .../pagemem/PageIdDistributionTest.java         |  234 +
 .../pagemem/PageMemoryImplNoLoadTest.java       |   98 +
 .../persistence/pagemem/PageMemoryImplTest.java |  119 +
 .../AbstractNodeJoinTemplate.java               |  756 +++
 .../IgniteChangeGlobalStateAbstractTest.java    |  366 ++
 .../IgniteChangeGlobalStateCacheTest.java       |  163 +
 ...IgniteChangeGlobalStateDataStreamerTest.java |  112 +
 ...gniteChangeGlobalStateDataStructureTest.java |  278 +
 .../IgniteChangeGlobalStateFailOverTest.java    |  355 ++
 .../IgniteChangeGlobalStateServiceTest.java     |  102 +
 .../IgniteChangeGlobalStateTest.java            |  711 +++
 .../IgniteStandByClusterTest.java               |  322 ++
 .../extended/GridActivateExtensionTest.java     |  239 +
 .../extended/GridActivationAtomicCacheSuit.java |   79 +
 .../GridActivationCacheAbstractTestSuit.java    |  108 +
 .../GridActivationLocalAndNearCacheSuit.java    |   47 +
 .../GridActivationPartitionedCacheSuit.java     |   80 +
 .../GridActivationReplicatedCacheSuit.java      |   63 +
 .../join/JoinActiveNodeToActiveCluster.java     |  442 ++
 .../join/JoinActiveNodeToInActiveCluster.java   |  231 +
 .../join/JoinInActiveNodeToActiveCluster.java   |  373 ++
 .../join/JoinInActiveNodeToInActiveCluster.java |  230 +
 ...ctiveNodeToActiveClusterWithPersistence.java |  106 +
 ...iveNodeToInActiveClusterWithPersistence.java |   31 +
 ...ctiveNodeToActiveClusterWithPersistence.java |   86 +
 ...iveNodeToInActiveClusterWithPersistence.java |   31 +
 ...gniteAbstractStandByClientReconnectTest.java |  336 ++
 .../IgniteStandByClientReconnectTest.java       |  283 ++
 ...eStandByClientReconnectToNewClusterTest.java |  289 ++
 .../persistence/tree/io/TrackingPageIOTest.java |  283 ++
 .../BinaryAtomicCacheLocalEntriesSelfTest.java  |   30 -
 .../BinaryTxCacheLocalEntriesSelfTest.java      |   84 -
 .../GridCacheQueryTransformerSelfTest.java      |    4 +-
 .../GridCacheSwapScanQueryAbstractSelfTest.java |  435 --
 .../query/GridCacheSwapScanQuerySelfTest.java   |   25 -
 .../IgniteCacheQueryCacheDestroySelfTest.java   |    2 -
 .../cache/query/IndexingSpiQuerySelfTest.java   |  105 +-
 .../cache/query/IndexingSpiQueryTxSelfTest.java |   24 +-
 .../IndexingSpiQueryWithH2IndexingSelfTest.java |   36 +
 .../continuous/CacheContinuousBatchAckTest.java |   87 +-
 ...heContinuousBatchForceServerModeAckTest.java |    8 +-
 ...FailoverAtomicPrimaryWriteOrderSelfTest.java |   50 -
 ...tinuousQueryAsyncFailoverAtomicSelfTest.java |   42 +
 ...eContinuousQueryAsyncFilterListenerTest.java |  190 +-
 ...nuousQueryConcurrentPartitionUpdateTest.java |  389 ++
 ...CacheContinuousQueryCounterAbstractTest.java |    8 +-
 .../CacheContinuousQueryEventBufferTest.java    |  217 +
 ...acheContinuousQueryExecuteInPrimaryTest.java |   10 +-
 ...usQueryFactoryFilterRandomOperationTest.java |    2 -
 ...ContinuousQueryFailoverAbstractSelfTest.java |  232 +-
 ...ryFailoverAtomicNearEnabledSelfSelfTest.java |    9 +-
 ...tomicPrimaryWriteOrderOffheapTieredTest.java |   33 -
 ...FailoverAtomicPrimaryWriteOrderSelfTest.java |   44 -
 ...usQueryFailoverAtomicReplicatedSelfTest.java |    9 +-
 ...heContinuousQueryFailoverAtomicSelfTest.java |   36 +
 ...tinuousQueryFailoverTxOffheapTieredTest.java |   32 -
 .../CacheContinuousQueryLostPartitionTest.java  |    6 +-
 ...ontinuousQueryOperationFromCallbackTest.java |    8 +-
 .../CacheContinuousQueryOperationP2PTest.java   |   39 +-
 .../CacheContinuousQueryOrderingEventTest.java  |  136 +-
 ...acheContinuousQueryRandomOperationsTest.java |  331 +-
 .../CacheContinuousQueryVariationsTest.java     |   11 +-
 ...eEntryProcessorExternalizableFailedTest.java |    4 +-
 .../CacheEntryProcessorNonSerializableTest.java |    9 +-
 ...CacheKeepBinaryIterationNearEnabledTest.java |    6 +-
 ...acheKeepBinaryIterationStoreEnabledTest.java |    6 +-
 ...CacheKeepBinaryIterationSwapEnabledTest.java |   56 -
 .../CacheKeepBinaryIterationTest.java           |  111 +-
 .../ClientReconnectContinuousQueryTest.java     |  201 +
 .../ContinuousQueryPeerClassLoadingTest.java    |  142 +
 ...yRemoteFilterMissingInClassPathSelfTest.java |    6 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   90 +-
 ...eContinuousQueryAtomicOffheapTieredTest.java |   32 -
 ...eContinuousQueryAtomicOffheapValuesTest.java |   32 -
 .../GridCacheContinuousQueryConcurrentTest.java |   58 +-
 ...eContinuousQueryMultiNodesFilteringTest.java |   10 +-
 ...dCacheContinuousQueryNodesFilteringTest.java |    2 +-
 ...dCacheContinuousQueryReplicatedSelfTest.java |    8 +-
 ...eContinuousQueryReplicatedTxOneNodeTest.java |   11 +-
 ...CacheContinuousQueryTxOffheapTieredTest.java |   32 -
 ...CacheContinuousQueryTxOffheapValuesTest.java |   32 -
 ...niteCacheContinuousQueryBackupQueueTest.java |   23 +-
 ...CacheContinuousQueryClientReconnectTest.java |   10 +-
 .../IgniteCacheContinuousQueryClientTest.java   |   45 +-
 ...eCacheContinuousQueryImmutableEntryTest.java |   22 +-
 ...teCacheContinuousQueryNoUnsubscribeTest.java |   20 +-
 ...IgniteCacheContinuousQueryReconnectTest.java |   12 +-
 ...idCacheWriteBehindStoreAbstractSelfTest.java |   24 +-
 .../GridCacheWriteBehindStoreAbstractTest.java  |    5 +-
 ...heWriteBehindStoreMultithreadedSelfTest.java |   87 +-
 ...BehindStorePartitionedMultiNodeSelfTest.java |   10 +-
 .../GridCacheWriteBehindStoreSelfTest.java      |  159 +-
 .../IgniteCacheWriteBehindNoUpdateSelfTest.java |    6 +-
 ...CacheClientWriteBehindStoreAbstractTest.java |   14 +-
 ...teCacheClientWriteBehindStoreAtomicTest.java |    7 -
 ...ClientWriteBehindStoreNonCoalescingTest.java |  168 +
 .../transactions/DepthFirstSearchTest.java      |   18 +-
 .../cache/transactions/TxDeadlockCauseTest.java |  279 +
 ...DeadlockDetectionMessageMarshallingTest.java |  116 +
 .../TxDeadlockDetectionNoHangsTest.java         |    4 +-
 .../transactions/TxDeadlockDetectionTest.java   |    4 +-
 .../TxDeadlockDetectionUnmasrhalErrorsTest.java |  225 +
 ...timisticDeadlockDetectionCrossCacheTest.java |    4 +-
 .../TxOptimisticDeadlockDetectionTest.java      |   46 +-
 ...simisticDeadlockDetectionCrossCacheTest.java |    7 +-
 .../TxPessimisticDeadlockDetectionTest.java     |   23 +-
 .../CacheVersionedEntryAbstractTest.java        |   27 +-
 ...edEntryPartitionedAtomicOffHeapSelfTest.java |   35 -
 ...PartitionedTransactionalOffHeapSelfTest.java |   36 -
 ...nedEntryReplicatedAtomicOffHeapSelfTest.java |   35 -
 ...yReplicatedTransactionalOffHeapSelfTest.java |   36 -
 .../clock/GridTimeSyncProcessorSelfTest.java    |  224 -
 .../closure/GridClosureProcessorRemoteTest.java |    2 +-
 .../closure/GridClosureProcessorSelfTest.java   |   78 +-
 .../closure/GridClosureSerializationTest.java   |    4 +-
 .../cluster/GridAddressResolverSelfTest.java    |    6 +-
 .../cluster/GridUpdateNotifierSelfTest.java     |   10 +-
 ...ComputeJobExecutionErrorToLogManualTest.java |   14 +-
 ...gniteComputeConfigVariationsFullApiTest.java |  617 ++-
 ...puteCustomExecutorConfigurationSelfTest.java |   85 +
 .../IgniteComputeCustomExecutorSelfTest.java    |  245 +
 .../compute/PublicThreadpoolStarvationTest.java |    4 +-
 .../continuous/GridEventConsumeSelfTest.java    |  203 +-
 .../continuous/GridMessageListenSelfTest.java   |    4 +-
 .../IgniteNoCustomEventsOnNodeStart.java        |    4 +-
 .../database/BPlusTreeFakeReuseSelfTest.java    |   63 +
 .../database/BPlusTreeReuseSelfTest.java        |  131 +
 .../processors/database/BPlusTreeSelfTest.java  | 1833 +++++++
 .../database/FreeListImplSelfTest.java          |  563 ++
 .../database/IgniteDbAbstractTest.java          |  393 ++
 .../IgniteDbClientNearCachePutGetTest.java      |   39 +
 .../database/IgniteDbDynamicCacheSelfTest.java  |  157 +
 .../IgniteDbMemoryLeakAbstractTest.java         |  272 +
 .../IgniteDbMemoryLeakLargeObjectsTest.java     |   56 +
 .../IgniteDbMemoryLeakLargePagesTest.java       |   33 +
 .../IgniteDbMemoryLeakNonTransactionalTest.java |   31 +
 .../database/IgniteDbMemoryLeakTest.java        |   46 +
 .../IgniteDbMemoryLeakWithExpirationTest.java   |   44 +
 .../database/IgniteDbMultiNodePutGetTest.java   |   33 +
 .../database/IgniteDbPutGetAbstractTest.java    | 1211 +++++
 .../database/IgniteDbSingleNodePutGetTest.java  |   33 +
 .../IgniteDbSingleNodeTinyPutGetTest.java       |  151 +
 .../database/MemoryMetricsSelfTest.java         |  348 ++
 .../database/MetadataStorageSelfTest.java       |  171 +
 .../DataStreamProcessorSelfTest.java            |   76 +-
 .../datastreamer/DataStreamerImplSelfTest.java  |   50 +-
 .../DataStreamerMultiThreadedSelfTest.java      |    6 +-
 .../DataStreamerMultinodeCreateCacheTest.java   |    4 +-
 .../datastreamer/DataStreamerTimeoutTest.java   |    4 +-
 .../DataStreamerUpdateAfterLoadTest.java        |   27 +-
 .../IgniteDataStreamerPerformanceTest.java      |    8 +-
 ...faultIgfsSecondaryFileSystemTestAdapter.java |    4 +-
 .../igfs/IgfsAbstractBaseSelfTest.java          |   64 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |   32 +-
 .../IgfsAtomicPrimaryOffheapTieredSelfTest.java |   39 -
 .../IgfsAtomicPrimaryOffheapValuesSelfTest.java |   39 -
 .../igfs/IgfsBackupFailoverSelfTest.java        |   20 +-
 ...lockMessageSystemPoolStarvationSelfTest.java |   31 +-
 ...sCachePerBlockLruEvictionPolicySelfTest.java |   25 +-
 .../processors/igfs/IgfsCacheSelfTest.java      |   26 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |    8 +-
 .../igfs/IgfsDataManagerSelfTest.java           |   40 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |   39 +-
 .../processors/igfs/IgfsFileInfoSelfTest.java   |   16 +-
 .../processors/igfs/IgfsIgniteMock.java         |   99 +-
 ...fsLocalSecondaryFileSystemProxySelfTest.java |    1 -
 ...IgfsLocalSecondaryFileSystemTestAdapter.java |    2 +-
 .../processors/igfs/IgfsMaxSizeSelfTest.java    |  122 -
 .../igfs/IgfsMetaManagerSelfTest.java           |   15 +-
 .../processors/igfs/IgfsMetricsSelfTest.java    |   22 +-
 .../internal/processors/igfs/IgfsMock.java      |   52 +-
 .../processors/igfs/IgfsModesSelfTest.java      |  185 +-
 .../processors/igfs/IgfsOneClientNodeTest.java  |   20 +-
 .../igfs/IgfsPrimaryOffheapTieredSelfTest.java  |   33 -
 .../igfs/IgfsPrimaryOffheapValuesSelfTest.java  |   33 -
 .../IgfsPrimaryOptimziedMarshallerSelfTest.java |   28 -
 .../processors/igfs/IgfsProcessorSelfTest.java  |   42 +-
 .../igfs/IgfsProcessorValidationSelfTest.java   |  341 +-
 ...gfsSecondaryFileSystemInjectionSelfTest.java |   16 +-
 .../IgfsSecondaryFileSystemTestAdapter.java     |    2 +-
 ...IpcEndpointRegistrationAbstractSelfTest.java |   21 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |   86 +-
 .../processors/igfs/IgfsStartCacheTest.java     |   34 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |   55 +-
 .../processors/igfs/IgfsTaskSelfTest.java       |   35 +-
 .../IgfsAbstractRecordResolverSelfTest.java     |   16 +-
 ...niteMessagingConfigVariationFullApiTest.java |  108 +-
 .../OdbcConfigurationValidationSelfTest.java    |  186 +
 .../odbc/OdbcEscapeSequenceSelfTest.java        |    2 +-
 .../odbc/OdbcProcessorValidationSelfTest.java   |  175 -
 .../cache/GridCacheCommandHandlerSelfTest.java  |    4 +
 .../query/GridQueryCommandHandlerTest.java      |    3 +-
 .../top/CacheTopologyCommandHandlerTest.java    |  152 +
 .../service/ClosureServiceClientsNodesTest.java |   14 +-
 .../service/GridServiceClientNodeTest.java      |   11 +-
 .../GridServiceContinuousQueryRedeployTest.java |  167 +
 .../GridServiceProcessorAbstractSelfTest.java   |  328 +-
 ...ServiceProcessorMultiNodeConfigSelfTest.java |  156 +-
 .../GridServiceProcessorMultiNodeSelfTest.java  |  196 +-
 .../GridServiceProcessorProxySelfTest.java      |   24 +-
 .../GridServiceProcessorStopSelfTest.java       |   82 +-
 ...GridServiceProxyClientReconnectSelfTest.java |    6 +-
 .../GridServiceProxyNodeStopSelfTest.java       |    4 +-
 .../GridServiceSerializationSelfTest.java       |    6 +-
 ...gniteServiceConfigVariationsFullApiTest.java |   13 +-
 ...yment2ClassLoadersDefaultMarshallerTest.java |   18 +-
 ...ent2ClassLoadersOptimizedMarshallerTest.java |    2 +-
 ...oymentClassLoadingDefaultMarshallerTest.java |   16 +-
 ...mentClassLoadingOptimizedMarshallerTest.java |    2 +-
 .../IgniteServiceDynamicCachesSelfTest.java     |   18 +-
 ...gniteServiceProxyTimeoutInitializedTest.java |    6 +-
 .../service/IgniteServiceReassignmentTest.java  |    4 +-
 .../ServicePredicateAccessCacheTest.java        |   10 +-
 .../internal/util/GridArraysSelfTest.java       |  129 +
 .../internal/util/StripedExecutorTest.java      |  168 +
 .../util/future/IgniteCacheFutureImplTest.java  |   46 +
 .../util/future/IgniteFutureImplTest.java       |  504 +-
 .../nio/GridNioEmbeddedFutureSelfTest.java      |    2 +-
 .../util/future/nio/GridNioFutureSelfTest.java  |   25 +-
 ...GridUnsafeDataOutputArraySizingSelfTest.java |    4 +-
 .../ipc/shmem/IpcSharedMemoryNodeStartup.java   |   30 +-
 .../ipc/shmem/IpcSharedMemorySpaceSelfTest.java |    6 +-
 .../ipc/shmem/IpcSharedMemoryUtilsSelfTest.java |    2 +-
 .../IpcSharedMemoryBenchmarkReader.java         |    3 +-
 .../internal/util/nio/GridNioSelfTest.java      |    2 +-
 .../internal/util/nio/GridNioSslSelfTest.java   |    2 +-
 .../nio/IgniteExceptionInNioWorkerSelfTest.java |    8 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |   12 +-
 .../offheap/GridOffHeapMapAbstractSelfTest.java |    3 +-
 .../unsafe/GridUnsafeMemorySelfTest.java        |   38 +-
 .../tostring/IncludeSensitiveAtomicTest.java    |   29 +
 .../util/tostring/IncludeSensitiveTest.java     |  149 +
 .../IncludeSensitiveTransactionalTest.java      |   43 +
 .../apache/ignite/lang/GridTupleSelfTest.java   |   35 -
 .../IgniteOffheapReadWriteLockSelfTest.java     |  487 ++
 .../loadtest/GridSingleExecutionTest.java       |   14 +-
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |  252 -
 .../loadtests/GridCacheMultiNodeLoadTest.java   |    7 +-
 .../cache/GridCacheAbstractLoadTest.java        |    9 +-
 .../cache/GridCacheDataStructuresLoadTest.java  |    2 +-
 .../loadtests/cache/GridCacheLoadTest.java      |    4 +-
 .../loadtests/cache/GridCacheSwapLoadTest.java  |  320 --
 .../GridCacheWriteBehindStoreLoadTest.java      |    1 -
 .../capacity/GridCapacityLoadTest.java          |    2 +-
 .../capacity/spring-capacity-cache.xml          |    5 +-
 .../loadtests/colocation/GridTestKey.java       |   28 +-
 .../loadtests/colocation/GridTestMain.java      |    7 +-
 .../loadtests/colocation/spring-colocation.xml  |    5 -
 .../communication/GridIoManagerBenchmark.java   |    8 +-
 .../communication/GridIoManagerBenchmark0.java  |   31 +-
 .../GridTcpCommunicationBenchmark.java          |    2 +-
 .../communication/GridTestMessage.java          |    2 +-
 .../GridContinuousOperationsLoadTest.java       |    2 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |    9 +-
 .../multisplit/GridMultiSplitsLoadTest.java     |   11 +-
 .../GridMultiSplitsRedeployLoadTest.java        |    2 +-
 ...ridSingleSplitsNewNodesAbstractLoadTest.java |   27 +-
 ...idSingleSplitsNewNodesMulticastLoadTest.java |    9 +-
 .../GridSingleSplitsRedeployLoadTest.java       |    2 +-
 .../direct/session/GridSessionLoadTest.java     |    2 +-
 .../direct/stealing/GridStealingLoadTest.java   |    4 +-
 .../loadtests/discovery/GridGcTimeoutTest.java  |    2 +-
 .../ignite/loadtests/dsi/GridDsiClient.java     |   12 +-
 .../ignite/loadtests/dsi/GridDsiPerfJob.java    |    7 +-
 .../loadtests/hashmap/GridCacheTestContext.java |   13 +-
 .../loadtests/hashmap/GridHashMapLoadTest.java  |    4 +-
 ...GridJobExecutionLoadTestClientSemaphore.java |    9 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |   10 +-
 .../ignite/loadtests/job/GridJobLoadTest.java   |    6 +-
 .../loadtests/job/GridJobLoadTestSubmitter.java |    7 +-
 .../mergesort/GridMergeSortLoadTask.java        |    7 +-
 .../loadtests/nio/GridNioBenchmarkTest.java     |    2 +-
 .../swap/GridSwapEvictAllBenchmark.java         |  309 --
 ...namicProxySerializationMultiJvmSelfTest.java |   19 +-
 .../marshaller/GridMarshallerAbstractTest.java  |   31 +-
 .../GridMarshallerPerformanceTest.java          |    4 +-
 .../marshaller/MarshallerContextSelfTest.java   |   96 +-
 .../marshaller/MarshallerContextTestImpl.java   |    1 +
 .../OptimizedMarshallerEnumSelfTest.java        |   87 -
 .../OptimizedMarshallerNodeFailoverTest.java    |  357 --
 .../OptimizedMarshallerPooledSelfTest.java      |   44 -
 .../optimized/OptimizedMarshallerSelfTest.java  |  283 --
 ...arshallerSerialPersistentFieldsSelfTest.java |  114 -
 .../optimized/OptimizedMarshallerTest.java      |  790 ---
 .../OptimizedObjectStreamSelfTest.java          | 2157 --------
 .../TestTcpDiscoveryIpFinderAdapter.java        |   43 -
 .../marshaller/optimized/package-info.java      |   22 -
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    4 +-
 .../ignite/messaging/GridMessagingSelfTest.java |  202 +-
 .../messaging/IgniteMessagingSendAsyncTest.java |  495 ++
 .../IgniteMessagingWithClientTest.java          |   10 +-
 .../p2p/GridAbstractMultinodeRedeployTest.java  |    4 +-
 .../GridP2PContinuousDeploymentSelfTest.java    |   18 +-
 .../GridP2PDifferentClassLoaderSelfTest.java    |    4 +-
 .../p2p/GridP2PDoubleDeploymentSelfTest.java    |    4 +-
 .../p2p/GridP2PHotRedeploymentSelfTest.java     |    4 +-
 .../p2p/GridP2PJobClassLoaderSelfTest.java      |    4 +-
 .../p2p/GridP2PLocalDeploymentSelfTest.java     |  113 +-
 .../GridP2PMissedResourceCacheSizeSelfTest.java |    4 +-
 .../ignite/p2p/GridP2PNodeLeftSelfTest.java     |    4 +-
 .../p2p/GridP2PRecursionTaskSelfTest.java       |    4 +-
 .../p2p/GridP2PRemoteClassLoadersSelfTest.java  |    4 +-
 .../p2p/GridP2PSameClassLoaderSelfTest.java     |    6 +-
 .../ignite/p2p/GridP2PTimeoutSelfTest.java      |    4 +-
 .../ignite/p2p/GridP2PUndeploySelfTest.java     |    6 +-
 .../platform/PlatformCacheWriteMetricsTask.java |  159 +-
 .../platform/PlatformComputeEchoTask.java       |   32 +-
 .../platform/PlatformEventsWriteEventTask.java  |   14 +-
 .../PlatformJavaObjectFactoryProxySelfTest.java |    2 +-
 .../ignite/platform/PlatformSqlQueryTask.java   |    2 +-
 .../ignite/platform/PlatformStopIgniteTask.java |   21 +-
 .../plugin/PlatformTestPluginConfiguration.java |   63 +
 ...rmTestPluginConfigurationClosureFactory.java |   61 +
 .../plugin/PlatformTestPluginException.java     |   34 +
 .../plugin/PlatformTestPluginExtension.java     |    2 +-
 .../plugin/PlatformTestPluginProvider.java      |   14 +-
 .../plugin/PlatformTestPluginTarget.java        |  151 +-
 .../cache/PlatformGetCachePluginsTask.java      |   85 +
 .../PlatformTestCachePluginConfiguration.java   |   55 +
 ...formTestCachePluginConfigurationClosure.java |   48 +
 ...tCachePluginConfigurationClosureFactory.java |   37 +
 .../cache/PlatformTestCachePluginProvider.java  |   72 +
 .../SecurityPermissionSetBuilderTest.java       |   83 +-
 .../services/ServiceThreadPoolSelfTest.java     |  133 +
 ...SessionCancelSiblingsFromFutureSelfTest.java |    6 +-
 ...ridSessionCancelSiblingsFromJobSelfTest.java |    6 +-
 ...idSessionCancelSiblingsFromTaskSelfTest.java |    6 +-
 .../GridSessionCheckpointAbstractSelfTest.java  |    2 +-
 .../GridSessionCollisionSpiSelfTest.java        |    2 +-
 ...idSessionFutureWaitJobAttributeSelfTest.java |   13 +-
 ...dSessionFutureWaitTaskAttributeSelfTest.java |    6 +-
 .../session/GridSessionJobFailoverSelfTest.java |    4 +-
 ...GridSessionJobWaitTaskAttributeSelfTest.java |    6 +-
 .../ignite/session/GridSessionLoadSelfTest.java |    4 +-
 .../GridSessionSetFutureAttributeSelfTest.java  |    6 +-
 ...nSetFutureAttributeWaitListenerSelfTest.java |    6 +-
 ...GridSessionSetJobAttributeOrderSelfTest.java |    8 +-
 .../GridSessionSetJobAttributeSelfTest.java     |    6 +-
 ...sionSetJobAttributeWaitListenerSelfTest.java |   13 +-
 .../GridSessionSetTaskAttributeSelfTest.java    |   13 +-
 ...GridSessionTaskWaitJobAttributeSelfTest.java |   13 +-
 .../GridSessionWaitAttributeSelfTest.java       |   17 +-
 .../spi/GridSpiStartStopAbstractTest.java       |    2 +-
 .../spi/GridTcpSpiForwardingSelfTest.java       |   10 +-
 .../CacheCheckpointSpiSecondCacheSelfTest.java  |   11 +-
 .../GridAbstractCommunicationSelfTest.java      |    4 +-
 .../communication/GridCacheMessageSelfTest.java |   60 +-
 .../spi/communication/GridTestMessage.java      |    4 +-
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |   10 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |    4 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |    4 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    2 +-
 .../tcp/GridTcpCommunicationSpiLanTest.java     |    2 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |    4 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |    4 +-
 ...GridTcpCommunicationSpiRecoverySelfTest.java |    4 +-
 .../tcp/IgniteCacheSslStartStopSelfTest.java    |    4 +-
 ...CommunicationRecoveryAckClosureSelfTest.java |    4 +-
 .../tcp/TcpCommunicationSpiDropNodesTest.java   |  322 ++
 .../TcpCommunicationSpiFaultyClientTest.java    |  269 +
 .../discovery/AbstractDiscoverySelfTest.java    |   36 +-
 .../discovery/AuthenticationRestartTest.java    |   88 +
 ...gniteClientReconnectMassiveShutdownTest.java |   11 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   10 +-
 ...lientDiscoverySpiFailureTimeoutSelfTest.java |  245 +-
 .../tcp/TcpClientDiscoverySpiMulticastTest.java |    4 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  147 +-
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |    4 +-
 .../TcpDiscoveryMarshallerCheckSelfTest.java    |    8 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   20 +-
 ...veryNodeAttributesUpdateOnReconnectTest.java |  110 +
 ...DiscoveryNodeConfigConsistentIdSelfTest.java |    6 +-
 .../TcpDiscoveryNodeConsistentIdSelfTest.java   |    4 +-
 .../discovery/tcp/TcpDiscoveryRestartTest.java  |    4 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   74 +-
 .../tcp/TcpDiscoverySnapshotHistoryTest.java    |    4 +-
 .../tcp/TcpDiscoverySpiConfigSelfTest.java      |    6 +-
 .../TcpDiscoverySpiFailureTimeoutSelfTest.java  |  111 +-
 .../tcp/TcpDiscoverySpiWildcardSelfTest.java    |    4 +-
 .../TcpDiscoverySslSecuredUnsecuredTest.java    |  185 +
 .../discovery/tcp/TcpDiscoverySslSelfTest.java  |    4 +-
 .../tcp/TestReconnectPluginProvider.java        |  118 +
 .../discovery/tcp/TestReconnectProcessor.java   |  136 +
 .../TcpDiscoveryIpFinderAbstractSelfTest.java   |    7 +
 .../vm/TcpDiscoveryVmIpFinderSelfTest.java      |    9 +-
 .../spi/failover/GridFailoverTestContext.java   |    7 +-
 .../GridInternalTasksLoadBalancingSelfTest.java |   34 +-
 .../GridSwapSpaceSpiAbstractSelfTest.java       |  652 ---
 .../GridSwapSpaceSpiConsistencySelfTest.java    |  131 -
 .../file/GridFileSwapCompactionSelfTest.java    |  131 -
 .../file/GridFileSwapSpaceSpiSelfTest.java      |  456 --
 .../inmemory/GridTestSwapSpaceSpi.java          |  518 --
 .../noop/GridNoopSwapSpaceSpiSelfTest.java      |   61 -
 .../ignite/spi/swapspace/package-info.java      |   22 -
 .../startup/GridRandomCommandLineLoader.java    |    2 +-
 .../ignite/startup/GridVmNodesStarter.java      |    4 +-
 .../stream/socket/SocketStreamerSelfTest.java   |   10 +-
 .../testframework/GridSpiTestContext.java       |   10 +-
 .../ignite/testframework/GridStringLogger.java  |   47 +-
 .../ignite/testframework/GridTestNode.java      |   22 +-
 .../ignite/testframework/GridTestUtils.java     |  286 +-
 .../config/GridTestProperties.java              |   12 +-
 .../configvariations/ConfigFactory.java         |    8 +-
 .../configvariations/ConfigVariations.java      |   52 +-
 .../ConfigVariationsFactory.java                |    6 +-
 .../testframework/junits/GridAbstractTest.java  |  472 +-
 .../junits/GridTestKernalContext.java           |    8 +-
 ...IgniteCacheConfigVariationsAbstractTest.java |   71 +-
 .../IgniteConfigVariationsAbstractTest.java     |   12 +-
 .../ignite/testframework/junits/IgniteMock.java |   72 +-
 .../junits/IgniteTestResources.java             |   69 +-
 .../cache/GridAbstractCacheStoreSelfTest.java   |   16 +-
 .../junits/common/GridAbstractExamplesTest.java |    2 +-
 .../junits/common/GridCommonAbstractTest.java   |  727 ++-
 .../common/GridRollingRestartAbstractTest.java  |    6 +-
 .../junits/logger/GridTestLog4jLogger.java      |    4 +-
 .../multijvm/IgniteCacheProcessProxy.java       |  225 +-
 .../multijvm/IgniteClusterProcessProxy.java     |   25 +-
 .../multijvm/IgniteEventsProcessProxy.java      |   31 +
 .../junits/multijvm/IgniteNodeRunner.java       |   56 +-
 .../junits/multijvm/IgniteProcessProxy.java     |  206 +-
 .../junits/spi/GridSpiAbstractConfigTest.java   |    2 +-
 .../junits/spi/GridSpiAbstractTest.java         |    9 +-
 .../ConfigVariationsTestSuiteBuilderTest.java   |    4 +-
 .../testframework/test/ParametersTest.java      |    7 +-
 ...naryObjectsTxDeadlockDetectionTestSuite.java |   37 -
 .../ignite/testsuites/IgniteBasicTestSuite.java |   40 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |  104 -
 .../IgniteBinaryCacheFullApiTestSuite.java      |   37 -
 .../testsuites/IgniteBinaryCacheTestSuite.java  |   26 +-
 ...ObjectsCacheDataStructuresSelfTestSuite.java |   37 -
 ...BinaryObjectsCacheExpiryPolicyTestSuite.java |   38 -
 ...gniteBinaryObjectsCacheRestartTestSuite.java |   37 -
 .../IgniteBinaryObjectsCacheTestSuite2.java     |   37 -
 .../IgniteBinaryObjectsCacheTestSuite3.java     |   15 +-
 .../IgniteBinaryObjectsCacheTestSuite4.java     |   37 -
 ...IgniteBinaryObjectsComputeGridTestSuite.java |    4 -
 .../IgniteBinaryObjectsTestSuite.java           |   29 +-
 ...iteBinarySimpleNameMapperBasicTestSuite.java |    3 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |   21 +-
 .../IgniteCacheEvictionSelfTestSuite.java       |   25 +-
 .../IgniteCacheFailoverTestSuite.java           |   10 -
 .../IgniteCacheFailoverTestSuite2.java          |    2 -
 .../IgniteCacheFailoverTestSuite3.java          |    2 -
 ...IgniteCacheFullApiMultiJvmSelfTestSuite.java |   60 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |  137 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |    9 +-
 .../IgniteCacheRestartTestSuite2.java           |    3 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   80 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   71 +-
 .../testsuites/IgniteCacheTestSuite3.java       |   24 +-
 .../testsuites/IgniteCacheTestSuite4.java       |   65 +-
 .../testsuites/IgniteCacheTestSuite5.java       |   39 +-
 .../IgniteCacheTxRecoverySelfTestSuite.java     |    5 +
 .../IgniteCacheWriteBehindTestSuite.java        |    2 +
 .../IgniteClientReconnectTestSuite.java         |    2 +
 ...teBasicConfigVariationsFullApiTestSuite.java |    2 +-
 .../testsuites/IgniteComputeGridTestSuite.java  |   12 +-
 .../testsuites/IgniteDatabaseTestSuite.java     |   40 +
 .../testsuites/IgniteDbMemoryLeakTestSuite.java |   49 +
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   17 +-
 .../testsuites/IgniteKernalSelfTestSuite.java   |   15 +-
 .../testsuites/IgniteLangSelfTestSuite.java     |    3 +
 .../IgniteMarshallerSelfTestSuite.java          |   14 +-
 ...essagingConfigVariationFullApiTestSuite.java |    2 +-
 .../IgnitePdsOutOfMemoryTestSuite.java          |   38 +
 .../ignite/testsuites/IgnitePdsTestSuite.java   |   76 +
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   74 +
 .../testsuites/IgniteRestHandlerTestSuite.java  |    2 +
 ...ServiceConfigVariationsFullApiTestSuite.java |    2 +-
 .../IgniteSpiCommunicationSelfTestSuite.java    |    5 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |    9 +-
 .../IgniteSpiSwapSpaceSelfTestSuite.java        |   44 -
 .../ignite/testsuites/IgniteSpiTestSuite.java   |    3 -
 .../testsuites/IgniteStandByClusterSuite.java   |   72 +
 .../IgniteTopologyValidatorTestSuit.java        |   43 -
 .../IgniteTopologyValidatorTestSuite.java       |   64 +
 .../testsuites/IgniteUtilSelfTestSuite.java     |   11 +
 .../TxDeadlockDetectionTestSuite.java           |    4 +
 .../util/AttributeNodeFilterSelfTest.java       |    4 +-
 .../apache/ignite/util/GridIntListSelfTest.java |  153 +
 .../ignite/util/GridMessageCollectionTest.java  |    5 +-
 .../ignite/util/GridPartitionMapSelfTest.java   |  162 +
 .../apache/ignite/util/GridRandomSelfTest.java  |   17 -
 .../util/GridTopologyHeapSizeSelfTest.java      |    4 +-
 .../webapp/META-INF/ignite-webapp-config.xml    |   28 +-
 modules/docker/1.9.0/Dockerfile                 |   46 +
 modules/docker/1.9.0/run.sh                     |   51 +
 modules/docker/2.0.0/Dockerfile                 |   46 +
 modules/docker/2.0.0/run.sh                     |   51 +
 modules/docker/Dockerfile                       |    4 +-
 modules/extdata/p2p/pom.xml                     |    8 +-
 ...CacheDeploymentCachePluginConfiguration.java |   10 +-
 .../tests/p2p/CacheDeploymentTestTask1.java     |    2 +-
 .../tests/p2p/CacheDeploymentTestTask3.java     |    2 +-
 .../p2p/GridP2PContinuousDeploymentTask1.java   |    2 +-
 .../p2p/NoValueClassOnServerAbstractClient.java |   90 +
 .../tests/p2p/P2PTestTaskExternalPath2.java     |    4 +-
 .../CacheNoValueClassOnServerTestClient.java    |   79 +-
 .../apache/ignite/tests/p2p/cache/Person.java   |   41 +
 ...DataStructuresNoClassOnServerTestClient.java |  181 +
 .../extdata/uri/modules/uri-dependency/pom.xml  |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/flink/pom.xml                           |    4 +-
 .../sink/flink/FlinkIgniteSinkSelfTest.java     |   15 +-
 modules/flume/pom.xml                           |    2 +-
 modules/gce/pom.xml                             |    2 +-
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |   33 +-
 modules/geospatial/pom.xml                      |    2 +-
 .../query/h2/opt/GridH2SpatialIndex.java        |  101 +-
 .../h2/GridBinaryH2IndexingGeoSelfTest.java     |   35 -
 .../query/h2/GridH2IndexingGeoSelfTest.java     |  285 --
 .../query/h2/H2IndexingAbstractGeoSelfTest.java |  677 +++
 .../query/h2/H2IndexingGeoSelfTest.java         |   30 +
 .../h2/H2IndexingSegmentedGeoSelfTest.java      |   30 +
 .../testsuites/GeoSpatialIndexingTestSuite.java |    9 +-
 modules/hadoop/pom.xml                          |   11 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |   23 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  698 +--
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  481 +-
 .../IgniteHadoopWeightedMapReducePlanner.java   |   26 +-
 .../org/apache/ignite/hadoop/package-info.java  |   22 -
 .../ignite/hadoop/planner/package-info.java     |   22 +
 .../hadoop/igfs/HadoopIgfsEndpoint.java         |    6 +-
 ...doopIgfsSecondaryFileSystemDelegateImpl.java |   13 +-
 .../hadoop/impl/igfs/HadoopIgfsEx.java          |    8 +
 .../hadoop/impl/igfs/HadoopIgfsFuture.java      |    3 -
 .../hadoop/impl/igfs/HadoopIgfsInProc.java      |  215 +-
 .../hadoop/impl/igfs/HadoopIgfsOutProc.java     |   12 +
 .../hadoop/impl/igfs/HadoopIgfsUtils.java       |    6 +
 .../hadoop/impl/igfs/HadoopIgfsWrapper.java     |  113 +-
 .../hadoop/impl/v1/HadoopV1ReduceTask.java      |    2 +-
 .../hadoop/impl/v2/HadoopV2Context.java         |    7 +-
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |   10 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |   20 +-
 .../hadoop/shuffle/HadoopShuffle.java           |    9 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |    7 +-
 .../shuffle/streams/HadoopOffheapBuffer.java    |    2 +-
 .../HadoopEmbeddedTaskExecutor.java             |    2 +-
 .../taskexecutor/HadoopExecutorService.java     |   10 +-
 .../external/HadoopExternalTaskExecutor.java    |    7 +-
 .../HadoopExternalCommunication.java            |   29 +-
 .../communication/HadoopIpcToNioAdapter.java    |    7 +-
 .../communication/HadoopMarshallerFilter.java   |   10 +-
 .../resources/META-INF/classnames.properties    |   17 +-
 .../test/config/hadoop-fs-open-test/grid-0.xml  |  125 +
 .../test/config/hadoop-fs-open-test/grid-1.xml  |  125 +
 .../test/config/hadoop-fs-open-test/grid-2.xml  |  125 +
 .../test/config/igfs-cli-config-dual-async.xml  |  134 +
 .../test/config/igfs-cli-config-dual-sync.xml   |  132 +
 .../src/test/config/igfs-cli-config-primary.xml |  123 +
 .../src/test/config/igfs-cli-config-proxy.xml   |  132 +
 .../impl/HadoopAbstractMapReduceTest.java       |   13 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |   36 +-
 .../hadoop/impl/HadoopGroupingTest.java         |    4 +-
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |    4 +-
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |    4 +-
 .../impl/HadoopNoHadoopMapReduceTest.java       |    4 +-
 .../hadoop/impl/HadoopSortingExternalTest.java  |    8 +-
 .../hadoop/impl/HadoopSortingTest.java          |    4 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |   22 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |    4 +-
 .../hadoop/impl/HadoopValidationSelfTest.java   |    4 +-
 .../HadoopClientProtocolEmbeddedSelfTest.java   |    4 +-
 ...opClientProtocolMultipleServersSelfTest.java |   14 +-
 .../client/HadoopClientProtocolSelfTest.java    |    9 +-
 .../impl/igfs/Hadoop1DualAbstractTest.java      |    6 +-
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   37 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   78 +-
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |    4 +-
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |    4 +-
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   15 +-
 ...adoopIgfsSecondaryFileSystemTestAdapter.java |    2 +-
 ...oopSecondaryFileSystemConfigurationTest.java |   91 +-
 .../hadoop/impl/igfs/IgfsEventsTestSuite.java   |    2 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |   14 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  220 +-
 ...opFileSystemClientBasedAbstractSelfTest.java |  193 +
 ...pFileSystemClientBasedDualAsyncSelfTest.java |   38 +
 ...opFileSystemClientBasedDualSyncSelfTest.java |   38 +
 ...niteHadoopFileSystemClientBasedOpenTest.java |  305 ++
 ...oopFileSystemClientBasedPrimarySelfTest.java |   38 +
 ...adoopFileSystemClientBasedProxySelfTest.java |   37 +
 .../IgniteHadoopFileSystemClientSelfTest.java   |   60 +-
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |  160 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   53 +-
 .../IgniteHadoopFileSystemLoggerSelfTest.java   |   32 +-
 ...niteHadoopFileSystemLoggerStateSelfTest.java |   24 +-
 ...adoopFileSystemLoopbackAbstractSelfTest.java |    4 +-
 ...oopbackExternalToClientAbstractSelfTest.java |   61 +
 ...opbackExternalToClientDualAsyncSelfTest.java |   33 +
 ...oopbackExternalToClientDualSyncSelfTest.java |   33 +
 ...LoopbackExternalToClientPrimarySelfTest.java |   33 +
 ...emLoopbackExternalToClientProxySelfTest.java |   33 +
 ...condaryFileSystemInitializationSelfTest.java |  217 -
 ...teHadoopFileSystemShmemAbstractSelfTest.java |    4 +-
 ...ileSystemShmemEmbeddedDualAsyncSelfTest.java |   33 -
 ...FileSystemShmemEmbeddedDualSyncSelfTest.java |   33 -
 ...pFileSystemShmemEmbeddedPrimarySelfTest.java |   33 -
 ...ileSystemShmemEmbeddedSecondarySelfTest.java |   33 -
 ...emShmemExternalToClientAbstractSelfTest.java |  106 +
 ...mShmemExternalToClientDualAsyncSelfTest.java |   33 +
 ...emShmemExternalToClientDualSyncSelfTest.java |   33 +
 ...temShmemExternalToClientPrimarySelfTest.java |   33 +
 ...ystemShmemExternalToClientProxySelfTest.java |   33 +
 .../collections/HadoopSkipListSelfTest.java     |   37 -
 .../taskexecutor/HadoopExecutorServiceTest.java |    3 +-
 .../HadoopExternalTaskExecutionSelfTest.java    |   16 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   24 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   17 +-
 modules/hibernate-4.2/README.txt                |   48 +
 modules/hibernate-4.2/licenses/apache-2.0.txt   |  202 +
 modules/hibernate-4.2/pom.xml                   |  159 +
 .../HibernateAbstractRegionAccessStrategy.java  |  102 +
 .../hibernate/HibernateCollectionRegion.java    |  100 +
 .../cache/hibernate/HibernateEntityRegion.java  |  112 +
 .../hibernate/HibernateGeneralDataRegion.java   |   76 +
 .../cache/hibernate/HibernateKeyWrapper.java    |   80 +
 .../hibernate/HibernateNaturalIdRegion.java     |  103 +
 .../hibernate/HibernateQueryResultsRegion.java  |   70 +
 .../ignite/cache/hibernate/HibernateRegion.java |   99 +
 .../cache/hibernate/HibernateRegionFactory.java |  179 +
 .../hibernate/HibernateTimestampsRegion.java    |   39 +
 .../HibernateTransactionalDataRegion.java       |   84 +
 .../ignite/cache/hibernate/package-info.java    |   24 +
 .../hibernate/CacheHibernateBlobStore.java      |  542 ++
 .../CacheHibernateBlobStoreEntry.hbm.xml        |   31 +
 .../hibernate/CacheHibernateBlobStoreEntry.java |   89 +
 .../CacheHibernateBlobStoreFactory.java         |  235 +
 .../CacheHibernateStoreSessionListener.java     |  222 +
 .../cache/store/hibernate/package-info.java     |   22 +
 .../src/test/config/factory-cache.xml           |   59 +
 .../src/test/config/factory-cache1.xml          |   61 +
 .../config/factory-incorrect-store-cache.xml    |   56 +
 .../HibernateL2CacheConfigurationSelfTest.java  |  409 ++
 .../hibernate/HibernateL2CacheMultiJvmTest.java |  440 ++
 .../hibernate/HibernateL2CacheSelfTest.java     | 1953 +++++++
 .../HibernateL2CacheStrategySelfTest.java       |  597 +++
 .../HibernateL2CacheTransactionalSelfTest.java  |  154 +
 ...nateL2CacheTransactionalUseSyncSelfTest.java |   31 +
 .../CacheHibernateBlobStoreNodeRestartTest.java |   46 +
 .../CacheHibernateBlobStoreSelfTest.java        |  113 +
 .../CacheHibernateStoreFactorySelfTest.java     |  288 ++
 ...heHibernateStoreSessionListenerSelfTest.java |  238 +
 .../cache/store/hibernate/hibernate.cfg.xml     |   42 +
 .../cache/store/hibernate/package-info.java     |   22 +
 .../IgniteBinaryHibernateTestSuite.java         |   37 +
 .../testsuites/IgniteHibernateTestSuite.java    |   59 +
 modules/hibernate-5.1/README.txt                |   48 +
 modules/hibernate-5.1/licenses/apache-2.0.txt   |  202 +
 modules/hibernate-5.1/pom.xml                   |  159 +
 .../HibernateAbstractRegionAccessStrategy.java  |  103 +
 .../hibernate/HibernateCollectionRegion.java    |  114 +
 .../cache/hibernate/HibernateEntityRegion.java  |  128 +
 .../hibernate/HibernateGeneralDataRegion.java   |   79 +
 .../cache/hibernate/HibernateKeyWrapper.java    |  109 +
 .../hibernate/HibernateNaturalIdRegion.java     |  113 +
 .../hibernate/HibernateQueryResultsRegion.java  |   70 +
 .../ignite/cache/hibernate/HibernateRegion.java |   99 +
 .../cache/hibernate/HibernateRegionFactory.java |  168 +
 .../hibernate/HibernateTimestampsRegion.java    |   39 +
 .../HibernateTransactionalDataRegion.java       |   84 +
 .../ignite/cache/hibernate/package-info.java    |   24 +
 .../hibernate/CacheHibernateBlobStore.java      |  543 ++
 .../CacheHibernateBlobStoreEntry.hbm.xml        |   31 +
 .../hibernate/CacheHibernateBlobStoreEntry.java |   89 +
 .../CacheHibernateBlobStoreFactory.java         |  235 +
 .../CacheHibernateStoreSessionListener.java     |  224 +
 .../cache/store/hibernate/package-info.java     |   22 +
 .../src/test/config/factory-cache.xml           |   59 +
 .../src/test/config/factory-cache1.xml          |   61 +
 .../config/factory-incorrect-store-cache.xml    |   56 +
 .../HibernateL2CacheConfigurationSelfTest.java  |  407 ++
 .../hibernate/HibernateL2CacheMultiJvmTest.java |  429 ++
 .../hibernate/HibernateL2CacheSelfTest.java     | 1959 +++++++
 .../HibernateL2CacheStrategySelfTest.java       |  569 +++
 .../HibernateL2CacheTransactionalSelfTest.java  |  154 +
 ...nateL2CacheTransactionalUseSyncSelfTest.java |   31 +
 .../CacheHibernateBlobStoreNodeRestartTest.java |   46 +
 .../CacheHibernateBlobStoreSelfTest.java        |  114 +
 .../CacheHibernateStoreFactorySelfTest.java     |  256 +
 ...heHibernateStoreSessionListenerSelfTest.java |  242 +
 .../cache/store/hibernate/hibernate.cfg.xml     |   42 +
 .../cache/store/hibernate/package-info.java     |   22 +
 .../IgniteBinaryHibernate5TestSuite.java        |   37 +
 .../testsuites/IgniteHibernate5TestSuite.java   |   59 +
 modules/hibernate-core/pom.xml                  |   76 +
 .../HibernateAccessStrategyAdapter.java         |  340 ++
 .../HibernateAccessStrategyFactory.java         |  235 +
 .../cache/hibernate/HibernateCacheProxy.java    |  776 +++
 .../hibernate/HibernateExceptionConverter.java  |   29 +
 .../hibernate/HibernateKeyTransformer.java      |   29 +
 .../HibernateNonStrictAccessStrategy.java       |  230 +
 .../HibernateReadOnlyAccessStrategy.java        |  105 +
 .../HibernateReadWriteAccessStrategy.java       |  326 ++
 .../HibernateTransactionalAccessStrategy.java   |  141 +
 .../ignite/cache/hibernate/package-info.java    |   24 +
 modules/hibernate/README.txt                    |   48 -
 modules/hibernate/licenses/apache-2.0.txt       |  202 -
 modules/hibernate/pom.xml                       |  146 -
 .../HibernateAbstractRegionAccessStrategy.java  |   98 -
 .../HibernateAccessStrategyAdapter.java         |  377 --
 .../hibernate/HibernateCollectionRegion.java    |  101 -
 .../cache/hibernate/HibernateEntityRegion.java  |  113 -
 .../hibernate/HibernateGeneralDataRegion.java   |   72 -
 .../hibernate/HibernateNaturalIdRegion.java     |  101 -
 .../HibernateNonStrictAccessStrategy.java       |  223 -
 .../hibernate/HibernateQueryResultsRegion.java  |   71 -
 .../HibernateReadOnlyAccessStrategy.java        |  108 -
 .../HibernateReadWriteAccessStrategy.java       |  329 --
 .../ignite/cache/hibernate/HibernateRegion.java |  101 -
 .../cache/hibernate/HibernateRegionFactory.java |  233 -
 .../hibernate/HibernateTimestampsRegion.java    |   40 -
 .../HibernateTransactionalAccessStrategy.java   |  141 -
 .../HibernateTransactionalDataRegion.java       |  108 -
 .../ignite/cache/hibernate/package-info.java    |   24 -
 .../hibernate/CacheHibernateBlobStore.java      |  542 --
 .../CacheHibernateBlobStoreEntry.hbm.xml        |   31 -
 .../hibernate/CacheHibernateBlobStoreEntry.java |   89 -
 .../CacheHibernateBlobStoreFactory.java         |  235 -
 .../CacheHibernateStoreSessionListener.java     |  222 -
 .../cache/store/hibernate/package-info.java     |   22 -
 .../hibernate/src/test/config/factory-cache.xml |   59 -
 .../src/test/config/factory-cache1.xml          |   61 -
 .../config/factory-incorrect-store-cache.xml    |   56 -
 .../HibernateL2CacheConfigurationSelfTest.java  |  408 --
 .../hibernate/HibernateL2CacheSelfTest.java     | 1949 -------
 .../HibernateL2CacheTransactionalSelfTest.java  |  154 -
 ...nateL2CacheTransactionalUseSyncSelfTest.java |   31 -
 .../CacheHibernateBlobStoreNodeRestartTest.java |   54 -
 .../CacheHibernateBlobStoreSelfTest.java        |  113 -
 .../CacheHibernateStoreFactorySelfTest.java     |  285 --
 ...heHibernateStoreSessionListenerSelfTest.java |  238 -
 .../cache/store/hibernate/hibernate.cfg.xml     |   42 -
 .../cache/store/hibernate/package-info.java     |   22 -
 .../testsuites/IgniteHibernateTestSuite.java    |   57 -
 modules/indexing/pom.xml                        |    9 +-
 .../cache/query/CacheQueryPartitionInfo.java    |  110 +
 .../cache/query/GridCacheTwoStepQuery.java      |  108 +-
 .../processors/cache/query/QueryTable.java      |  164 +
 .../query/h2/DmlStatementsProcessor.java        |  454 +-
 .../query/h2/GridH2ResultSetIterator.java       |  191 -
 .../query/h2/H2ConnectionWrapper.java           |   67 +
 .../internal/processors/query/h2/H2Cursor.java  |  108 +
 .../processors/query/h2/H2DatabaseType.java     |  161 +
 .../processors/query/h2/H2DmlPlanKey.java       |   66 +
 .../processors/query/h2/H2FieldsIterator.java   |   50 +
 .../processors/query/h2/H2KeyValueIterator.java |   48 +
 .../query/h2/H2ResultSetIterator.java           |  189 +
 .../processors/query/h2/H2RowDescriptor.java    |  489 ++
 .../internal/processors/query/h2/H2Schema.java  |  130 +
 .../processors/query/h2/H2SqlFieldMetadata.java |  111 +
 .../processors/query/h2/H2StatementCache.java   |   73 +
 .../processors/query/h2/H2TableDescriptor.java  |  350 ++
 .../processors/query/h2/H2TableEngine.java      |   89 +
 .../query/h2/H2TwoStepCachedQuery.java          |   63 +
 .../query/h2/H2TwoStepCachedQueryKey.java       |  100 +
 .../internal/processors/query/h2/H2Utils.java   |  244 +
 .../processors/query/h2/IgniteH2Indexing.java   | 2663 ++++------
 .../query/h2/database/H2PkHashIndex.java        |  325 ++
 .../query/h2/database/H2RowFactory.java         |   80 +
 .../processors/query/h2/database/H2Tree.java    |  250 +
 .../query/h2/database/H2TreeIndex.java          |  434 ++
 .../query/h2/database/InlineIndexHelper.java    |  558 ++
 .../query/h2/database/io/H2ExtrasInnerIO.java   |  140 +
 .../query/h2/database/io/H2ExtrasLeafIO.java    |  135 +
 .../query/h2/database/io/H2InnerIO.java         |   74 +
 .../query/h2/database/io/H2LeafIO.java          |   74 +
 .../query/h2/database/io/H2RowLinkIO.java       |   30 +
 .../query/h2/database/util/CompareUtils.java    |  332 ++
 .../query/h2/ddl/DdlStatementsProcessor.java    |  316 ++
 .../query/h2/dml/FastUpdateArguments.java       |    4 +-
 .../processors/query/h2/dml/UpdatePlan.java     |   33 +-
 .../query/h2/dml/UpdatePlanBuilder.java         |   65 +-
 .../query/h2/opt/DistributedJoinMode.java       |   51 +
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  171 +-
 .../query/h2/opt/GridH2CollocationModel.java    |   86 +-
 .../query/h2/opt/GridH2IndexBase.java           |  592 ++-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   78 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |   14 +-
 .../query/h2/opt/GridH2MetaTable.java           |   13 +-
 .../query/h2/opt/GridH2PrimaryScanIndex.java    |   90 +
 .../query/h2/opt/GridH2ProxyIndex.java          |  204 +
 .../query/h2/opt/GridH2ProxySpatialIndex.java   |   70 +
 .../query/h2/opt/GridH2QueryContext.java        |   84 +-
 .../processors/query/h2/opt/GridH2Row.java      |   66 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |   89 +-
 .../query/h2/opt/GridH2RowFactory.java          |   15 +
 .../query/h2/opt/GridH2ScanIndex.java           |  280 +
 .../query/h2/opt/GridH2SystemIndexFactory.java  |   38 +
 .../processors/query/h2/opt/GridH2Table.java    | 1034 ++--
 .../query/h2/opt/GridH2TreeIndex.java           |  330 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    |   32 +-
 .../query/h2/opt/GridLuceneIndex.java           |   36 +-
 .../processors/query/h2/sql/DmlAstUtils.java    |   93 +-
 .../processors/query/h2/sql/GridSqlAlias.java   |   31 +-
 .../processors/query/h2/sql/GridSqlArray.java   |    8 +-
 .../processors/query/h2/sql/GridSqlAst.java     |   61 +
 .../processors/query/h2/sql/GridSqlColumn.java  |   85 +-
 .../processors/query/h2/sql/GridSqlConst.java   |    6 +-
 .../query/h2/sql/GridSqlCreateIndex.java        |  121 +
 .../query/h2/sql/GridSqlCreateTable.java        |  222 +
 .../query/h2/sql/GridSqlDropIndex.java          |   82 +
 .../query/h2/sql/GridSqlDropTable.java          |   79 +
 .../processors/query/h2/sql/GridSqlElement.java |   43 +-
 .../query/h2/sql/GridSqlFunction.java           |   18 +-
 .../processors/query/h2/sql/GridSqlJoin.java    |   42 +-
 .../processors/query/h2/sql/GridSqlKeyword.java |    3 +-
 .../query/h2/sql/GridSqlOperation.java          |    6 +-
 .../query/h2/sql/GridSqlOperationType.java      |   25 +-
 .../query/h2/sql/GridSqlParameter.java          |    4 +-
 .../query/h2/sql/GridSqlPlaceholder.java        |    2 +-
 .../processors/query/h2/sql/GridSqlQuery.java   |   91 +-
 .../query/h2/sql/GridSqlQueryParser.java        |  743 ++-
 .../query/h2/sql/GridSqlQuerySplitter.java      | 2217 ++++++--
 .../processors/query/h2/sql/GridSqlSelect.java  |  121 +-
 .../query/h2/sql/GridSqlSortColumn.java         |   41 +
 .../query/h2/sql/GridSqlStatement.java          |   11 +-
 .../query/h2/sql/GridSqlSubquery.java           |   31 +-
 .../processors/query/h2/sql/GridSqlTable.java   |   57 +-
 .../processors/query/h2/sql/GridSqlType.java    |    6 +-
 .../processors/query/h2/sql/GridSqlUnion.java   |   66 +-
 .../processors/query/h2/sql/GridSqlValue.java   |   25 -
 .../query/h2/twostep/GridMapQueryExecutor.java  |  475 +-
 .../query/h2/twostep/GridMergeIndex.java        |  648 ++-
 .../query/h2/twostep/GridMergeIndexSorted.java  |  385 ++
 .../h2/twostep/GridMergeIndexUnsorted.java      |  103 +-
 .../query/h2/twostep/GridMergeTable.java        |   78 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  858 ++--
 .../query/h2/twostep/GridResultPage.java        |   34 +-
 .../query/h2/twostep/GridThreadLocalTable.java  |   14 +-
 .../query/h2/twostep/ReduceQueryRun.java        |  157 +
 .../query/h2/twostep/msg/GridH2Array.java       |    2 +-
 .../query/h2/twostep/msg/GridH2Boolean.java     |    2 +-
 .../query/h2/twostep/msg/GridH2Byte.java        |    2 +-
 .../query/h2/twostep/msg/GridH2Bytes.java       |    2 +-
 .../query/h2/twostep/msg/GridH2CacheObject.java |   41 +-
 .../query/h2/twostep/msg/GridH2Date.java        |    2 +-
 .../query/h2/twostep/msg/GridH2Decimal.java     |    2 +-
 .../query/h2/twostep/msg/GridH2Double.java      |    2 +-
 .../query/h2/twostep/msg/GridH2Float.java       |    2 +-
 .../query/h2/twostep/msg/GridH2Geometry.java    |    2 +-
 .../h2/twostep/msg/GridH2IndexRangeRequest.java |   64 +-
 .../twostep/msg/GridH2IndexRangeResponse.java   |   64 +-
 .../query/h2/twostep/msg/GridH2Integer.java     |    2 +-
 .../query/h2/twostep/msg/GridH2JavaObject.java  |    2 +-
 .../query/h2/twostep/msg/GridH2Long.java        |    2 +-
 .../query/h2/twostep/msg/GridH2Null.java        |    2 +-
 .../h2/twostep/msg/GridH2QueryRequest.java      |  236 +-
 .../query/h2/twostep/msg/GridH2RowMessage.java  |    2 +-
 .../query/h2/twostep/msg/GridH2RowRange.java    |    2 +-
 .../h2/twostep/msg/GridH2RowRangeBounds.java    |    2 +-
 .../query/h2/twostep/msg/GridH2Short.java       |    2 +-
 .../query/h2/twostep/msg/GridH2String.java      |    2 +-
 .../query/h2/twostep/msg/GridH2Time.java        |    2 +-
 .../query/h2/twostep/msg/GridH2Timestamp.java   |    2 +-
 .../query/h2/twostep/msg/GridH2Uuid.java        |    2 +-
 .../twostep/msg/GridH2ValueMessageFactory.java  |    6 +-
 .../cache/BinarySerializationQuerySelfTest.java |   35 +-
 ...CacheAbstractQueryDetailMetricsSelfTest.java |    4 +-
 .../CacheAbstractQueryMetricsSelfTest.java      |    4 +-
 .../CacheBinaryKeyConcurrentQueryTest.java      |    6 +-
 .../cache/CacheIndexStreamerTest.java           |   14 +-
 .../cache/CacheIndexingOffheapCleanupTest.java  |  178 -
 .../cache/CacheIteratorScanQueryTest.java       |  110 +
 .../CacheOffheapBatchIndexingBaseTest.java      |   12 +-
 .../CacheOffheapBatchIndexingMultiTypeTest.java |   17 +-
 ...CacheOffheapBatchIndexingSingleTypeTest.java |   20 +-
 .../CacheOperationsWithExpirationTest.java      |   73 +-
 .../cache/CacheQueryBuildValueTest.java         |    8 +-
 .../cache/CacheQueryEvictDataLostTest.java      |  120 +
 .../cache/CacheQueryFilterExpiredTest.java      |  111 +
 .../cache/CacheQueryNewClientSelfTest.java      |    4 +-
 .../CacheQueryOffheapEvictDataLostTest.java     |  138 -
 .../CacheRandomOperationsMultithreadedTest.java |   72 +-
 ...CacheScanPartitionQueryFallbackSelfTest.java |   20 +-
 .../cache/CacheSqlQueryValueCopySelfTest.java   |  230 +-
 .../ClientReconnectAfterClusterRestartTest.java |   22 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   54 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |  570 ---
 .../cache/GridCacheOffHeapSelfTest.java         |  498 +-
 .../GridCacheOffheapIndexEntryEvictTest.java    |   15 +-
 .../cache/GridCacheOffheapIndexGetSelfTest.java |   42 +-
 .../GridCacheQueryIndexDisabledSelfTest.java    |   24 +-
 .../GridCacheQuerySerializationSelfTest.java    |    4 +-
 .../cache/GridCacheQuerySimpleBenchmark.java    |   10 +-
 .../cache/GridCacheQueryTestValue.java          |    2 +-
 .../processors/cache/GridCacheSwapSelfTest.java |  724 ---
 .../cache/GridIndexingWithNoopSwapSelfTest.java |   13 +-
 .../IgniteBinaryObjectFieldsQuerySelfTest.java  |   16 +-
 ...aryObjectQueryArgumentsOffheapLocalTest.java |   28 -
 ...teBinaryObjectQueryArgumentsOffheapTest.java |   30 -
 .../IgniteBinaryObjectQueryArgumentsTest.java   |   23 +-
 ...eBinaryWrappedObjectFieldsQuerySelfTest.java |    3 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  542 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |  180 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  638 ++-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |    6 +-
 .../IgniteCacheCollocatedQuerySelfTest.java     |   11 +-
 ...acheConfigurationPrimitiveTypesSelfTest.java |   54 +-
 .../IgniteCacheCrossCacheJoinRandomTest.java    |   10 +-
 .../IgniteCacheDeleteSqlQuerySelfTest.java      |    6 +-
 ...acheDistributedJoinCollocatedAndNotTest.java |    8 +-
 ...acheDistributedJoinCustomAffinityMapper.java |    8 +-
 .../IgniteCacheDistributedJoinNoIndexTest.java  |   12 +-
 ...ributedJoinPartitionedAndReplicatedTest.java |    8 +-
 ...CacheDistributedJoinQueryConditionsTest.java |   24 +-
 .../cache/IgniteCacheDistributedJoinTest.java   |   16 +-
 ...cheDuplicateEntityConfigurationSelfTest.java |    4 +-
 .../IgniteCacheFieldsQueryNoDataSelfTest.java   |    6 +-
 ...teCacheFullTextQueryNodeJoiningSelfTest.java |   10 +-
 .../cache/IgniteCacheGroupsSqlTest.java         |  312 ++
 .../IgniteCacheInsertSqlQuerySelfTest.java      |   71 +-
 ...PartitionedAndReplicatedCollocationTest.java |   13 +-
 ...teCacheJoinPartitionedAndReplicatedTest.java |   67 +-
 ...IgniteCacheJoinQueryWithAffinityKeyTest.java |   26 +-
 .../cache/IgniteCacheLargeResultSelfTest.java   |    9 +-
 ...eLockPartitionOnAffinityRunAbstractTest.java |    8 +-
 ...PartitionOnAffinityRunAtomicCacheOpTest.java |   50 +-
 ...niteCacheLockPartitionOnAffinityRunTest.java |   41 +-
 ...titionOnAffinityRunWithCollisionSpiTest.java |    6 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |   34 -
 .../IgniteCacheMultipleIndexedTypesTest.java    |  200 +
 .../cache/IgniteCacheNoClassQuerySelfTest.java  |   35 +-
 .../IgniteCacheObjectKeyIndexingSelfTest.java   |  124 +
 .../cache/IgniteCacheOffheapEvictQueryTest.java |   16 +-
 .../cache/IgniteCacheOffheapIndexScanTest.java  |   11 +-
 ...CacheOffheapTieredMultithreadedSelfTest.java |  302 --
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    4 +-
 ...hePartitionedQueryMultiThreadedSelfTest.java |    8 +-
 ...IgniteCachePrimitiveFieldsQuerySelfTest.java |    4 +-
 .../cache/IgniteCacheQueriesLoadTest1.java      |    8 +-
 .../IgniteCacheQueryH2IndexingLeakTest.java     |    8 +-
 .../cache/IgniteCacheQueryIndexSelfTest.java    |    4 +-
 .../cache/IgniteCacheQueryLoadSelfTest.java     |   42 +-
 ...QueryMultiThreadedOffHeapTieredSelfTest.java |   38 -
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |  156 +-
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   28 -
 ...eCacheQueryOffheapMultiThreadedSelfTest.java |   28 -
 ...gniteCacheSqlQueryMultiThreadedSelfTest.java |   10 +-
 .../IgniteCacheStarvationOnRebalanceTest.java   |    6 +-
 .../cache/IgniteCacheUnionDuplicatesTest.java   |   21 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |    9 +-
 ...ClientReconnectCacheQueriesFailoverTest.java |   16 +-
 .../cache/IgniteClientReconnectQueriesTest.java |    4 +-
 .../cache/IgniteCrossCachesJoinsQueryTest.java  |  108 +-
 ...dexingAndGroupPutGetPersistenceSelfTest.java |   42 +
 ...leNodeWithIndexingPutGetPersistenceTest.java |   55 +
 .../cache/IncorrectQueryEntityTest.java         |   82 +
 .../cache/QueryEntityCaseMismatchTest.java      |  107 +
 .../cache/SqlFieldsQuerySelfTest.java           |   37 +-
 ...lientQueryReplicatedNodeRestartSelfTest.java |   10 +-
 ...stributedPartitionQueryAbstractSelfTest.java |  652 +++
 ...utedPartitionQueryConfigurationSelfTest.java |   92 +
 ...butedPartitionQueryNodeRestartsSelfTest.java |  112 +
 ...eCacheDistributedPartitionQuerySelfTest.java |   90 +
 ...niteCacheDistributedQueryCancelSelfTest.java |   12 +-
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java |   12 +-
 ...artitionedFieldsQueryP2PEnabledSelfTest.java |    4 +-
 ...niteCachePartitionedFieldsQuerySelfTest.java |   13 +-
 ...achePartitionedQueryP2PDisabledSelfTest.java |    4 +-
 .../IgniteCachePartitionedQuerySelfTest.java    |   91 +-
 ...cheQueryAbstractDistributedJoinSelfTest.java |    6 +-
 .../IgniteCacheQueryNoRebalanceSelfTest.java    |    4 +-
 .../near/IgniteCacheQueryNodeFailTest.java      |    8 +-
 ...QueryNodeRestartDistributedJoinSelfTest.java |  165 +-
 .../IgniteCacheQueryNodeRestartSelfTest.java    |    9 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   11 +-
 ...ReplicatedFieldsQueryP2PEnabledSelfTest.java |    4 +-
 ...iteCacheReplicatedFieldsQueryROSelfTest.java |   27 +
 ...gniteCacheReplicatedFieldsQuerySelfTest.java |    6 +-
 ...CacheReplicatedQueryP2PDisabledSelfTest.java |    4 +-
 .../IgniteCacheReplicatedQuerySelfTest.java     |   25 +-
 .../cache/index/AbstractSchemaSelfTest.java     |  649 +++
 .../index/DuplicateKeyValueClassesSelfTest.java |   94 +
 .../DynamicIndexAbstractBasicSelfTest.java      | 1176 +++++
 .../DynamicIndexAbstractConcurrentSelfTest.java | 1076 ++++
 .../index/DynamicIndexAbstractSelfTest.java     |  464 ++
 .../index/DynamicIndexClientBasicSelfTest.java  |   28 +
 ...ndexPartitionedAtomicConcurrentSelfTest.java |   33 +
 ...titionedTransactionalConcurrentSelfTest.java |   33 +
 ...IndexReplicatedAtomicConcurrentSelfTest.java |   33 +
 ...plicatedTransactionalConcurrentSelfTest.java |   33 +
 .../index/DynamicIndexServerBasicSelfTest.java  |   28 +
 ...amicIndexServerCoordinatorBasicSelfTest.java |   28 +
 ...namicIndexServerNodeFIlterBasicSelfTest.java |   28 +
 ...erverNodeFilterCoordinatorBasicSelfTest.java |   30 +
 .../index/H2DynamicIndexAbstractSelfTest.java   |  400 ++
 ...namicIndexAtomicPartitionedNearSelfTest.java |   26 +
 ...H2DynamicIndexAtomicPartitionedSelfTest.java |   39 +
 .../H2DynamicIndexAtomicReplicatedSelfTest.java |   39 +
 ...dexTransactionalPartitionedNearSelfTest.java |   26 +
 ...icIndexTransactionalPartitionedSelfTest.java |   39 +
 ...micIndexTransactionalReplicatedSelfTest.java |   39 +
 ...exingComplexClientAtomicPartitionedTest.java |   33 +
 ...dexingComplexClientAtomicReplicatedTest.java |   33 +
 ...mplexClientTransactionalPartitionedTest.java |   33 +
 ...omplexClientTransactionalReplicatedTest.java |   33 +
 ...exingComplexServerAtomicPartitionedTest.java |   33 +
 ...dexingComplexServerAtomicReplicatedTest.java |   33 +
 ...mplexServerTransactionalPartitionedTest.java |   33 +
 ...omplexServerTransactionalReplicatedTest.java |   33 +
 .../index/H2DynamicIndexingComplexTest.java     |  356 ++
 .../cache/index/H2DynamicTableSelfTest.java     |  929 ++++
 .../index/QueryEntityValidationSelfTest.java    |  162 +
 .../cache/index/SchemaExchangeSelfTest.java     |  639 +++
 .../IgniteCacheLocalFieldsQuerySelfTest.java    |    4 +-
 ...eCacheLocalQueryCancelOrTimeoutSelfTest.java |   10 +-
 .../local/IgniteCacheLocalQuerySelfTest.java    |    4 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |   41 +-
 .../ttl/CacheTtlAtomicAbstractSelfTest.java     |   29 +
 .../cache/ttl/CacheTtlAtomicLocalSelfTest.java  |   34 +
 .../ttl/CacheTtlAtomicPartitionedSelfTest.java  |   34 +
 .../ttl/CacheTtlOffheapAbstractSelfTest.java    |   29 -
 .../CacheTtlOffheapAtomicAbstractSelfTest.java  |   29 -
 .../ttl/CacheTtlOffheapAtomicLocalSelfTest.java |   34 -
 ...acheTtlOffheapAtomicPartitionedSelfTest.java |   34 -
 ...TtlOffheapTransactionalAbstractSelfTest.java |   29 -
 ...cheTtlOffheapTransactionalLocalSelfTest.java |   34 -
 ...OffheapTransactionalPartitionedSelfTest.java |   34 -
 .../ttl/CacheTtlOnheapAbstractSelfTest.java     |   29 -
 .../CacheTtlOnheapAtomicAbstractSelfTest.java   |   29 -
 .../ttl/CacheTtlOnheapAtomicLocalSelfTest.java  |   34 -
 ...CacheTtlOnheapAtomicPartitionedSelfTest.java |   34 -
 ...eTtlOnheapTransactionalAbstractSelfTest.java |   29 -
 ...acheTtlOnheapTransactionalLocalSelfTest.java |   34 -
 ...lOnheapTransactionalPartitionedSelfTest.java |   34 -
 .../CacheTtlTransactionalAbstractSelfTest.java  |   29 +
 .../ttl/CacheTtlTransactionalLocalSelfTest.java |   34 +
 ...acheTtlTransactionalPartitionedSelfTest.java |   34 +
 .../database/IgniteDbMemoryLeakIndexedTest.java |   33 +
 .../IgniteDbMemoryLeakSqlQueryTest.java         |   76 +
 ...IgniteDbMultiNodeWithIndexingPutGetTest.java |   28 +
 ...gniteDbSingleNodeWithIndexingPutGetTest.java |  249 +
 .../IgnitePersistentStoreSchemaLoadTest.java    |  297 ++
 .../IgniteCacheGroupsCompareQueryTest.java      |   45 +
 ...teCacheGroupsSqlDistributedJoinSelfTest.java |   30 +
 ...roupsSqlSegmentedIndexMultiNodeSelfTest.java |   28 +
 ...iteCacheGroupsSqlSegmentedIndexSelfTest.java |   37 +
 .../query/IgniteQueryDedicatedPoolTest.java     |  212 +
 .../query/IgniteSqlDistributedJoinSelfTest.java |  199 +
 .../query/IgniteSqlKeyValueFieldsTest.java      |  392 ++
 .../query/IgniteSqlQueryDedicatedPoolTest.java  |  110 -
 .../processors/query/IgniteSqlRoutingTest.java  |  552 ++
 .../query/IgniteSqlSchemaIndexingTest.java      |   42 +-
 ...gniteSqlSegmentedIndexMultiNodeSelfTest.java |   28 +
 .../query/IgniteSqlSegmentedIndexSelfTest.java  |  329 ++
 .../query/IgniteSqlSplitterSelfTest.java        | 1250 ++++-
 .../processors/query/SqlSchemaSelfTest.java     |  263 +
 .../query/h2/GridH2IndexRebuildTest.java        |  261 -
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  476 +-
 .../query/h2/IgniteSqlQueryMinMaxTest.java      |  376 ++
 .../h2/database/InlineIndexHelperTest.java      |  357 ++
 .../query/h2/opt/GridH2TableSelfTest.java       |  278 +-
 .../h2/sql/AbstractH2CompareQueryTest.java      |  104 +-
 .../query/h2/sql/BaseH2CompareQueryTest.java    |  369 +-
 .../query/h2/sql/GridQueryParsingTest.java      |  744 ++-
 .../query/h2/sql/H2CompareBigQueryTest.java     |  156 +-
 ...onnectorConfigurationValidationSelfTest.java |  240 +
 .../FetchingQueryCursorStressTest.java          |  277 +
 .../tcp/GridOrderedMessageCancelSelfTest.java   |    4 +-
 .../IgniteBinaryCacheQueryTestSuite.java        |    9 -
 .../IgniteBinaryCacheQueryTestSuite2.java       |    4 -
 .../IgniteBinaryCacheQueryTestSuite3.java       |   39 -
 .../IgniteBinaryCacheQueryTestSuite4.java       |   39 -
 .../IgniteCacheQuerySelfTestSuite.java          |  130 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |   32 +-
 .../IgniteCacheQuerySelfTestSuite3.java         |   23 +-
 .../IgniteCacheQuerySelfTestSuite4.java         |   12 +-
 .../IgniteCacheWithIndexingTestSuite.java       |   35 +-
 ...IgniteDbMemoryLeakWithIndexingTestSuite.java |   40 +
 .../IgniteDistributedJoinTestSuite.java         |   55 +
 .../IgniteH2IndexingSpiTestSuite.java           |   37 -
 .../IgnitePdsWithIndexingCoreTestSuite.java     |   57 +
 .../IgnitePdsWithIndexingTestSuite.java         |   46 +
 .../processors/query/h2/sql/bigQuery.sql        |   50 +-
 modules/jcl/pom.xml                             |    2 +-
 modules/jms11/pom.xml                           |    2 +-
 .../stream/jms11/IgniteJmsStreamerTest.java     |   28 +-
 modules/jta/pom.xml                             |    2 +-
 .../processors/cache/jta/CacheJtaManager.java   |    3 +-
 .../processors/cache/jta/CacheJtaResource.java  |    8 +-
 .../cache/AbstractCacheJtaSelfTest.java         |    8 +-
 ...CacheJtaConfigurationValidationSelfTest.java |    6 +-
 ...CacheJtaFactoryConfigValidationSelfTest.java |    6 +-
 .../cache/GridJtaLifecycleAwareSelfTest.java    |    4 +-
 ...titionedCacheJtaLookupClassNameSelfTest.java |    2 +-
 modules/kafka/pom.xml                           |    2 +-
 .../stream/kafka/connect/IgniteSourceTask.java  |    4 -
 .../kafka/KafkaIgniteStreamerSelfTest.java      |    8 +-
 .../ignite/stream/kafka/TestKafkaBroker.java    |    2 +-
 .../kafka/connect/IgniteSinkConnectorTest.java  |    3 +-
 .../connect/IgniteSourceConnectorTest.java      |    3 +-
 modules/kubernetes/DEVNOTES.txt                 |   63 +
 modules/kubernetes/README.txt                   |   33 +
 modules/kubernetes/config/Dockerfile            |   62 +
 modules/kubernetes/config/example-kube.xml      |   44 +
 .../kubernetes/config/ignite-deployment.yaml    |   41 +
 modules/kubernetes/config/ignite-service.yaml   |   29 +
 modules/kubernetes/config/run.sh                |   50 +
 modules/kubernetes/licenses/apache-2.0.txt      |  202 +
 modules/kubernetes/pom.xml                      |   93 +
 .../TcpDiscoveryKubernetesIpFinder.java         |  317 ++
 .../tcp/ipfinder/kubernetes/package-info.java   |   22 +
 .../TcpDiscoveryKubernetesIpFinderSelfTest.java |   93 +
 .../tcp/ipfinder/kubernetes/package-info.java   |   22 +
 .../testsuites/IgniteKubernetesTestSuite.java   |   41 +
 modules/log4j/pom.xml                           |    2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |    4 +-
 .../log4j/GridLog4jCorrectFileNameTest.java     |    8 +-
 modules/log4j2/pom.xml                          |    6 +-
 .../ignite/logger/log4j2/Log4J2Logger.java      |   23 +-
 .../log4j2/GridLog4j2CorrectFileNameTest.java   |    6 +-
 .../log4j2/GridLog4j2InitializedTest.java       |    5 +-
 .../log4j2/GridLog4j2LoggingFileTest.java       |    3 +-
 .../logger/log4j2/Log4j2LoggerSelfTest.java     |    6 +-
 .../log4j2/Log4j2LoggerVerboseModeSelfTest.java |    6 +-
 modules/mesos/pom.xml                           |    2 +-
 .../apache/ignite/mesos/IgniteFramework.java    |  111 +-
 .../org/apache/ignite/mesos/IgniteTask.java     |    3 +-
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |   39 +-
 modules/ml/README.txt                           |   15 +
 modules/ml/licenses/apache-2.0.txt              |  202 +
 modules/ml/licenses/mit.txt                     |    7 +
 modules/ml/pom.xml                              |  114 +
 .../main/java/org/apache/ignite/ml/Model.java   |   39 +
 .../ml/clustering/BaseKMeansClusterer.java      |   98 +
 .../apache/ignite/ml/clustering/Clusterer.java  |   32 +
 .../ml/clustering/ClusterizationModel.java      |   29 +
 .../clustering/KMeansDistributedClusterer.java  |  298 ++
 .../ml/clustering/KMeansLocalClusterer.java     |  174 +
 .../ignite/ml/clustering/KMeansModel.java       |   79 +
 .../ignite/ml/clustering/WeightedClusterer.java |   38 +
 .../ignite/ml/clustering/package-info.java      |   22 +
 .../java/org/apache/ignite/ml/math/Algebra.java |  590 +++
 .../org/apache/ignite/ml/math/Constants.java    |   59 +
 .../org/apache/ignite/ml/math/Destroyable.java  |   30 +
 .../apache/ignite/ml/math/DistanceMeasure.java  |   39 +
 .../ignite/ml/math/EuclideanDistance.java       |   48 +
 .../ignite/ml/math/IdentityValueMapper.java     |   54 +
 .../org/apache/ignite/ml/math/KeyMapper.java    |   33 +
 .../org/apache/ignite/ml/math/MathUtils.java    |   31 +
 .../java/org/apache/ignite/ml/math/Matrix.java  |  520 ++
 .../apache/ignite/ml/math/MatrixKeyMapper.java  |   30 +
 .../apache/ignite/ml/math/MatrixStorage.java    |   58 +
 .../apache/ignite/ml/math/MetaAttributes.java   |   76 +
 .../org/apache/ignite/ml/math/MurmurHash.java   |  247 +
 .../org/apache/ignite/ml/math/Precision.java    |  588 +++
 .../apache/ignite/ml/math/StorageConstants.java |   52 +
 .../ignite/ml/math/StorageOpsMetrics.java       |   49 +
 .../java/org/apache/ignite/ml/math/Tracer.java  |  463 ++
 .../org/apache/ignite/ml/math/ValueMapper.java  |   37 +
 .../java/org/apache/ignite/ml/math/Vector.java  |  499 ++
 .../apache/ignite/ml/math/VectorKeyMapper.java  |   29 +
 .../apache/ignite/ml/math/VectorStorage.java    |   53 +
 .../org/apache/ignite/ml/math/VectorUtils.java  |   41 +
 .../decompositions/CholeskyDecomposition.java   |  310 ++
 .../decompositions/DecompositionSupport.java    |  105 +
 .../math/decompositions/EigenDecomposition.java |  936 ++++
 .../ml/math/decompositions/LUDecomposition.java |  383 ++
 .../ml/math/decompositions/QRDecomposition.java |  246 +
 .../SingularValueDecomposition.java             |  623 +++
 .../ml/math/decompositions/package-info.java    |   22 +
 .../math/exceptions/CardinalityException.java   |   36 +
 .../math/exceptions/ColumnIndexException.java   |   35 +
 .../math/exceptions/ConvergenceException.java   |   48 +
 .../ml/math/exceptions/IndexException.java      |   35 +
 .../exceptions/InsufficientDataException.java   |   44 +
 .../exceptions/MathArithmeticException.java     |   45 +
 .../MathIllegalArgumentException.java           |   37 +
 .../exceptions/MathIllegalNumberException.java  |   51 +
 .../exceptions/MathIllegalStateException.java   |   49 +
 .../math/exceptions/MathRuntimeException.java   |   47 +
 .../ml/math/exceptions/NoDataException.java     |   45 +
 .../NonPositiveDefiniteMatrixException.java     |   35 +
 .../exceptions/NonSquareMatrixException.java    |   33 +
 .../exceptions/NonSymmetricMatrixException.java |   35 +
 .../math/exceptions/NullArgumentException.java  |   27 +
 .../exceptions/NumberIsTooSmallException.java   |   79 +
 .../ml/math/exceptions/RowIndexException.java   |   35 +
 .../exceptions/SingularMatrixException.java     |   33 +
 .../exceptions/UnknownProviderException.java    |   35 +
 .../UnsupportedOperationException.java          |   44 +
 .../ignite/ml/math/exceptions/package-info.java |   22 +
 .../ignite/ml/math/functions/Functions.java     |  194 +
 .../ml/math/functions/IgniteBiConsumer.java     |   29 +
 .../ml/math/functions/IgniteBiFunction.java     |   29 +
 .../ml/math/functions/IgniteConsumer.java       |   29 +
 .../ml/math/functions/IgniteDoubleFunction.java |   29 +
 .../ml/math/functions/IgniteFunction.java       |   30 +
 .../math/functions/IntDoubleToVoidFunction.java |   25 +
 .../functions/IntIntDoubleToVoidFunction.java   |   32 +
 .../math/functions/IntIntToDoubleFunction.java  |   24 +
 .../ignite/ml/math/functions/package-info.java  |   22 +
 .../apache/ignite/ml/math/impls/CacheUtils.java |  446 ++
 .../ml/math/impls/matrix/AbstractMatrix.java    |  887 ++++
 .../ml/math/impls/matrix/CacheMatrix.java       |  159 +
 .../impls/matrix/DenseLocalOffHeapMatrix.java   |   85 +
 .../impls/matrix/DenseLocalOnHeapMatrix.java    |   86 +
 .../ml/math/impls/matrix/DiagonalMatrix.java    |  101 +
 .../ml/math/impls/matrix/FunctionMatrix.java    |   95 +
 .../ignite/ml/math/impls/matrix/MatrixView.java |   84 +
 .../ml/math/impls/matrix/PivotedMatrixView.java |  241 +
 .../ml/math/impls/matrix/RandomMatrix.java      |   97 +
 .../impls/matrix/SparseDistributedMatrix.java   |  153 +
 .../impls/matrix/SparseLocalOnHeapMatrix.java   |   72 +
 .../math/impls/matrix/TransposedMatrixView.java |   84 +
 .../ml/math/impls/matrix/package-info.java      |   22 +
 .../ignite/ml/math/impls/package-info.java      |   22 +
 .../storage/matrix/ArrayMatrixStorage.java      |  161 +
 .../storage/matrix/CacheMatrixStorage.java      |  185 +
 .../matrix/DenseOffHeapMatrixStorage.java       |  206 +
 .../storage/matrix/DiagonalMatrixStorage.java   |  136 +
 .../storage/matrix/FunctionMatrixStorage.java   |  179 +
 .../impls/storage/matrix/MapWrapperStorage.java |  110 +
 .../storage/matrix/MatrixDelegateStorage.java   |  205 +
 .../storage/matrix/PivotedMatrixStorage.java    |  256 +
 .../storage/matrix/RandomMatrixStorage.java     |  176 +
 .../matrix/SparseDistributedMatrixStorage.java  |  318 ++
 .../matrix/SparseLocalOnHeapMatrixStorage.java  |  228 +
 .../math/impls/storage/matrix/package-info.java |   22 +
 .../storage/vector/ArrayVectorStorage.java      |  135 +
 .../storage/vector/CacheVectorStorage.java      |  175 +
 .../storage/vector/ConstantVectorStorage.java   |  134 +
 .../storage/vector/DelegateVectorStorage.java   |  163 +
 .../vector/DenseLocalOffHeapVectorStorage.java  |  172 +
 .../storage/vector/FunctionVectorStorage.java   |  143 +
 .../storage/vector/MatrixVectorStorage.java     |  184 +
 .../storage/vector/PivotedVectorStorage.java    |  176 +
 .../storage/vector/RandomVectorStorage.java     |  152 +
 .../SingleElementVectorDelegateStorage.java     |  145 +
 .../vector/SingleElementVectorStorage.java      |  145 +
 .../vector/SparseLocalOffHeapVectorStorage.java |  153 +
 .../vector/SparseLocalOnHeapVectorStorage.java  |  185 +
 .../math/impls/storage/vector/package-info.java |   22 +
 .../impls/vector/AbstractReadOnlyVector.java    |  125 +
 .../ml/math/impls/vector/AbstractVector.java    |  907 ++++
 .../ml/math/impls/vector/CacheVector.java       |  140 +
 .../ml/math/impls/vector/ConstantVector.java    |   84 +
 .../ml/math/impls/vector/DelegatingVector.java  |  396 ++
 .../impls/vector/DenseLocalOffHeapVector.java   |   89 +
 .../impls/vector/DenseLocalOnHeapVector.java    |  104 +
 .../ml/math/impls/vector/FunctionVector.java    |  112 +
 .../ml/math/impls/vector/MapWrapperVector.java  |   49 +
 .../ml/math/impls/vector/MatrixVectorView.java  |  144 +
 .../ml/math/impls/vector/PivotedVectorView.java |  163 +
 .../ml/math/impls/vector/RandomVector.java      |  130 +
 .../math/impls/vector/SingleElementVector.java  |  102 +
 .../impls/vector/SingleElementVectorView.java   |   97 +
 .../impls/vector/SparseLocalOffHeapVector.java  |   47 +
 .../ml/math/impls/vector/SparseLocalVector.java |   80 +
 .../ignite/ml/math/impls/vector/VectorView.java |   86 +
 .../ml/math/impls/vector/package-info.java      |   22 +
 .../org/apache/ignite/ml/math/package-info.java |   22 +
 .../ignite/ml/math/statistics/Variance.java     |   53 +
 .../org/apache/ignite/ml/math/util/MapUtil.java |   38 +
 .../apache/ignite/ml/math/util/MatrixUtil.java  |  156 +
 .../AbstractMultipleLinearRegression.java       |  358 ++
 .../regressions/MultipleLinearRegression.java   |   71 +
 .../OLSMultipleLinearRegression.java            |  272 +
 .../regressions/RegressionsErrorMessages.java   |   28 +
 .../ignite/ml/regressions/package-info.java     |   22 +
 .../ignite/ml/math/d3-matrix-template.html      |  119 +
 .../ignite/ml/math/d3-vector-template.html      |  104 +
 .../org/apache/ignite/ml/IgniteMLTestSuite.java |   37 +
 .../java/org/apache/ignite/ml/TestUtils.java    |  248 +
 .../ml/clustering/ClusteringTesetSuite.java     |   32 +
 .../KMeansDistributedClustererTest.java         |  184 +
 .../ml/clustering/KMeansLocalClustererTest.java |   46 +
 .../apache/ignite/ml/clustering/KMeansUtil.java |   33 +
 .../apache/ignite/ml/math/ExternalizeTest.java  |   67 +
 .../ml/math/MathImplDistributedTestSuite.java   |   39 +
 .../ignite/ml/math/MathImplLocalTestSuite.java  |  124 +
 .../ignite/ml/math/MathImplMainTestSuite.java   |   34 +
 .../org/apache/ignite/ml/math/TracerTest.java   |  198 +
 .../ignite/ml/math/benchmark/MathBenchmark.java |  205 +
 .../math/benchmark/MathBenchmarkSelfTest.java   |  100 +
 .../ignite/ml/math/benchmark/ResultsWriter.java |  127 +
 .../ml/math/benchmark/VectorBenchmarkTest.java  |  138 +
 .../ignite/ml/math/benchmark/package-info.java  |   18 +
 .../CholeskyDecompositionTest.java              |  160 +
 .../decompositions/EigenDecompositionTest.java  |  193 +
 .../decompositions/LUDecompositionTest.java     |  252 +
 .../decompositions/QRDecompositionTest.java     |  141 +
 .../SingularValueDecompositionTest.java         |  122 +
 .../ignite/ml/math/impls/MathTestConstants.java |   88 +
 .../ml/math/impls/matrix/CacheMatrixTest.java   |  371 ++
 .../DenseLocalOffHeapMatrixConstructorTest.java |   65 +
 .../DenseLocalOnHeapMatrixConstructorTest.java  |   71 +
 .../math/impls/matrix/DiagonalMatrixTest.java   |  209 +
 .../matrix/FunctionMatrixConstructorTest.java   |  113 +
 .../math/impls/matrix/MatrixAttributeTest.java  |  156 +
 .../matrix/MatrixImplementationFixtures.java    |  381 ++
 .../impls/matrix/MatrixImplementationsTest.java | 1113 ++++
 .../impls/matrix/MatrixKeyMapperForTests.java   |   74 +
 .../impls/matrix/MatrixViewConstructorTest.java |  114 +
 .../PivotedMatrixViewConstructorTest.java       |  129 +
 .../matrix/RandomMatrixConstructorTest.java     |   71 +
 .../matrix/SparseDistributedMatrixTest.java     |  303 ++
 .../SparseLocalOnHeapMatrixConstructorTest.java |   53 +
 .../impls/matrix/TransposedMatrixViewTest.java  |   87 +
 .../storage/matrix/MatrixArrayStorageTest.java  |   63 +
 .../storage/matrix/MatrixBaseStorageTest.java   |   89 +
 .../matrix/MatrixOffHeapStorageTest.java        |   39 +
 .../storage/matrix/MatrixStorageFixtures.java   |  143 +
 .../matrix/MatrixStorageImplementationTest.java |   73 +
 .../SparseDistributedMatrixStorageTest.java     |  126 +
 .../RandomAccessSparseVectorStorageTest.java    |   60 +
 .../SparseLocalOffHeapVectorStorageTest.java    |   79 +
 .../storage/vector/VectorArrayStorageTest.java  |   58 +
 .../storage/vector/VectorBaseStorageTest.java   |   69 +
 .../vector/VectorOffheapStorageTest.java        |   73 +
 .../math/impls/vector/AbstractVectorTest.java   |  544 ++
 .../ml/math/impls/vector/CacheVectorTest.java   |  436 ++
 .../vector/ConstantVectorConstructorTest.java   |   52 +
 .../vector/DelegatingVectorConstructorTest.java |   62 +
 .../DenseLocalOffHeapVectorConstructorTest.java |   59 +
 .../DenseLocalOnHeapVectorConstructorTest.java  |  163 +
 .../vector/FunctionVectorConstructorTest.java   |  121 +
 .../math/impls/vector/MatrixVectorViewTest.java |  226 +
 .../PivotedVectorViewConstructorTest.java       |  211 +
 .../vector/RandomVectorConstructorTest.java     |  145 +
 .../SingleElementVectorConstructorTest.java     |  159 +
 .../SingleElementVectorViewConstructorTest.java |  137 +
 .../SparseLocalVectorConstructorTest.java       |   54 +
 .../math/impls/vector/VectorAttributesTest.java |  217 +
 .../ml/math/impls/vector/VectorFoldMapTest.java |  122 +
 .../vector/VectorImplementationsFixtures.java   |  655 +++
 .../impls/vector/VectorImplementationsTest.java |  861 ++++
 .../math/impls/vector/VectorIterableTest.java   |  376 ++
 .../ml/math/impls/vector/VectorNormTest.java    |  247 +
 .../math/impls/vector/VectorToMatrixTest.java   |  308 ++
 .../ml/math/impls/vector/VectorViewTest.java    |  162 +
 .../AbstractMultipleLinearRegressionTest.java   |  164 +
 .../OLSMultipleLinearRegressionTest.java        |  819 +++
 .../ml/regressions/RegressionsTestSuite.java    |   32 +
 modules/mqtt/pom.xml                            |    2 +-
 .../stream/mqtt/IgniteMqttStreamerTest.java     |   12 +-
 modules/osgi-karaf/pom.xml                      |    2 +-
 .../osgi-karaf/src/main/resources/features.xml  |   11 +-
 modules/osgi-paxlogging/pom.xml                 |    2 +-
 modules/osgi/README.txt                         |   30 -
 modules/osgi/pom.xml                            |    3 +-
 .../IgniteAbstractOsgiContextActivator.java     |    2 +-
 .../activators/BasicIgniteTestActivator.java    |    2 +-
 modules/platforms/.gitignore                    |   31 +
 modules/platforms/cpp/binary/Makefile.am        |   68 +-
 .../platforms/cpp/binary/include/Makefile.am    |   57 +-
 .../binary/binary_array_identity_resolver.h     |   64 -
 .../include/ignite/binary/binary_consts.h       |   86 +-
 .../include/ignite/binary/binary_containers.h   |   14 +-
 .../ignite/binary/binary_identity_resolver.h    |   61 -
 .../include/ignite/binary/binary_object.h       |   99 +-
 .../include/ignite/binary/binary_raw_reader.h   |   25 +-
 .../include/ignite/binary/binary_raw_writer.h   |   35 +-
 .../include/ignite/binary/binary_reader.h       |   27 +-
 .../binary/include/ignite/binary/binary_type.h  |  139 +-
 .../include/ignite/binary/binary_writer.h       |   34 +-
 .../include/ignite/impl/binary/binary_common.h  |   12 +
 .../ignite/impl/binary/binary_field_meta.h      |  110 +
 .../ignite/impl/binary/binary_id_resolver.h     |   93 +-
 .../ignite/impl/binary/binary_object_header.h   |   22 +-
 .../ignite/impl/binary/binary_object_impl.h     |  154 +-
 .../ignite/impl/binary/binary_reader_impl.h     |  262 +-
 .../include/ignite/impl/binary/binary_schema.h  |   29 +-
 .../ignite/impl/binary/binary_type_handler.h    |   47 +-
 .../ignite/impl/binary/binary_type_impl.h       |  136 +-
 .../ignite/impl/binary/binary_type_manager.h    |   50 +-
 .../ignite/impl/binary/binary_type_snapshot.h   |   82 +-
 .../ignite/impl/binary/binary_type_updater.h    |   19 +-
 .../include/ignite/impl/binary/binary_utils.h   |   41 +-
 .../ignite/impl/binary/binary_writer_impl.h     |  106 +-
 .../cpp/binary/project/vs/binary.vcxproj        |    7 +-
 .../binary/project/vs/binary.vcxproj.filters    |   17 +-
 .../binary/binary_array_identity_resolver.cpp   |   42 -
 .../cpp/binary/src/binary/binary_raw_reader.cpp |   12 +-
 .../cpp/binary/src/binary/binary_raw_writer.cpp |   10 +
 .../cpp/binary/src/binary/binary_reader.cpp     |   14 +-
 .../cpp/binary/src/binary/binary_writer.cpp     |   10 +
 .../src/impl/binary/binary_field_meta.cpp       |   42 +
 .../src/impl/binary/binary_object_header.cpp    |   26 +-
 .../src/impl/binary/binary_object_impl.cpp      |  182 +-
 .../src/impl/binary/binary_reader_impl.cpp      |  227 +-
 .../binary/src/impl/binary/binary_schema.cpp    |   17 +-
 .../src/impl/binary/binary_type_handler.cpp     |   45 +-
 .../binary/src/impl/binary/binary_type_impl.cpp |   61 +
 .../src/impl/binary/binary_type_manager.cpp     |  194 +-
 .../src/impl/binary/binary_type_snapshot.cpp    |   50 +-
 .../src/impl/binary/binary_type_updater.cpp     |   32 -
 .../cpp/binary/src/impl/binary/binary_utils.cpp |   12 +
 .../src/impl/binary/binary_writer_impl.cpp      |  119 +-
 modules/platforms/cpp/common/Makefile.am        |   52 +-
 modules/platforms/cpp/common/configure.ac       |   62 +
 .../platforms/cpp/common/include/Makefile.am    |    7 +-
 .../common/include/ignite/common/cancelable.h   |   65 +
 .../common/include/ignite/common/concurrent.h   |   59 +-
 .../include/ignite/common/platform_utils.h      |    6 +-
 .../cpp/common/include/ignite/common/promise.h  |  219 +
 .../common/include/ignite/common/shared_state.h |  382 ++
 .../cpp/common/include/ignite/common/utils.h    |   78 +-
 .../cpp/common/include/ignite/future.h          |  284 ++
 .../cpp/common/include/ignite/ignite_error.h    |    8 +-
 .../cpp/common/include/ignite/reference.h       |   14 +-
 .../platforms/cpp/common/include/ignite/time.h  |  138 +
 .../linux/include/ignite/common/concurrent_os.h |  234 +-
 .../include/ignite/common/dynamic_load_os.h     |  131 +
 .../os/linux/src/common/dynamic_load_os.cpp     |   90 +
 .../os/linux/src/common/platform_utils.cpp      |   18 +-
 .../win/include/ignite/common/concurrent_os.h   |  169 +-
 .../win/include/ignite/common/dynamic_load_os.h |  133 +
 .../common/os/win/src/common/concurrent_os.cpp  |   26 +-
 .../os/win/src/common/dynamic_load_os.cpp       |  115 +
 .../common/os/win/src/common/platform_utils.cpp |   26 +-
 .../cpp/common/project/vs/common.vcxproj        |    9 +-
 .../common/project/vs/common.vcxproj.filters    |   30 +-
 .../platforms/cpp/common/project/vs/targetver.h |   25 -
 .../platforms/cpp/common/src/common/utils.cpp   |   33 +-
 .../platforms/cpp/common/src/ignite_error.cpp   |   68 +-
 modules/platforms/cpp/common/src/time.cpp       |   83 +
 modules/platforms/cpp/configure.ac              |    2 +-
 modules/platforms/cpp/configure.acrel           |    2 +-
 modules/platforms/cpp/core-test/Makefile.am     |   95 +-
 .../cpp/core-test/config/cache-identity-32.xml  |   53 +
 .../core-test/config/cache-identity-default.xml |  154 +
 .../cpp/core-test/config/cache-identity.xml     |   91 +-
 .../cpp/core-test/config/cache-query-32.xml     |   52 +
 .../config/cache-query-continuous-32.xml        |   48 +
 .../config/cache-query-continuous-default.xml   |   86 +
 .../core-test/config/cache-query-continuous.xml |   61 +-
 .../core-test/config/cache-query-default.xml    |  182 +
 .../cpp/core-test/config/cache-query.xml        |   91 +-
 .../cpp/core-test/config/cache-store-32.xml     |   52 +
 .../core-test/config/cache-store-default.xml    |   71 +
 .../cpp/core-test/config/cache-store.xml        |   34 +
 .../cpp/core-test/config/cache-test-32.xml      |   52 +
 .../cpp/core-test/config/cache-test-default.xml |  133 +
 .../cpp/core-test/config/cache-test.xml         |  106 +-
 modules/platforms/cpp/core-test/configure.ac    |   62 +
 .../core-test/include/ignite/binary_test_defs.h |  205 +-
 .../include/ignite/binary_test_utils.h          |   53 +
 .../cpp/core-test/include/ignite/complex_type.h |   32 +-
 .../cpp/core-test/include/ignite/test_type.h    |   55 +-
 .../cpp/core-test/include/ignite/test_utils.h   |   31 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |   25 +
 .../project/vs/core-test.vcxproj.filters        |   54 +
 .../src/binary_identity_resolver_test.cpp       |  223 +-
 .../cpp/core-test/src/binary_object_test.cpp    |  308 +-
 .../src/binary_reader_writer_raw_test.cpp       |  777 +--
 .../core-test/src/binary_reader_writer_test.cpp | 1013 +---
 .../cpp/core-test/src/binary_session_test.cpp   |   20 +-
 .../cpp/core-test/src/binary_test_defs.cpp      |    5 +
 .../cpp/core-test/src/cache_invoke_test.cpp     |  528 ++
 .../cpp/core-test/src/cache_query_test.cpp      |  451 +-
 .../cpp/core-test/src/cache_store_test.cpp      |  160 +
 .../platforms/cpp/core-test/src/cache_test.cpp  |  320 +-
 .../cpp/core-test/src/cluster_test.cpp          |   86 +
 .../cpp/core-test/src/compute_test.cpp          |  513 ++
 .../cpp/core-test/src/concurrent_test.cpp       |   47 +
 .../cpp/core-test/src/continuous_query_test.cpp |  218 +-
 .../cpp/core-test/src/date_time_test.cpp        |  287 ++
 .../platforms/cpp/core-test/src/future_test.cpp |  474 ++
 .../cpp/core-test/src/handle_registry_test.cpp  |    2 +-
 .../cpp/core-test/src/ignite_error_test.cpp     |    2 +-
 .../cpp/core-test/src/ignition_test.cpp         |   18 +-
 .../cpp/core-test/src/interop_memory_test.cpp   |    5 +-
 .../cpp/core-test/src/interop_test.cpp          |    4 +
 .../cpp/core-test/src/reference_test.cpp        |   14 +-
 .../platforms/cpp/core-test/src/test_utils.cpp  |   17 +
 .../cpp/core-test/src/transactions_test.cpp     |   82 +-
 modules/platforms/cpp/core/Makefile.am          |   80 +-
 modules/platforms/cpp/core/configure.ac         |   62 +
 modules/platforms/cpp/core/include/Makefile.am  |   89 +-
 .../cpp/core/include/ignite/cache/cache.h       |  466 +-
 .../cpp/core/include/ignite/cache/cache_entry.h |   15 +
 .../ignite/cache/cache_entry_processor.h        |   81 +
 .../core/include/ignite/cache/cache_peek_mode.h |   67 +-
 .../cache/event/cache_entry_event_filter.h      |  109 +
 .../include/ignite/cache/mutable_cache_entry.h  |  176 +
 .../cache/query/continuous/continuous_query.h   |   37 +-
 .../cpp/core/include/ignite/cache/query/query.h |    1 -
 .../include/ignite/cache/query/query_argument.h |  134 -
 .../include/ignite/cache/query/query_cursor.h   |   30 +-
 .../core/include/ignite/cache/query/query_sql.h |   26 +-
 .../ignite/cache/query/query_sql_fields.h       |   80 +-
 .../cpp/core/include/ignite/compute/compute.h   |  167 +
 .../core/include/ignite/compute/compute_func.h  |   65 +
 .../platforms/cpp/core/include/ignite/ignite.h  |   55 +-
 .../cpp/core/include/ignite/ignite_binding.h    |  169 +
 .../include/ignite/ignite_binding_context.h     |   88 +
 .../core/include/ignite/ignite_configuration.h  |    2 -
 .../cpp/core/include/ignite/ignition.h          |   14 +-
 .../impl/binary/binary_type_updater_impl.h      |    6 +-
 .../cpp/core/include/ignite/impl/bindings.h     |  119 +
 .../impl/cache/cache_entry_processor_holder.h   |  281 +
 .../core/include/ignite/impl/cache/cache_impl.h |  167 +-
 .../cache/event/cache_entry_event_filter_base.h |   66 +
 .../event/cache_entry_event_filter_holder.h     |  185 +
 .../continuous/continuous_query_handle_impl.h   |   10 -
 .../query/continuous/continuous_query_impl.h    |   60 +-
 .../ignite/impl/cache/query/query_argument.h    |  137 +
 .../ignite/impl/cache/query/query_impl.h        |    7 +
 .../ignite/impl/cluster/cluster_group_impl.h    |  100 +
 .../ignite/impl/compute/cancelable_impl.h       |   70 +
 .../include/ignite/impl/compute/compute_impl.h  |  160 +
 .../ignite/impl/compute/compute_job_holder.h    |  212 +
 .../ignite/impl/compute/compute_job_result.h    |  273 +
 .../ignite/impl/compute/compute_task_holder.h   |  298 ++
 .../cpp/core/include/ignite/impl/helpers.h      |   57 +
 .../include/ignite/impl/ignite_binding_impl.h   |  135 +
 .../include/ignite/impl/ignite_environment.h    |  145 +-
 .../cpp/core/include/ignite/impl/ignite_impl.h  |   74 +-
 .../ignite/impl/interop/interop_target.h        |   76 +-
 .../core/include/ignite/impl/module_manager.h   |  157 +
 .../cpp/core/include/ignite/impl/operations.h   |  198 +-
 .../ignite/impl/transactions/transaction_impl.h |    4 +-
 .../impl/transactions/transactions_impl.h       |   10 +-
 .../include/ignite/transactions/transaction.h   |   12 +-
 .../ignite/transactions/transaction_consts.h    |  181 +-
 .../include/ignite/transactions/transactions.h  |   16 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   26 +-
 .../cpp/core/project/vs/core.vcxproj.filters    |   90 +-
 modules/platforms/cpp/core/src/ignite.cpp       |   17 +-
 modules/platforms/cpp/core/src/ignition.cpp     |  373 +-
 .../impl/binary/binary_type_updater_impl.cpp    |  106 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |  204 +-
 .../continuous/continuous_query_handle_impl.cpp |   20 +-
 .../core/src/impl/cache/query/query_impl.cpp    |   47 +-
 .../src/impl/cluster/cluster_group_impl.cpp     |   78 +
 .../core/src/impl/compute/cancelable_impl.cpp   |   59 +
 .../cpp/core/src/impl/compute/compute_impl.cpp  |   35 +
 .../cpp/core/src/impl/ignite_binding_impl.cpp   |   88 +
 .../cpp/core/src/impl/ignite_environment.cpp    |  476 +-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |   39 +-
 .../core/src/impl/interop/interop_target.cpp    |  105 +-
 .../src/impl/transactions/transaction_impl.cpp  |   18 +-
 .../src/impl/transactions/transactions_impl.cpp |  102 +-
 .../cpp/core/src/transactions/transaction.cpp   |    8 +-
 .../cpp/core/src/transactions/transactions.cpp  |   20 +-
 modules/platforms/cpp/examples/Makefile.am      |    3 +-
 .../cpp/examples/compute-example/Makefile.am    |   59 +
 .../compute-example/config/compute-example.xml  |   52 +
 .../project/vs/compute-example.vcxproj          |  102 +
 .../project/vs/compute-example.vcxproj.filters  |   27 +
 .../compute-example/src/compute_example.cpp     |  179 +
 modules/platforms/cpp/examples/configure.ac     |    5 +-
 .../project/vs/continuous-query-example.vcxproj |    4 -
 .../src/continuous_query_example.cpp            |  132 +-
 .../examples/include/ignite/examples/address.h  |   26 +-
 .../include/ignite/examples/organization.h      |   26 +-
 .../examples/include/ignite/examples/person.h   |   17 +-
 .../cpp/examples/odbc-example/Makefile.am       |   12 +-
 .../odbc-example/config/example-odbc.xml        |    6 -
 .../examples/odbc-example/src/odbc_example.cpp  |    7 +-
 .../cpp/examples/project/vs/ignite-examples.sln |   18 +-
 .../cpp/examples/put-get-example/Makefile.am    |   58 +
 .../put-get-example/config/example-cache.xml    |   67 +
 .../project/vs/put-get-example.vcxproj          |  110 +
 .../project/vs/put-get-example.vcxproj.filters  |   38 +
 .../put-get-example/src/put_get_example.cpp     |  125 +
 .../cpp/examples/putget-example/Makefile.am     |   58 -
 .../putget-example/config/example-cache.xml     |   75 -
 .../project/vs/putget-example.vcxproj           |  107 -
 .../project/vs/putget-example.vcxproj.filters   |   30 -
 .../putget-example/src/putget_example.cpp       |  126 -
 .../cpp/examples/query-example/Makefile.am      |   12 +-
 .../project/vs/query-example.vcxproj            |    3 +
 .../project/vs/query-example.vcxproj.filters    |    8 +
 .../query-example/src/query_example.cpp         |   11 +-
 modules/platforms/cpp/ignite/configure.ac       |   62 +
 .../cpp/jni/include/ignite/jni/exports.h        |    3 +-
 .../platforms/cpp/jni/include/ignite/jni/java.h |    9 +-
 .../cpp/jni/include/ignite/jni/utils.h          |   78 +-
 .../platforms/cpp/jni/os/linux/src/utils.cpp    |  227 +-
 modules/platforms/cpp/jni/os/win/src/utils.cpp  |  233 +-
 .../platforms/cpp/jni/project/vs/jni.vcxproj    |    1 -
 .../cpp/jni/project/vs/jni.vcxproj.filters      |    3 -
 modules/platforms/cpp/jni/project/vs/module.def |    3 +-
 .../platforms/cpp/jni/project/vs/targetver.h    |   25 -
 modules/platforms/cpp/jni/src/exports.cpp       |    8 +-
 modules/platforms/cpp/jni/src/java.cpp          |   83 +-
 .../cpp/odbc-test/config/queries-default.xml    |    7 +-
 .../cpp/odbc-test/config/queries-test-32.xml    |   49 +
 .../odbc-test/config/queries-test-default.xml   |   39 +
 .../odbc-test/config/queries-test-noodbc-32.xml |   53 +
 .../odbc-test/config/queries-test-noodbc.xml    |    6 +-
 .../cpp/odbc-test/config/queries-test.xml       |   11 +-
 .../cpp/odbc-test/include/complex_type.h        |   53 +-
 .../odbc-test/include/sql_test_suite_fixture.h  |    7 +-
 .../platforms/cpp/odbc-test/include/test_type.h |   67 +-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |    5 +
 .../project/vs/odbc-test.vcxproj.filters        |   15 +
 .../cpp/odbc-test/src/api_robustness_test.cpp   |   75 +-
 .../src/application_data_buffer_test.cpp        |  346 +-
 .../platforms/cpp/odbc-test/src/column_test.cpp |   63 +-
 .../cpp/odbc-test/src/configuration_test.cpp    |  120 +-
 .../cpp/odbc-test/src/connection_info_test.cpp  |   72 +-
 .../platforms/cpp/odbc-test/src/cursor_test.cpp |    6 +-
 .../cpp/odbc-test/src/queries_test.cpp          |  851 +++-
 .../platforms/cpp/odbc-test/src/row_test.cpp    |   17 +-
 .../src/sql_date_time_functions_test.cpp        |    5 +
 .../src/sql_esc_convert_function_test.cpp       |    8 +-
 .../odbc-test/src/sql_test_suite_fixture.cpp    |   37 +-
 .../cpp/odbc-test/src/sql_types_test.cpp        |   24 +-
 .../platforms/cpp/odbc-test/src/test_utils.cpp  |    8 +
 modules/platforms/cpp/odbc/Makefile.am          |    3 +
 modules/platforms/cpp/odbc/README.txt           |   17 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |    2 +
 .../ignite/odbc/app/application_data_buffer.h   |   59 +-
 .../odbc/include/ignite/odbc/app/parameter.h    |   20 +-
 .../include/ignite/odbc/app/parameter_set.h     |  268 +
 .../cpp/odbc/include/ignite/odbc/column.h       |    2 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |  350 +-
 .../include/ignite/odbc/config/configuration.h  |   76 +-
 .../ignite/odbc/config/connection_info.h        |    2 +-
 .../cpp/odbc/include/ignite/odbc/connection.h   |   26 +-
 .../ignite/odbc/diagnostic/diagnosable.h        |    8 +-
 .../odbc/diagnostic/diagnosable_adapter.h       |    8 +-
 .../ignite/odbc/diagnostic/diagnostic_record.h  |    4 +-
 .../odbc/diagnostic/diagnostic_record_storage.h |    8 +-
 .../cpp/odbc/include/ignite/odbc/environment.h  |   10 +-
 .../cpp/odbc/include/ignite/odbc/message.h      |  495 +-
 .../odbc/include/ignite/odbc/protocol_version.h |  103 +-
 .../include/ignite/odbc/query/batch_query.h     |  160 +
 .../ignite/odbc/query/column_metadata_query.h   |   10 +-
 .../odbc/include/ignite/odbc/query/data_query.h |   35 +-
 .../ignite/odbc/query/foreign_keys_query.h      |    8 +-
 .../ignite/odbc/query/primary_keys_query.h      |    8 +-
 .../cpp/odbc/include/ignite/odbc/query/query.h  |   35 +-
 .../ignite/odbc/query/special_columns_query.h   |    8 +-
 .../ignite/odbc/query/table_metadata_query.h    |   10 +-
 .../include/ignite/odbc/query/type_info_query.h |   10 +-
 .../cpp/odbc/include/ignite/odbc/row.h          |    2 +-
 .../cpp/odbc/include/ignite/odbc/statement.h    |  115 +-
 .../odbc/system/ui/dsn_configuration_window.h   |   51 +-
 .../cpp/odbc/include/ignite/odbc/type_traits.h  |   96 +-
 .../cpp/odbc/install/ignite-odbc-amd64.wxs      |    2 +-
 .../cpp/odbc/install/ignite-odbc-x86.wxs        |    2 +-
 .../cpp/odbc/install/install_amd64.cmd          |   14 +-
 .../platforms/cpp/odbc/install/install_x86.cmd  |    8 +-
 .../ignite/odbc/system/ui/custom_window.h       |   12 +-
 .../odbc/os/win/src/system/ui/custom_window.cpp |    4 +-
 .../src/system/ui/dsn_configuration_window.cpp  |  126 +-
 .../cpp/odbc/os/win/src/system_dsn.cpp          |    2 +-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |    5 +
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |   15 +
 .../odbc/src/app/application_data_buffer.cpp    |  614 ++-
 .../platforms/cpp/odbc/src/app/parameter.cpp    |   24 +-
 .../cpp/odbc/src/app/parameter_set.cpp          |  242 +
 modules/platforms/cpp/odbc/src/column.cpp       |   32 +-
 modules/platforms/cpp/odbc/src/common_types.cpp |   54 +-
 .../cpp/odbc/src/config/configuration.cpp       |    8 +-
 .../cpp/odbc/src/config/connection_info.cpp     |   48 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |  115 +-
 .../odbc/src/diagnostic/diagnosable_adapter.cpp |    4 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |   46 +-
 .../diagnostic/diagnostic_record_storage.cpp    |   78 +-
 modules/platforms/cpp/odbc/src/dsn_config.cpp   |   10 +-
 modules/platforms/cpp/odbc/src/environment.cpp  |   62 +-
 modules/platforms/cpp/odbc/src/message.cpp      |  378 ++
 modules/platforms/cpp/odbc/src/odbc.cpp         |   29 +-
 .../platforms/cpp/odbc/src/protocol_version.cpp |  138 +-
 .../cpp/odbc/src/query/batch_query.cpp          |  198 +
 .../odbc/src/query/column_metadata_query.cpp    |  125 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |  132 +-
 .../cpp/odbc/src/query/foreign_keys_query.cpp   |   26 +-
 .../cpp/odbc/src/query/primary_keys_query.cpp   |   71 +-
 .../odbc/src/query/special_columns_query.cpp    |   26 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |   81 +-
 .../cpp/odbc/src/query/type_info_query.cpp      |  231 +-
 modules/platforms/cpp/odbc/src/result_page.cpp  |    2 +-
 modules/platforms/cpp/odbc/src/row.cpp          |    4 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |  530 +-
 modules/platforms/cpp/odbc/src/type_traits.cpp  |   59 +-
 modules/platforms/cpp/odbc/src/utility.cpp      |   17 +-
 .../Apache.Ignite.AspNet.Tests/App.config       |    6 +-
 .../ExpiryCacheHolderTest.cs                    |   22 +-
 .../IgniteOutputCacheProviderTest.cs            |    2 +-
 .../IgniteSessionStateStoreProviderTest.cs      |    2 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../Apache.Ignite.AspNet.nuspec                 |    2 +-
 .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs     |    4 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../Config/ignite-config.xml                    |    1 -
 .../Apache.Ignite.Core.Tests.NuGet/NuGet.config |    2 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../SchemaTest.cs                               |    2 +-
 .../StartupTest.cs                              |   77 +
 .../Apache.Ignite.Core.Tests.NuGet/TestUtil.cs  |   30 +
 .../install-package.ps1                         |   31 +-
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |    1 +
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../TestExtensions.cs                           |   35 +
 .../Apache.Ignite.Core.Tests.csproj             |   65 +-
 .../Binary/BinaryBuilderSelfTest.cs             |  423 +-
 .../BinaryBuilderSelfTestArrayIdentity.cs       |   34 -
 .../BinaryBuilderSelfTestDynamicRegistration.cs |   42 +
 .../Binary/BinaryBuilderSelfTestSimpleName.cs   |   35 +
 .../Binary/BinaryCompactFooterInteropTest.cs    |   19 +-
 .../Binary/BinaryDateTimeTest.cs                |  204 +
 .../Binary/BinaryDynamicRegistrationTest.cs     |  535 ++
 .../Binary/BinaryEqualityComparerTest.cs        |  217 +-
 .../Binary/BinaryNameMapperTest.cs              |  115 +
 .../Binary/BinaryReaderWriterTest.cs            |    8 +
 .../Binary/BinarySelfTest.cs                    |  284 +-
 .../Binary/BinarySelfTestFullFooter.cs          |    5 +-
 .../Binary/BinarySelfTestSimpleName.cs          |   33 +
 .../Binary/EnumsTest.cs                         |  547 ++
 .../Binary/EnumsTestOnline.cs                   |   45 +
 .../Binary/JavaBinaryInteropTest.cs             |  182 +
 .../Serializable/AdvancedSerializationTest.cs   |  228 +
 .../BasicSerializableObjectsTest.cs             |  124 +
 .../Binary/Serializable/CallbacksTest.cs        |  369 ++
 .../Binary/Serializable/DelegatesTest.cs        |  161 +
 .../Binary/Serializable/ObjectReferenceTests.cs |  131 +
 .../Binary/Serializable/PrimitivesTest.cs       |  754 +++
 .../Binary/Serializable/SqlDmlTest.cs           |  280 +
 .../Binary/TypeNameParserTest.cs                |  291 ++
 .../Binary/TypeResolverTest.cs                  |  104 +
 .../BinaryConfigurationTest.cs                  |   49 +-
 .../Cache/AddArgCacheEntryProcessor.cs          |   91 +
 .../Cache/Affinity/AffinityFieldTest.cs         |    6 +-
 .../Affinity/AffinityFunctionSpringTest.cs      |    6 +-
 .../Cache/Affinity/AffinityFunctionTest.cs      |  148 +-
 .../Cache/Affinity/AffinityTest.cs              |   16 +-
 .../BinarizableAddArgCacheEntryProcessor.cs     |   53 +
 .../Cache/BinarizableTestException.cs           |   51 +
 .../Cache/CacheAbstractTest.cs                  |  513 +-
 .../Cache/CacheAbstractTransactionalTest.cs     |    1 +
 .../Cache/CacheConfigurationTest.cs             |  121 +-
 .../Cache/CacheDynamicStartTest.cs              |  168 +-
 .../Cache/CacheForkedTest.cs                    |    8 +-
 .../Cache/CacheMetricsTest.cs                   |  100 +-
 .../Cache/CacheNearTest.cs                      |   38 +-
 .../Cache/CacheSwapSpaceTest.cs                 |  122 -
 .../Cache/CacheTestAsyncWrapper.cs              |   34 +-
 .../Cache/CacheTestKey.cs                       |   68 +
 .../Cache/MemoryMetricsTest.cs                  |  154 +
 .../Cache/NonSerializableCacheEntryProcessor.cs |   40 +
 .../Cache/NonSerializableException.cs           |   40 +
 .../Cache/PartitionLossTest.cs                  |  260 +
 .../Cache/PersistentStoreTest.cs                |  108 +
 .../Cache/Query/CacheDmlQueriesTest.cs          |  216 +-
 .../Query/CacheDmlQueriesTestSimpleName.cs      |   35 +
 .../Cache/Query/CacheLinqTest.cs                | 1599 ------
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   17 +-
 .../Cache/Query/CacheQueriesTest.cs             |  217 +-
 .../Cache/Query/CacheQueriesTestSimpleName.cs   |   35 +
 .../Continuous/ContinuousQueryAbstractTest.cs   |  153 +-
 .../Continuous/ContinuousQueryJavaFilterTest.cs |   23 +-
 .../Query/Continuous/ContinuousQueryTest.cs     |  115 +
 .../Query/Linq/CacheLinqTest.Aggregates.cs      |   91 +
 .../Cache/Query/Linq/CacheLinqTest.Base.cs      |  506 ++
 .../Query/Linq/CacheLinqTest.CompiledQuery.cs   |  215 +
 .../Cache/Query/Linq/CacheLinqTest.Contains.cs  |  128 +
 .../Cache/Query/Linq/CacheLinqTest.Custom.cs    |  107 +
 .../Cache/Query/Linq/CacheLinqTest.DateTime.cs  |   89 +
 .../Cache/Query/Linq/CacheLinqTest.Functions.cs |  211 +
 .../Query/Linq/CacheLinqTest.Introspection.cs   |  145 +
 .../Linq/CacheLinqTest.Join.LocalCollection.cs  |  182 +
 .../Cache/Query/Linq/CacheLinqTest.Join.cs      |  310 ++
 .../Cache/Query/Linq/CacheLinqTest.Misc.cs      |  350 ++
 .../Cache/Query/Linq/CacheLinqTest.Numerics.cs  |  131 +
 .../Cache/Query/Linq/CacheLinqTest.Strings.cs   |   91 +
 .../Cache/Query/Linq/CacheLinqTestSimpleName.cs |   35 +
 .../Query/Linq/CacheLinqTestSqlEscapeAll.cs     |   34 +
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |    9 +-
 .../Cache/Store/CacheStoreAdapterTest.cs        |   14 +-
 .../Cache/Store/CacheStoreSessionTest.cs        |   19 +-
 .../Cache/Store/CacheStoreTest.cs               |   72 +-
 .../Cache/Store/CacheTestParallelLoadStore.cs   |   16 +-
 .../Cache/Store/CacheTestStore.cs               |   13 +-
 .../Cache/TestReferenceObject.cs                |   40 +
 .../Compute/AbstractTaskTest.cs                 |   57 +-
 .../Compute/BinarizableClosureTaskTest.cs       |   29 +-
 .../Compute/BinarizableTaskTest.cs              |   15 -
 .../Compute/CancellationTest.cs                 |    4 +-
 .../Compute/ClosureTaskTest.cs                  |    6 +-
 .../Compute/ComputeApiTest.cs                   |  201 +-
 .../Compute/ComputeApiTestFullFooter.cs         |    4 +-
 .../Compute/FailoverTaskSelfTest.cs             |   52 +-
 .../Compute/IgniteExceptionTaskSelfTest.cs      |  147 +-
 .../Compute/MixedClusterTest.cs                 |   33 +-
 .../Compute/ResourceTaskTest.cs                 |   73 +-
 .../Compute/SerializableClosureTaskTest.cs      |   36 +-
 .../Compute/TaskAdapterTest.cs                  |    6 -
 .../Compute/TaskResultTest.cs                   |   12 -
 .../Config/Cache/Affinity/affinity-function.xml |    4 +-
 .../Cache/Affinity/affinity-function2.xml       |    2 +-
 .../Config/Compute/compute-grid1.xml            |    6 +-
 .../Config/Compute/compute-grid2.xml            |   13 +-
 .../Config/Compute/compute-grid3.xml            |    2 +-
 .../Config/Compute/compute-standalone.xml       |    4 +-
 .../Config/Dynamic/dynamic-data.xml             |    2 -
 .../Config/Lifecycle/lifecycle-beans.xml        |    2 +-
 .../Config/Lifecycle/lifecycle-no-beans.xml     |    2 +-
 .../Config/Log/dotnet-log4j.xml                 |    2 +-
 .../Config/cache-binarizables.xml               |   20 +-
 .../Config/cache-query-continuous.xml           |  130 +-
 .../Config/cache-query.xml                      |    2 +-
 .../Config/marshaller-invalid.xml               |    2 +-
 .../native-client-test-cache-affinity.xml       |    1 +
 .../Config/native-client-test-cache.xml         |   17 -
 .../Config/spring-test.xml                      |   14 +
 .../Config/start-test-grid1.xml                 |    2 +-
 .../Config/start-test-grid2.xml                 |    2 +-
 .../ConsoleRedirectTest.cs                      |    9 +-
 .../DataStructures/AtomicLongTest.cs            |    1 +
 .../DataStructures/AtomicReferenceTest.cs       |    2 -
 .../Dataload/DataStreamerTest.cs                |   55 +-
 .../Dataload/DataStreamerTestTopologyChange.cs  |    2 +-
 .../Deployment/GetAddressFunc.cs                |   35 +
 .../PeerAssemblyLoadingAllApisTest.cs           |  167 +
 .../Deployment/PeerAssemblyLoadingTest.cs       |  205 +
 .../PeerAssemblyLoadingVersioningTest.cs        |  164 +
 .../Deployment/ProcessNameFunc.cs               |   50 +
 .../Deployment/ProcessNameTask.cs               |   74 +
 .../Deployment/RuntimeDependencyFunc.cs         |   34 +
 .../Deployment/peer_assembly_app.config         |   35 +
 .../Apache.Ignite.Core.Tests/DeploymentTest.cs  |   31 +-
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |  165 +-
 .../Examples/ExamplesTest.cs                    |   40 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |   43 +-
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |   93 +-
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |   18 +-
 .../IgniteConfigurationSectionTest.cs           |    9 +-
 .../IgniteConfigurationSerializerTest.cs        |  319 +-
 .../IgniteConfigurationTest.cs                  |  274 +-
 .../IgniteStartStopTest.cs                      |   79 +-
 .../Apache.Ignite.Core.Tests/IgniteTestBase.cs  |   51 +-
 .../Apache.Ignite.Core.Tests/LifecycleTest.cs   |   58 +-
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |   11 +-
 .../Log/CustomLoggerTest.cs                     |   14 +-
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   24 +-
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |   43 +-
 .../Cache/CacheJavaPluginConfiguration.cs       |   45 +
 .../Plugin/Cache/CacheJavaPluginTest.cs         |  113 +
 .../Plugin/Cache/CachePlugin.cs                 |  127 -
 .../Plugin/Cache/CachePluginConfiguration.cs    |   40 -
 .../Plugin/Cache/CachePluginTest.cs             |  215 -
 .../Plugin/PluginTest.cs                        |  124 +-
 .../Plugin/TestIgnitePluginConfiguration.cs     |   26 +
 .../Plugin/TestIgnitePluginException.cs         |   64 +
 .../Plugin/TestIgnitePluginProvider.cs          |   33 +-
 .../Process/IgniteProcess.cs                    |   12 +-
 .../ProjectFilesTest.cs                         |    7 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |    5 +-
 .../SerializationTest.cs                        |  240 -
 .../Services/ServiceProxyTest.cs                |   40 +-
 .../Services/ServicesTest.cs                    |    3 +
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   41 +-
 .../dotnet/Apache.Ignite.Core.Tests/app.config  |    4 +-
 .../Apache.Ignite.Core.Tests/custom_app.config  |    2 +-
 .../Apache.Ignite.Core.Schema.nuspec            |    2 +-
 .../Apache.Ignite.Core.csproj                   |   69 +-
 .../Apache.Ignite.Core.nuspec                   |   14 +-
 .../Binary/BinaryArrayEqualityComparer.cs       |  149 -
 .../Binary/BinaryBasicNameMapper.cs             |  129 +
 .../Binary/BinaryConfiguration.cs               |   24 +-
 .../Binary/BinaryReflectiveSerializer.cs        |   44 +-
 .../Binary/BinaryTypeConfiguration.cs           |   17 +-
 .../dotnet/Apache.Ignite.Core/Binary/IBinary.cs |   24 +
 .../Apache.Ignite.Core/Binary/IBinaryObject.cs  |    8 +
 .../Binary/IBinaryObjectBuilder.cs              |   10 -
 .../Apache.Ignite.Core/Binary/IBinaryType.cs    |    6 +
 .../Binary/TimestampAttribute.cs                |   40 +
 .../Cache/Affinity/AffinityFunctionBase.cs      |  139 +
 .../Cache/Affinity/Fair/FairAffinityFunction.cs |   33 -
 .../Cache/Affinity/Fair/Package-Info.cs         |   26 -
 .../Cache/Affinity/IAffinityFunction.cs         |    3 +-
 .../Rendezvous/RendezvousAffinityFunction.cs    |    1 -
 .../Apache.Ignite.Core/Cache/CachePeekMode.cs   |    7 +-
 .../Configuration/CacheAtomicWriteOrderMode.cs  |   43 -
 .../Cache/Configuration/CacheConfiguration.cs   |  236 +-
 .../Cache/Configuration/CacheMemoryMode.cs      |   60 -
 .../Cache/Configuration/DataPageEvictionMode.cs |   59 +
 .../Cache/Configuration/MemoryConfiguration.cs  |  172 +
 .../Configuration/MemoryPolicyConfiguration.cs  |  196 +
 .../Configuration/NearCacheConfiguration.cs     |    2 +-
 .../Cache/Configuration/PartitionLossPolicy.cs  |   68 +
 .../Cache/Configuration/QueryEntity.cs          |   96 +-
 .../Cache/Configuration/QueryField.cs           |    2 +-
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |   49 +-
 .../Cache/ICacheEntryProcessorResult.cs         |   12 +-
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |   88 -
 .../Apache.Ignite.Core/Cache/IMemoryMetrics.cs  |   55 +
 .../Cache/Query/Continuous/ContinuousQuery.cs   |   59 +-
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |    4 +-
 .../Cache/Query/SqlFieldsQuery.cs               |   43 +-
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |   31 +
 .../Store/CacheParallelLoadStoreAdapter.cs      |   38 +-
 .../Cache/Store/CacheStoreAdapter.cs            |   30 +-
 .../Cache/Store/ICacheStore.cs                  |   39 +-
 .../Cache/Store/ICacheStoreSession.cs           |    2 +-
 .../Common/ExceptionFactory.cs                  |   31 +
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |   47 +-
 .../Apache.Ignite.Core/Common/JavaException.cs  |   62 +-
 .../Compute/ComputeTaskAdapter.cs               |    4 +-
 .../Configuration/Package-Info.cs               |   26 +
 .../Configuration/SqlConnectorConfiguration.cs  |  159 +
 .../Datastream/IDataStreamer.cs                 |   10 +-
 .../Deployment/PeerAssemblyLoadingMode.cs       |   55 +
 .../Discovery/Tcp/TcpDiscoverySpi.cs            |   43 -
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |   10 +-
 .../Apache.Ignite.Core/Events/EventBase.cs      |    7 +-
 .../Apache.Ignite.Core/Events/EventReader.cs    |    1 -
 .../Apache.Ignite.Core/Events/EventType.cs      |   57 -
 .../Events/IEventStorageSpi.cs                  |   33 +
 .../Apache.Ignite.Core/Events/JobEvent.cs       |    4 +-
 .../Events/MemoryEventStorageSpi.cs             |  100 +
 .../Events/NoopEventStorageSpi.cs               |   27 +
 .../Apache.Ignite.Core/Events/SwapSpaceEvent.cs |   51 -
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |    2 +-
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |   44 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  445 +-
 .../IgniteConfigurationSection.xsd              |  510 +-
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |  153 +-
 .../Impl/Binary/BinarizableSerializer.cs        |    4 +-
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |   78 +-
 .../Impl/Binary/BinaryArrayEqualityComparer.cs  |  161 +
 .../Impl/Binary/BinaryEnum.cs                   |   25 +-
 .../Binary/BinaryEqualityComparerSerializer.cs  |   99 -
 .../Impl/Binary/BinaryFieldEqualityComparer.cs  |  138 -
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |   59 +-
 .../Impl/Binary/BinaryObject.cs                 |   87 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |   93 +-
 .../Impl/Binary/BinaryObjectHeader.cs           |   21 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |    2 +
 .../Impl/Binary/BinaryProcessor.cs              |  100 +-
 .../Impl/Binary/BinaryReader.cs                 |  124 +-
 .../Impl/Binary/BinaryReflectiveActions.cs      |  115 +-
 .../BinaryReflectiveSerializerInternal.cs       |   98 +-
 .../Binary/BinarySurrogateTypeDescriptor.cs     |   42 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |  360 +-
 .../Impl/Binary/BinarySystemTypeSerializer.cs   |    2 +-
 .../Impl/Binary/BinaryUtils.cs                  |  497 +-
 .../Impl/Binary/BinaryWriter.cs                 |  302 +-
 .../Impl/Binary/BinaryWriterExtensions.cs       |    9 +-
 .../Impl/Binary/DateTimeHolder.cs               |  101 -
 .../Impl/Binary/DateTimeSerializer.cs           |   48 -
 .../Binary/DeserializationCallbackProcessor.cs  |  113 +
 .../Impl/Binary/IBinaryEqualityComparer.cs      |   53 -
 .../Impl/Binary/IBinarySerializerInternal.cs    |    8 +-
 .../Impl/Binary/IBinaryTypeDescriptor.cs        |   13 +-
 .../Impl/Binary/Io/BinaryHeapStream.cs          |    3 +
 .../Impl/Binary/Io/BinaryStreamAdapter.cs       |  119 -
 .../Impl/Binary/Io/IBinaryStream.cs             |    3 +-
 .../Impl/Binary/Io/IBinaryStreamProcessor.cs    |    5 +-
 .../Impl/Binary/Marshaller.cs                   |  469 +-
 .../Impl/Binary/Metadata/BinaryField.cs         |   72 +
 .../Impl/Binary/Metadata/BinaryType.cs          |  215 +-
 .../Binary/Metadata/BinaryTypeHashsetHandler.cs |   10 +-
 .../Impl/Binary/Metadata/BinaryTypeHolder.cs    |   47 +-
 .../Impl/Binary/Metadata/IBinaryTypeHandler.cs  |    4 +-
 .../Impl/Binary/ReflectionUtils.cs              |   50 +
 .../Impl/Binary/SerializableObjectHolder.cs     |   96 -
 .../Impl/Binary/SerializableSerializer.cs       |  656 ++-
 .../Impl/Binary/TypeNameParser.cs               |  407 ++
 .../Impl/Binary/TypeResolver.cs                 |  193 +-
 .../Impl/Binary/UserSerializerProxy.cs          |    4 +-
 .../Impl/Cache/Affinity/AffinityFunctionBase.cs |  140 -
 .../Affinity/AffinityFunctionSerializer.cs      |   21 +-
 .../Impl/Cache/CacheEntryFilterHolder.cs        |   13 +
 .../Impl/Cache/CacheEntryProcessorResult.cs     |   27 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   91 +-
 .../Impl/Cache/CacheMetricsImpl.cs              |   77 -
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |    6 +-
 .../Impl/Cache/IQueryEntityInternal.cs          |   31 +
 .../Impl/Cache/MemoryMetrics.cs                 |   62 +
 .../Impl/Cache/Store/CacheStore.cs              |  233 +-
 .../Impl/Cache/Store/CacheStoreInternal.cs      |  285 ++
 .../Impl/Cache/Store/ICacheStoreInternal.cs     |   43 +
 .../Impl/Cluster/ClusterGroupImpl.cs            |   89 +
 .../Impl/Collections/MultiValueDictionary.cs    |    2 +-
 .../Common/CopyOnWriteConcurrentDictionary.cs   |   27 +
 .../Impl/Common/DelegateConverter.cs            |   91 +-
 .../Impl/Common/DelegateTypeDescriptor.cs       |    3 +-
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |   16 +-
 .../Impl/Common/IFutureConverter.cs             |    4 +-
 .../Impl/Common/IFutureInternal.cs              |    3 +-
 .../Common/IgniteConfigurationXmlSerializer.cs  |  287 +-
 .../Impl/Common/IgniteHome.cs                   |   22 +-
 .../Impl/Common/LoadedAssembliesResolver.cs     |   10 +-
 .../Impl/Common/SerializableTypeDescriptor.cs   |  222 +
 .../Closure/ComputeAbstractClosureTask.cs       |    3 +-
 .../Impl/Compute/Closure/ComputeActionJob.cs    |    7 +-
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |    5 +-
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  |   29 +
 .../Impl/Compute/ComputeFunc.cs                 |    7 +-
 .../Impl/Compute/ComputeImpl.cs                 |    3 +-
 .../Impl/Compute/ComputeJob.cs                  |    5 +-
 .../Impl/Compute/ComputeJobHolder.cs            |    9 +-
 .../Impl/Compute/ComputeOutFunc.cs              |    5 +-
 .../Impl/Compute/ComputeTaskHolder.cs           |    8 +-
 .../Impl/Deployment/AssemblyLoader.cs           |  105 +
 .../Impl/Deployment/AssemblyRequest.cs          |   68 +
 .../Impl/Deployment/AssemblyRequestResult.cs    |   80 +
 .../Impl/Deployment/GetAssemblyFunc.cs          |  105 +
 .../Impl/Deployment/PeerAssemblyResolver.cs     |  213 +
 .../Impl/Deployment/PeerLoadingExtensions.cs    |   65 +
 .../Impl/Deployment/PeerLoadingObjectHolder.cs  |   90 +
 .../PeerLoadingObjectHolderSerializer.cs        |   49 +
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   97 +-
 .../Apache.Ignite.Core/Impl/Handle/Handle.cs    |    2 +-
 .../Impl/Handle/HandleRegistry.cs               |    2 +-
 .../Apache.Ignite.Core/Impl/Handle/IHandle.cs   |    2 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |   99 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  464 --
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |   22 +-
 .../Impl/LifecycleBeanHolder.cs                 |   66 -
 .../Impl/LifecycleHandlerHolder.cs              |   66 +
 .../Impl/Memory/IPlatformMemory.cs              |    5 +-
 .../Impl/Memory/PlatformMemory.cs               |    3 +-
 .../Impl/Memory/PlatformMemoryManager.cs        |    4 +-
 .../Impl/Memory/PlatformMemoryPool.cs           |    3 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |    9 +-
 .../Impl/Memory/PlatformRawMemory.cs            |   96 -
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |   50 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |   46 +-
 .../Impl/Plugin/Cache/CachePluginContext.cs     |   82 -
 .../Impl/Plugin/Cache/CachePluginProcessor.cs   |   77 -
 .../Plugin/Cache/CachePluginProviderProxy.cs    |   75 -
 .../Plugin/Cache/ICachePluginProviderProxy.cs   |   52 -
 .../Impl/Plugin/PluginContext.cs                |   28 +
 .../Impl/Plugin/PluginProcessor.cs              |  128 +-
 .../Impl/Resource/ResourceProcessor.cs          |   10 -
 .../Impl/Resource/ResourceTypeDescriptor.cs     |   12 +-
 .../Impl/Services/ServiceProxySerializer.cs     |    4 +-
 .../Impl/SwapSpace/SwapSpaceSerializer.cs       |   99 -
 .../Transactions/CacheTransactionManager.cs     |   13 +
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |    3 +
 .../Impl/Unmanaged/UnmanagedCallbackOp.cs       |    5 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |   71 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   13 +-
 .../Interop/IPlatformTarget.cs                  |   12 +
 .../Lifecycle/ILifecycleBean.cs                 |   64 -
 .../Lifecycle/ILifecycleHandler.cs              |   64 +
 .../Lifecycle/LifecycleEventType.cs             |    2 +-
 .../NuGet/LINQPad/BinaryModeExample.linq        |    1 +
 .../NuGet/LINQPad/ComputeExample.linq           |   18 +-
 .../NuGet/LINQPad/PutGetExample.linq            |    6 +-
 .../NuGet/LINQPad/QueryExample.linq             |    6 +-
 .../PersistentStore/Package-Info.cs             |   26 +
 .../PersistentStoreConfiguration.cs             |  251 +
 .../PersistentStore/WalMode.cs                  |   45 +
 .../Cache/CachePluginProviderTypeAttribute.cs   |   52 -
 .../Plugin/Cache/ICachePluginConfiguration.cs   |   33 +-
 .../Plugin/Cache/ICachePluginContext.cs         |   47 -
 .../Plugin/Cache/ICachePluginProvider.cs        |   52 -
 .../Plugin/IPluginConfiguration.cs              |   19 +-
 .../Apache.Ignite.Core/Plugin/IPluginContext.cs |   24 +
 .../Apache.Ignite.Core/Plugin/PluginCallback.cs |   29 +
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../SwapSpace/File/FileSwapSpaceSpi.cs          |  105 -
 .../SwapSpace/ISwapSpaceSpi.cs                  |   35 -
 .../App.config                                  |    4 +-
 .../EntityFrameworkCacheInitializationTest.cs   |    2 +-
 .../EntityFrameworkCacheTest.cs                 |    2 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite.EntityFramework.nuspec        |    2 +-
 .../IgniteDbConfiguration.cs                    |    4 +-
 .../Impl/DbCache.cs                             |    2 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 modules/platforms/dotnet/Apache.Ignite.FxCop    |    3 +-
 .../Apache.Ignite.Linq.csproj                   |    6 +-
 .../Apache.Ignite.Linq.nuspec                   |    2 +-
 .../Apache.Ignite.Linq/CacheExtensions.cs       |   75 +
 .../dotnet/Apache.Ignite.Linq/CompiledQuery.cs  |  117 +-
 .../dotnet/Apache.Ignite.Linq/CompiledQuery2.cs |  257 -
 .../Apache.Ignite.Linq/Impl/AliasDictionary.cs  |   65 +-
 .../Impl/CacheFieldsQueryExecutor.cs            |   81 +-
 .../Impl/CacheFieldsQueryProvider.cs            |   39 +-
 .../Impl/CacheQueryExpressionVisitor.cs         |  213 +-
 .../Impl/CacheQueryModelVisitor.cs              |  195 +-
 .../Apache.Ignite.Linq/Impl/CacheQueryParser.cs |   46 +-
 .../Apache.Ignite.Linq/Impl/CacheQueryable.cs   |    3 +-
 .../Impl/CacheQueryableBase.cs                  |   14 +-
 .../Impl/Dml/RemoveAllExpressionNode.cs         |   88 +
 .../Impl/Dml/RemoveAllResultOperator.cs         |   61 +
 .../Apache.Ignite.Linq/Impl/EnumerableHelper.cs |   59 +
 .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs |   93 +-
 .../Impl/ICacheQueryableInternal.cs             |    6 -
 ...SequenceParameterNotNullExpressionVisitor.cs |   72 +
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |   79 +-
 .../dotnet/Apache.Ignite.Linq/Impl/QueryData.cs |   17 +-
 .../dotnet/Apache.Ignite.Linq/Impl/SqlTypes.cs  |   17 +-
 .../NuGet/LINQPad/QueryExample.linq             |    6 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../dotnet/Apache.Ignite.Linq/QueryOptions.cs   |   25 +-
 .../Apache.Ignite.Log4Net.nuspec                |    2 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../Apache.Ignite.NLog.nuspec                   |    2 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../dotnet/Apache.Ignite.sln.DotSettings        |    1 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |    1 -
 .../platforms/dotnet/Apache.Ignite/App.config   |   11 +
 .../dotnet/Apache.Ignite/Config/Configurator.cs |    9 +-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |    8 +-
 .../Apache.Ignite/Service/IgniteService.cs      |    6 +-
 .../Apache.Ignite/Service/ServiceDescription.cs |   32 -
 modules/platforms/dotnet/build.ps1              |   48 +-
 .../Apache.Ignite.Examples.csproj               |    1 +
 .../examples/Apache.Ignite.Examples/App.config  |   21 +-
 .../Compute/PeerAssemblyLoadingExample.cs       |   96 +
 .../Datagrid/LinqExample.cs                     |    2 +-
 .../Datagrid/MultiTieredCacheExample.cs         |   30 +-
 .../Datagrid/PutGetExample.cs                   |   13 +-
 .../Datagrid/StoreExample.cs                    |    1 +
 .../Misc/ClientReconnectExample.cs              |   40 +-
 .../Misc/LifecycleExample.cs                    |   12 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite.ExamplesDll/Binary/Address.cs |    2 -
 .../Compute/CharacterCountClosure.cs            |    1 -
 .../Datagrid/ContinuousQueryFilter.cs           |    2 -
 .../Datagrid/EmployeeStore.cs                   |   27 +-
 .../Datagrid/EmployeeStorePredicate.cs          |    2 -
 .../Messaging/RemoteOrderedListener.cs          |    1 -
 .../Messaging/RemoteUnorderedListener.cs        |    1 -
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Services/MapService.cs                      |    1 -
 modules/rest-http/pom.xml                       |    2 +-
 .../http/jetty/GridJettyObjectMapper.java       |  165 +-
 modules/rocketmq/README.txt                     |   25 +
 modules/rocketmq/licenses/apache-2.0.txt        |  202 +
 modules/rocketmq/pom.xml                        |   81 +
 .../stream/rocketmq/RocketMQStreamer.java       |  151 +
 .../ignite/stream/rocketmq/package-info.java    |   21 +
 .../stream/rocketmq/RocketMQStreamerTest.java   |  214 +
 .../rocketmq/RocketMQStreamerTestSuite.java     |   37 +
 .../stream/rocketmq/TestRocketMQServer.java     |  148 +
 .../ignite/stream/rocketmq/package-info.java    |   21 +
 modules/scalar-2.10/pom.xml                     |    4 +-
 modules/scalar/pom.xml                          |    4 +-
 .../scalar/pimps/ScalarProjectionPimp.scala     |   24 +-
 .../scala/org/apache/ignite/scalar/scalar.scala |   34 +-
 .../scalar/src/test/resources/spring-cache.xml  |    4 +-
 .../test/resources/spring-ping-pong-partner.xml |    2 +-
 .../scalar/tests/ScalarCacheQueriesSpec.scala   |    2 +-
 .../scalar/tests/ScalarProjectionSpec.scala     |    4 +-
 modules/schedule/pom.xml                        |    2 +-
 .../schedule/IgniteScheduleProcessor.java       |    2 +-
 .../processors/schedule/ScheduleFutureImpl.java |   73 +-
 .../schedule/GridScheduleSelfTest.java          |   84 +-
 modules/schema-import-db/README.txt             |    4 -
 modules/schema-import-db/pom.xml                |   55 -
 .../apache/ignite/schema/parser/DbColumn.java   |   88 -
 .../ignite/schema/parser/DbMetadataReader.java  |  147 -
 .../apache/ignite/schema/parser/DbTable.java    |   82 -
 .../parser/dialect/DB2MetadataDialect.java      |   33 -
 .../parser/dialect/DatabaseMetadataDialect.java |  102 -
 .../parser/dialect/JdbcMetadataDialect.java     |  199 -
 .../parser/dialect/MySQLMetadataDialect.java    |   82 -
 .../parser/dialect/OracleMetadataDialect.java   |  360 --
 modules/schema-import/README.txt                |  216 -
 modules/schema-import/pom.xml                   |  119 -
 .../main/java/media/data_connection_48x48.png   |  Bin 4443 -> 0 bytes
 .../src/main/java/media/error_48x48.png         |  Bin 4349 -> 0 bytes
 .../src/main/java/media/ignite_128x128.png      |  Bin 4917 -> 0 bytes
 .../src/main/java/media/ignite_16x16.png        |  Bin 608 -> 0 bytes
 .../src/main/java/media/ignite_24x24.png        |  Bin 930 -> 0 bytes
 .../src/main/java/media/ignite_32x32.png        |  Bin 1203 -> 0 bytes
 .../src/main/java/media/ignite_48x48.png        |  Bin 1868 -> 0 bytes
 .../src/main/java/media/ignite_64x64.png        |  Bin 2453 -> 0 bytes
 .../src/main/java/media/information_48x48.png   |  Bin 4102 -> 0 bytes
 .../src/main/java/media/question_48x48.png      |  Bin 3857 -> 0 bytes
 .../src/main/java/media/sign_warning_48x48.png  |  Bin 2988 -> 0 bytes
 .../schema-import/src/main/java/media/style.css |  134 -
 .../src/main/java/media/text_tree_48x48.png     |  Bin 2567 -> 0 bytes
 .../ignite/schema/generator/CodeGenerator.java  |  831 ---
 .../ignite/schema/generator/GeneratorUtils.java |   70 -
 .../ignite/schema/generator/XmlGenerator.java   |  446 --
 .../ignite/schema/model/PojoDescriptor.java     |  497 --
 .../apache/ignite/schema/model/PojoField.java   |  461 --
 .../ignite/schema/model/SchemaDescriptor.java   |   62 -
 .../schema/parser/DatabaseMetadataParser.java   |  115 -
 .../ignite/schema/ui/ConfirmCallable.java       |   83 -
 .../org/apache/ignite/schema/ui/Controls.java   |  697 ---
 .../org/apache/ignite/schema/ui/GridPaneEx.java |  180 -
 .../org/apache/ignite/schema/ui/MessageBox.java |  274 -
 .../apache/ignite/schema/ui/ModalDialog.java    |   50 -
 .../ignite/schema/ui/SchemaImportApp.java       | 2115 --------
 .../ignite/schema/ui/TextColumnValidator.java   |   32 -
 .../schema/test/AbstractSchemaImportTest.java   |  181 -
 .../test/generator/CodeGeneratorTest.java       |   81 -
 .../schema/test/generator/XmlGeneratorTest.java |   42 -
 .../ignite/schema/test/model/CacheConfig.txt    |  409 --
 .../apache/ignite/schema/test/model/Objects.txt |  531 --
 .../ignite/schema/test/model/ObjectsKey.txt     |   96 -
 .../ignite/schema/test/model/Primitives.txt     |  535 --
 .../ignite/schema/test/model/PrimitivesKey.txt  |   96 -
 .../org/apache/ignite/schema/test/model/Tst.txt |  535 --
 .../apache/ignite/schema/test/model/TstKey.txt  |   96 -
 .../schema/test/model/ignite-type-metadata.xml  |  666 ---
 .../test/parser/DbMetadataParserTest.java       |  134 -
 .../testsuites/IgniteSchemaImportTestSuite.java |   42 -
 modules/slf4j/pom.xml                           |    2 +-
 modules/spark-2.10/pom.xml                      |   56 +-
 modules/spark/pom.xml                           |  179 +-
 .../org/apache/ignite/spark/IgniteContext.scala |   24 +-
 .../apache/ignite/spark/JavaIgniteContext.scala |    6 +
 .../spark/JavaEmbeddedIgniteRDDSelfTest.java    |   26 +-
 .../spark/JavaStandaloneIgniteRDDSelfTest.java  |   71 +-
 .../org/apache/ignite/spark/IgniteRDDSpec.scala |   90 +-
 modules/spring-data/README.txt                  |   32 +
 modules/spring-data/licenses/apache-2.0.txt     |  202 +
 modules/spring-data/pom.xml                     |   78 +
 .../springdata/repository/IgniteRepository.java |   58 +
 .../config/EnableIgniteRepositories.java        |  119 +
 .../config/IgniteRepositoriesRegistar.java      |   36 +
 .../IgniteRepositoryConfigurationExtension.java |   49 +
 .../springdata/repository/config/Query.java     |   37 +
 .../repository/config/RepositoryConfig.java     |   39 +
 .../repository/config/package-info.java         |   22 +
 .../springdata/repository/package-info.java     |   22 +
 .../repository/query/IgniteQuery.java           |   83 +
 .../repository/query/IgniteQueryGenerator.java  |  243 +
 .../repository/query/IgniteRepositoryQuery.java |  306 ++
 .../repository/query/package-info.java          |   22 +
 .../support/IgniteRepositoryFactory.java        |  168 +
 .../support/IgniteRepositoryFactoryBean.java    |   92 +
 .../support/IgniteRepositoryImpl.java           |  160 +
 .../repository/support/package-info.java        |   22 +
 .../IgniteSpringDataCrudSelfTest.java           |  232 +
 .../IgniteSpringDataQueriesSelfTest.java        |  291 ++
 .../misc/ApplicationConfiguration.java          |   54 +
 .../apache/ignite/springdata/misc/Person.java   |   97 +
 .../springdata/misc/PersonRepository.java       |   92 +
 .../springdata/misc/PersonSecondRepository.java |   40 +
 .../testsuites/IgniteSpringDataTestSuite.java   |   41 +
 modules/spring/pom.xml                          |   15 +-
 .../org/apache/ignite/IgniteSpringBean.java     |   85 +-
 .../apache/ignite/cache/spring/SpringCache.java |   60 +-
 .../ignite/cache/spring/SpringCacheManager.java |   93 +-
 .../spring/SpringTransactionManager.java        |   91 +-
 .../src/test/config/jdbc-pojo-store-builtin.xml |    2 +-
 .../src/test/config/jdbc-pojo-store-obj.xml     |    2 +-
 modules/spring/src/test/config/node1.xml        |    2 +-
 modules/spring/src/test/config/store-cache1.xml |    2 +-
 .../src/test/java/config/ignite-test-config.xml |    2 +-
 .../test/java/config/spring-transactions.xml    |    2 +-
 .../GridSpringCacheManagerMultiJvmSelfTest.java |  134 +
 .../spring/GridSpringCacheManagerSelfTest.java  |   12 +-
 .../GridSpringDynamicCacheTestService.java      |   13 +
 .../SpringCacheManagerContextInjectionTest.java |    2 +-
 .../ignite/cache/spring/SpringCacheTest.java    |  177 +
 .../ignite/cache/spring/spring-caching.xml      |    2 +-
 .../ignite/cache/spring/spring-caching1.xml     |   56 +
 .../ignite/cache/spring/spring-caching2.xml     |   56 +
 .../jdbc/CacheJdbcBlobStoreFactorySelfTest.java |    2 +-
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |    6 +-
 .../store/jdbc/CachePojoStoreXmlSelfTest.java   |    4 +-
 .../ignite/internal/GridFactorySelfTest.java    |  157 +-
 .../GridSpringBeanSerializationSelfTest.java    |   17 +-
 .../internal/IgniteDynamicCacheConfigTest.java  |   10 +-
 .../java/org/apache/ignite/internal/cache.xml   |    3 -
 .../apache/ignite/internal/filtered-cache.xml   |    3 -
 .../apache/ignite/internal/invalid-cache.xml    |    6 -
 .../GridSpringResourceInjectionSelfTest.java    |   24 +-
 .../GridTransformSpringInjectionSelfTest.java   |    4 +-
 .../p2p/GridP2PUserVersionChangeSelfTest.java   |   14 +-
 .../org/apache/ignite/spring-injection-test.xml |    2 +-
 .../IgniteExcludeInConfigurationTest.java       |   14 +-
 .../IgniteStartFromStreamConfigurationTest.java |    4 +-
 .../spring/injection/spring-resource.tmpl.xml   |    2 +-
 .../org/apache/ignite/spring/sprint-exclude.xml |   23 +-
 .../testsuites/IgniteSpringTestSuite.java       |    3 +
 .../GridSpringTransactionManagerSelfTest.java   |   91 +-
 ...gTransactionManagerContextInjectionTest.java |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 ...gniteProjectionStartStopRestartSelfTest.java |    6 +-
 modules/storm/pom.xml                           |    2 +-
 modules/tools/pom.xml                           |    2 +-
 .../ant/beautifier/GridJavadocAntTask.java      |   19 +-
 .../ignite/tools/classgen/ClassesGenerator.java |    4 +-
 modules/twitter/pom.xml                         |    2 +-
 .../twitter/IgniteTwitterStreamerTest.java      |    8 +-
 modules/urideploy/pom.xml                       |    2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |   90 +-
 .../scanners/UriDeploymentScannerManager.java   |   12 +-
 .../scanners/http/UriDeploymentHttpScanner.java |    3 +-
 .../GridTaskUriDeploymentDeadlockSelfTest.java  |    6 +-
 .../ignite/p2p/GridP2PDisabledSelfTest.java     |    8 +-
 .../uri/GridUriDeploymentMd5CheckSelfTest.java  |    9 +-
 modules/visor-console-2.10/pom.xml              |    2 +-
 modules/visor-console/pom.xml                   |    2 +-
 .../ignite/visor/commands/VisorConsole.scala    |    1 -
 .../visor/commands/ack/VisorAckCommand.scala    |    5 +-
 .../commands/cache/VisorCacheClearCommand.scala |    7 +-
 .../commands/cache/VisorCacheCommand.scala      |  347 +-
 .../commands/cache/VisorCacheResetCommand.scala |    4 +-
 .../commands/cache/VisorCacheScanCommand.scala  |   19 +-
 .../commands/cache/VisorCacheStopCommand.scala  |    4 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |  145 -
 .../config/VisorConfigurationCommand.scala      |  185 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |   24 +-
 .../commands/events/VisorEventsCommand.scala    |   38 +-
 .../visor/commands/gc/VisorGcCommand.scala      |    4 +-
 .../visor/commands/node/VisorNodeCommand.scala  |    6 +-
 .../commands/tasks/VisorTasksCommand.scala      |   61 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   49 +-
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |    4 +-
 .../commands/alert/VisorAlertCommandSpec.scala  |    4 +-
 .../cache/VisorCacheClearCommandSpec.scala      |   28 +-
 .../commands/cache/VisorCacheCommandSpec.scala  |   10 +-
 .../cache/VisorCacheResetCommandSpec.scala      |   12 +-
 .../config/VisorConfigurationCommandSpec.scala  |    4 +-
 .../cswap/VisorCacheSwapCommandSpec.scala       |   89 -
 .../disco/VisorDiscoveryCommandSpec.scala       |    4 +-
 .../events/VisorEventsCommandSpec.scala         |    4 +-
 .../visor/commands/gc/VisorGcCommandSpec.scala  |    4 +-
 .../commands/open/VisorOpenCommandSpec.scala    |    6 +-
 .../commands/tasks/VisorTasksCommandSpec.scala  |   26 +-
 .../testsuites/VisorConsoleSelfTestSuite.scala  |    2 -
 modules/visor-plugins/pom.xml                   |    2 +-
 .../src/main/java/log4j.properties              |    2 +-
 modules/web-console/backend/.gitignore          |    2 +
 modules/web-console/backend/app/agent.js        |  854 ----
 modules/web-console/backend/app/agentSocket.js  |  249 +
 .../web-console/backend/app/agentsHandler.js    |  404 ++
 modules/web-console/backend/app/apiServer.js    |   68 +
 modules/web-console/backend/app/app.js          |   63 -
 modules/web-console/backend/app/browser.js      |  469 --
 .../web-console/backend/app/browsersHandler.js  |  307 ++
 modules/web-console/backend/app/mongo.js        |  185 +-
 modules/web-console/backend/app/routes.js       |    6 +-
 modules/web-console/backend/app/settings.js     |    8 +-
 modules/web-console/backend/index.js            |   53 +-
 modules/web-console/backend/package.json        |   52 +-
 .../web-console/backend/routes/activities.js    |    7 -
 modules/web-console/backend/routes/admin.js     |   11 +-
 modules/web-console/backend/routes/agent.js     |   57 -
 modules/web-console/backend/routes/demo.js      |   14 +-
 modules/web-console/backend/routes/downloads.js |   57 +
 modules/web-console/backend/routes/profile.js   |    4 +-
 .../web-console/backend/services/activities.js  |   30 +-
 modules/web-console/backend/services/agents.js  |   83 -
 modules/web-console/backend/services/caches.js  |    2 +
 .../web-console/backend/services/downloads.js   |   80 +
 modules/web-console/backend/services/igfss.js   |    5 +
 .../backend/services/notifications.js           |   52 +
 .../web-console/backend/services/sessions.js    |    6 +-
 modules/web-console/backend/services/users.js   |   87 +-
 .../web-console/backend/test/app/httpAgent.js   |    6 +-
 .../web-console/backend/test/routes/clusters.js |    4 +-
 .../backend/test/unit/ActivitiesService.test.js |  131 +
 .../docker/compose/docker-compose.yml           |    4 +-
 .../docker/compose/frontend/DockerfileBuild     |    2 +-
 .../web-console/docker/standalone/Dockerfile    |   10 +-
 .../docker/standalone/docker-compose.yml        |    2 +-
 modules/web-console/frontend/.babelrc           |    9 +-
 modules/web-console/frontend/.eslintrc          |   11 +-
 modules/web-console/frontend/.gitignore         |    4 +-
 modules/web-console/frontend/app/app.config.js  |   21 +-
 modules/web-console/frontend/app/app.js         |   77 +-
 .../activities-user-dialog.controller.js        |   39 +-
 .../activities-user-dialog.jade                 |   36 -
 .../activities-user-dialog.tpl.pug              |   42 +
 .../components/activities-user-dialog/index.js  |    8 +-
 .../app/components/bs-select-menu/controller.js |   28 +
 .../app/components/bs-select-menu/directive.js  |   30 +
 .../app/components/bs-select-menu/index.js      |   24 +
 .../app/components/bs-select-menu/style.scss    |   78 +
 .../app/components/bs-select-menu/template.pug  |   49 +
 .../cluster-select/cluster-select.controller.js |   66 +
 .../cluster-select/cluster-select.pug           |   45 +
 .../cluster-select/cluster-select.scss          |   30 +
 .../app/components/cluster-select/index.js      |   29 +
 .../form-field-datepicker.jade                  |   55 -
 .../form-field-datepicker.scss                  |   20 -
 .../grid-column-selector/component.js           |   29 +
 .../grid-column-selector/controller.js          |  111 +
 .../grid-column-selector/controller.spec.js     |  435 ++
 .../components/grid-column-selector/index.js    |   24 +
 .../components/grid-column-selector/style.scss  |   24 +
 .../grid-column-selector/template.pug           |   28 +
 .../app/components/ignite-icon/directive.js     |   67 +
 .../app/components/ignite-icon/index.js         |   25 +
 .../app/components/ignite-icon/style.scss       |   21 +
 .../app/components/input-dialog/index.js        |   24 +
 .../input-dialog/input-dialog.controller.js     |   35 +
 .../input-dialog/input-dialog.service.js        |   87 +
 .../input-dialog/input-dialog.tpl.pug           |   39 +
 .../list-of-registered-users/index.js           |    4 +-
 .../list-of-registered-users.categories.js      |   22 +-
 .../list-of-registered-users.column-defs.js     |   52 +-
 .../list-of-registered-users.controller.js      |  356 +-
 .../list-of-registered-users.jade               |   54 -
 .../list-of-registered-users.scss               |   35 +
 .../list-of-registered-users.tpl.pug            |   67 +
 .../page-configure-advanced/component.js        |   25 +
 .../page-configure-advanced/controller.js       |   39 +
 .../components/page-configure-advanced/index.js |   25 +
 .../page-configure-advanced/service.js          |   31 +
 .../page-configure-advanced/style.scss          |   82 +
 .../page-configure-advanced/template.pug        |   26 +
 .../page-configure-basic/component.js           |   25 +
 .../components/pcbScaleNumber.js                |   45 +
 .../page-configure-basic/controller.js          |  135 +
 .../page-configure-basic/controller.spec.js     |  207 +
 .../components/page-configure-basic/index.js    |   29 +
 .../mixins/pcb-form-field-size.pug              |   70 +
 .../components/page-configure-basic/reducer.js  |  112 +
 .../page-configure-basic/reducer.spec.js        |  191 +
 .../components/page-configure-basic/service.js  |  134 +
 .../page-configure-basic/service.spec.js        |  323 ++
 .../components/page-configure-basic/style.scss  |  143 +
 .../page-configure-basic/template.pug           |  175 +
 .../app/components/page-configure/component.js  |   25 +
 .../app/components/page-configure/controller.js |   30 +
 .../page-configure/controller.spec.js           |   48 +
 .../app/components/page-configure/index.js      |   29 +
 .../app/components/page-configure/reducer.js    |   83 +
 .../components/page-configure/reducer.spec.js   |  275 +
 .../services/ConfigurationDownload.js           |   67 +
 .../services/ConfigurationDownload.spec.js      |  116 +
 .../page-configure/services/ConfigureState.js   |   92 +
 .../page-configure/services/PageConfigure.js    |   64 +
 .../app/components/page-configure/style.scss    |   23 +
 .../app/components/page-configure/template.pug  |   30 +
 .../protect-from-bs-select-render/directive.js  |   32 +
 .../protect-from-bs-select-render/index.js      |   24 +
 .../ui-grid-header/ui-grid-header.jade          |   27 -
 .../ui-grid-header/ui-grid-header.scss          |   84 -
 .../ui-grid-settings/ui-grid-settings.jade      |   33 -
 .../ui-grid-settings/ui-grid-settings.scss      |   70 -
 .../components/user-notifications/controller.js |   55 +
 .../app/components/user-notifications/index.js  |   26 +
 .../components/user-notifications/service.js    |   58 +
 .../components/user-notifications/style.scss    |   55 +
 .../user-notifications/template.tpl.pug         |   42 +
 .../app/components/version-picker/component.js  |   52 +
 .../app/components/version-picker/index.js      |   25 +
 .../app/components/version-picker/style.scss    |   41 +
 .../app/components/version-picker/template.pug  |   33 +
 .../components/web-console-footer/component.js  |   23 +
 .../app/components/web-console-footer/index.js  |   23 +
 .../components/web-console-footer/style.scss    |   55 +
 .../components/web-console-footer/template.pug  |   19 +
 .../components/web-console-header/component.js  |   47 +
 .../app/components/web-console-header/index.js  |   23 +
 .../components/web-console-header/style.scss    |  165 +
 .../components/web-console-header/template.pug  |   36 +
 .../app/core/activities/Activities.data.js      |    5 -
 .../frontend/app/core/admin/Admin.data.js       |    2 +
 .../frontend/app/core/utils/maskNull.js         |   21 +
 modules/web-console/frontend/app/data/i18n.js   |    1 +
 .../frontend/app/data/pom-dependencies.json     |    1 +
 .../frontend/app/directives/btn-ignite-link.js  |   27 +
 .../app/directives/centered/centered.css        |   37 -
 .../directives/centered/centered.directive.js   |    2 +-
 .../app/directives/centered/centered.scss       |   37 +
 .../information/information.directive.js        |    4 +-
 .../app/directives/information/information.jade |   20 -
 .../app/directives/information/information.pug  |   20 +
 .../ui-ace-docker/ui-ace-docker.controller.js   |    8 +-
 .../ui-ace-docker/ui-ace-docker.directive.js    |    4 +-
 .../directives/ui-ace-docker/ui-ace-docker.jade |   31 -
 .../directives/ui-ace-docker/ui-ace-docker.pug  |   31 +
 .../ui-ace-java/ui-ace-java.controller.js       |   29 +-
 .../ui-ace-java/ui-ace-java.directive.js        |   16 +-
 .../app/directives/ui-ace-java/ui-ace-java.jade |   22 -
 .../app/directives/ui-ace-java/ui-ace-java.pug  |   22 +
 .../ui-ace-pojos/ui-ace-pojos.directive.js      |    4 +-
 .../directives/ui-ace-pojos/ui-ace-pojos.jade   |   40 -
 .../directives/ui-ace-pojos/ui-ace-pojos.pug    |   40 +
 .../ui-ace-pom/ui-ace-pom.controller.js         |    8 +-
 .../ui-ace-pom/ui-ace-pom.directive.js          |    4 +-
 .../app/directives/ui-ace-pom/ui-ace-pom.jade   |   17 -
 .../app/directives/ui-ace-pom/ui-ace-pom.pug    |   17 +
 .../ui-ace-sharp/ui-ace-sharp.directive.js      |    4 +-
 .../directives/ui-ace-sharp/ui-ace-sharp.jade   |   22 -
 .../directives/ui-ace-sharp/ui-ace-sharp.pug    |   22 +
 .../ui-ace-spring/ui-ace-spring.controller.js   |   29 +-
 .../ui-ace-spring/ui-ace-spring.directive.js    |   16 +-
 .../directives/ui-ace-spring/ui-ace-spring.jade |   17 -
 .../directives/ui-ace-spring/ui-ace-spring.pug  |   17 +
 .../frontend/app/filters/byName.filter.js       |    4 +-
 .../app/filters/domainsValidation.filter.js     |    4 +-
 .../frontend/app/filters/duration.filter.js     |    4 +-
 .../frontend/app/filters/hasPojo.filter.js      |    4 +-
 .../frontend/app/helpers/jade/form.jade         |   28 -
 .../frontend/app/helpers/jade/form.pug          |   28 +
 .../helpers/jade/form/form-field-checkbox.jade  |   38 -
 .../helpers/jade/form/form-field-checkbox.pug   |   39 +
 .../helpers/jade/form/form-field-datalist.jade  |   51 -
 .../helpers/jade/form/form-field-datalist.pug   |   51 +
 .../app/helpers/jade/form/form-field-down.jade  |   18 -
 .../app/helpers/jade/form/form-field-down.pug   |   18 +
 .../helpers/jade/form/form-field-dropdown.jade  |   51 -
 .../helpers/jade/form/form-field-dropdown.pug   |   49 +
 .../helpers/jade/form/form-field-feedback.jade  |   32 -
 .../helpers/jade/form/form-field-feedback.pug   |   32 +
 .../app/helpers/jade/form/form-field-label.jade |   23 -
 .../app/helpers/jade/form/form-field-label.pug  |   23 +
 .../helpers/jade/form/form-field-number.jade    |   53 -
 .../app/helpers/jade/form/form-field-number.pug |   52 +
 .../helpers/jade/form/form-field-password.jade  |   47 -
 .../helpers/jade/form/form-field-password.pug   |   47 +
 .../app/helpers/jade/form/form-field-text.jade  |   64 -
 .../app/helpers/jade/form/form-field-text.pug   |   47 +
 .../app/helpers/jade/form/form-field-up.jade    |   18 -
 .../app/helpers/jade/form/form-field-up.pug     |   18 +
 .../app/helpers/jade/form/form-group.jade       |   23 -
 .../app/helpers/jade/form/form-group.pug        |   23 +
 .../frontend/app/helpers/jade/mixins.jade       |  609 ---
 .../frontend/app/helpers/jade/mixins.pug        |  630 +++
 .../app/modules/agent/AgentManager.service.js   |  661 +++
 .../app/modules/agent/AgentModal.service.js     |   89 +
 .../frontend/app/modules/agent/agent.module.js  |  323 +-
 .../app/modules/branding/branding.provider.js   |    4 +-
 .../modules/branding/header-logo.directive.js   |    4 +-
 .../app/modules/branding/header-logo.jade       |   18 -
 .../app/modules/branding/header-logo.pug        |   18 +
 .../modules/branding/header-title.directive.js  |    6 +-
 .../branding/powered-by-apache.directive.js     |    7 +-
 .../app/modules/branding/powered-by-apache.jade |   18 -
 .../app/modules/branding/powered-by-apache.pug  |   18 +
 .../frontend/app/modules/cluster/Cache.js       |   52 +
 .../app/modules/cluster/CacheMetrics.js         |   59 +
 .../frontend/app/modules/cluster/Node.js        |   54 +
 .../frontend/app/modules/cluster/NodeMetrics.js |   19 +
 .../modules/configuration/Sidebar.provider.js   |   39 -
 .../modules/configuration/Version.service.js    |  176 +-
 .../configuration/configuration.module.js       |   10 +-
 .../generator/AbstractTransformer.js            |  118 +-
 .../modules/configuration/generator/Beans.js    |    8 +
 .../generator/ConfigurationGenerator.js         |  687 ++-
 .../configuration/generator/Docker.service.js   |   19 +-
 .../generator/JavaTransformer.service.js        |   41 +-
 .../configuration/generator/Maven.service.js    |  169 +-
 .../generator/Properties.service.js             |   63 +-
 .../generator/SpringTransformer.service.js      |   23 +-
 .../generator/defaults/Cache.service.js         |   11 +-
 .../generator/defaults/Cluster.service.js       |   75 +
 .../generator/defaults/IGFS.service.js          |    3 +-
 .../modules/configuration/sidebar.directive.js  |   30 -
 .../frontend/app/modules/demo/Demo.module.js    |   16 +-
 .../app/modules/dialog/dialog.factory.js        |    3 +-
 .../frontend/app/modules/dialog/dialog.jade     |   26 -
 .../frontend/app/modules/dialog/dialog.tpl.pug  |   26 +
 .../app/modules/form/field/down.directive.js    |    2 +-
 .../app/modules/form/field/up.directive.js      |    2 +-
 .../getting-started/GettingStarted.provider.js  |    3 +-
 .../frontend/app/modules/loading/loading.css    |   73 -
 .../app/modules/loading/loading.directive.js    |    8 +-
 .../frontend/app/modules/loading/loading.jade   |   23 -
 .../frontend/app/modules/loading/loading.pug    |   23 +
 .../frontend/app/modules/loading/loading.scss   |   73 +
 .../app/modules/navbar/userbar.directive.js     |    4 +-
 .../frontend/app/modules/nodes/Nodes.service.js |    5 +-
 .../app/modules/nodes/nodes-dialog.jade         |   35 -
 .../app/modules/nodes/nodes-dialog.tpl.pug      |   35 +
 .../app/modules/sql/Notebook.service.js         |    2 +-
 .../app/modules/sql/notebook.controller.js      |    6 +-
 .../frontend/app/modules/sql/sql.controller.js  |  229 +-
 .../frontend/app/modules/sql/sql.module.js      |   19 +-
 .../frontend/app/modules/states/admin.state.js  |   27 +-
 .../app/modules/states/configuration.state.js   |  100 +-
 .../states/configuration/caches/affinity.jade   |   82 -
 .../states/configuration/caches/affinity.pug    |   89 +
 .../configuration/caches/client-near-cache.jade |   50 -
 .../configuration/caches/client-near-cache.pug  |   50 +
 .../configuration/caches/concurrency.jade       |   65 -
 .../states/configuration/caches/concurrency.pug |   68 +
 .../states/configuration/caches/general.jade    |   69 -
 .../states/configuration/caches/general.pug     |  119 +
 .../states/configuration/caches/memory.jade     |  109 -
 .../states/configuration/caches/memory.pug      |  130 +
 .../configuration/caches/near-cache-client.jade |   51 -
 .../configuration/caches/near-cache-client.pug  |   51 +
 .../configuration/caches/near-cache-server.jade |   52 -
 .../configuration/caches/near-cache-server.pug  |   52 +
 .../configuration/caches/node-filter.jade       |   59 -
 .../states/configuration/caches/node-filter.pug |   52 +
 .../states/configuration/caches/query.jade      |  114 -
 .../states/configuration/caches/query.pug       |  131 +
 .../states/configuration/caches/rebalance.jade  |   66 -
 .../states/configuration/caches/rebalance.pug   |   66 +
 .../states/configuration/caches/statistics.jade |   39 -
 .../states/configuration/caches/statistics.pug  |   39 +
 .../states/configuration/caches/store.jade      |  250 -
 .../states/configuration/caches/store.pug       |  255 +
 .../states/configuration/clusters/atomic.jade   |   54 -
 .../states/configuration/clusters/atomic.pug    |   79 +
 .../configuration/clusters/attributes.jade      |   57 -
 .../configuration/clusters/attributes.pug       |   57 +
 .../states/configuration/clusters/binary.jade   |   77 -
 .../states/configuration/clusters/binary.pug    |   77 +
 .../configuration/clusters/cache-key-cfg.jade   |   50 -
 .../configuration/clusters/cache-key-cfg.pug    |   50 +
 .../configuration/clusters/checkpoint.jade      |   86 -
 .../configuration/clusters/checkpoint.pug       |   86 +
 .../configuration/clusters/checkpoint/fs.jade   |   66 -
 .../configuration/clusters/checkpoint/fs.pug    |   66 +
 .../configuration/clusters/checkpoint/jdbc.jade |   48 -
 .../configuration/clusters/checkpoint/jdbc.pug  |   48 +
 .../configuration/clusters/checkpoint/s3.jade   |  178 -
 .../configuration/clusters/checkpoint/s3.pug    |  178 +
 .../configuration/clusters/collision.jade       |   63 -
 .../states/configuration/clusters/collision.pug |   63 +
 .../clusters/collision/custom.jade              |   24 -
 .../configuration/clusters/collision/custom.pug |   24 +
 .../clusters/collision/fifo-queue.jade          |   27 -
 .../clusters/collision/fifo-queue.pug           |   27 +
 .../clusters/collision/job-stealing.jade        |   63 -
 .../clusters/collision/job-stealing.pug         |   63 +
 .../clusters/collision/priority-queue.jade      |   42 -
 .../clusters/collision/priority-queue.pug       |   42 +
 .../configuration/clusters/communication.jade   |  100 -
 .../configuration/clusters/communication.pug    |  100 +
 .../configuration/clusters/connector.jade       |  104 -
 .../states/configuration/clusters/connector.pug |  104 +
 .../configuration/clusters/deployment.jade      |  237 -
 .../configuration/clusters/deployment.pug       |  243 +
 .../configuration/clusters/discovery.jade       |   88 -
 .../states/configuration/clusters/discovery.pug |   92 +
 .../states/configuration/clusters/events.jade   |   68 -
 .../states/configuration/clusters/events.pug    |   70 +
 .../states/configuration/clusters/failover.jade |   73 -
 .../states/configuration/clusters/failover.pug  |   82 +
 .../states/configuration/clusters/general.jade  |   76 -
 .../states/configuration/clusters/general.pug   |   89 +
 .../clusters/general/discovery/cloud.jade       |  134 -
 .../clusters/general/discovery/cloud.pug        |  138 +
 .../clusters/general/discovery/google.jade      |   38 -
 .../clusters/general/discovery/google.pug       |   38 +
 .../clusters/general/discovery/jdbc.jade        |   31 -
 .../clusters/general/discovery/jdbc.pug         |   32 +
 .../clusters/general/discovery/kubernetes.pug   |   38 +
 .../clusters/general/discovery/multicast.jade   |   99 -
 .../clusters/general/discovery/multicast.pug    |  102 +
 .../clusters/general/discovery/s3.jade          |   27 -
 .../clusters/general/discovery/s3.pug           |   28 +
 .../clusters/general/discovery/shared.jade      |   23 -
 .../clusters/general/discovery/shared.pug       |   24 +
 .../clusters/general/discovery/vm.jade          |   79 -
 .../clusters/general/discovery/vm.pug           |   82 +
 .../clusters/general/discovery/zookeeper.jade   |   85 -
 .../clusters/general/discovery/zookeeper.pug    |   89 +
 .../bounded-exponential-backoff.jade            |   27 -
 .../retrypolicy/bounded-exponential-backoff.pug |   27 +
 .../discovery/zookeeper/retrypolicy/custom.jade |   24 -
 .../discovery/zookeeper/retrypolicy/custom.pug  |   24 +
 .../retrypolicy/exponential-backoff.jade        |   27 -
 .../retrypolicy/exponential-backoff.pug         |   27 +
 .../zookeeper/retrypolicy/forever.jade          |   22 -
 .../discovery/zookeeper/retrypolicy/forever.pug |   22 +
 .../zookeeper/retrypolicy/n-times.jade          |   25 -
 .../discovery/zookeeper/retrypolicy/n-times.pug |   25 +
 .../zookeeper/retrypolicy/one-time.jade         |   23 -
 .../zookeeper/retrypolicy/one-time.pug          |   23 +
 .../zookeeper/retrypolicy/until-elapsed.jade    |   25 -
 .../zookeeper/retrypolicy/until-elapsed.pug     |   25 +
 .../states/configuration/clusters/hadoop.pug    |  119 +
 .../states/configuration/clusters/igfs.jade     |   38 -
 .../states/configuration/clusters/igfs.pug      |   38 +
 .../configuration/clusters/load-balancing.jade  |  107 -
 .../configuration/clusters/load-balancing.pug   |  107 +
 .../states/configuration/clusters/logger.jade   |   66 -
 .../states/configuration/clusters/logger.pug    |   66 +
 .../configuration/clusters/logger/custom.jade   |   25 -
 .../configuration/clusters/logger/custom.pug    |   25 +
 .../configuration/clusters/logger/log4j.jade    |   50 -
 .../configuration/clusters/logger/log4j.pug     |   50 +
 .../configuration/clusters/logger/log4j2.jade   |   39 -
 .../configuration/clusters/logger/log4j2.pug    |   39 +
 .../configuration/clusters/marshaller.jade      |   76 -
 .../configuration/clusters/marshaller.pug       |   83 +
 .../states/configuration/clusters/memory.pug    |  124 +
 .../states/configuration/clusters/metrics.jade  |   51 -
 .../states/configuration/clusters/metrics.pug   |   51 +
 .../states/configuration/clusters/misc.pug      |   64 +
 .../states/configuration/clusters/odbc.jade     |   48 -
 .../states/configuration/clusters/odbc.pug      |   59 +
 .../configuration/clusters/persistence.pug      |   82 +
 .../states/configuration/clusters/service.pug   |   88 +
 .../configuration/clusters/sql-connector.pug    |   60 +
 .../states/configuration/clusters/ssl.jade      |  110 -
 .../states/configuration/clusters/ssl.pug       |  110 +
 .../states/configuration/clusters/swap.jade     |   72 -
 .../states/configuration/clusters/swap.pug      |   72 +
 .../states/configuration/clusters/thread.jade   |   48 -
 .../states/configuration/clusters/thread.pug    |   92 +
 .../states/configuration/clusters/time.jade     |   47 -
 .../states/configuration/clusters/time.pug      |   50 +
 .../configuration/clusters/transactions.jade    |   69 -
 .../configuration/clusters/transactions.pug     |   69 +
 .../states/configuration/domains/general.jade   |   52 -
 .../states/configuration/domains/general.pug    |   52 +
 .../states/configuration/domains/query.jade     |  172 -
 .../states/configuration/domains/query.pug      |  174 +
 .../states/configuration/domains/store.jade     |  127 -
 .../states/configuration/domains/store.pug      |  127 +
 .../modules/states/configuration/igfs/dual.jade |   42 -
 .../modules/states/configuration/igfs/dual.pug  |   42 +
 .../states/configuration/igfs/fragmentizer.jade |   43 -
 .../states/configuration/igfs/fragmentizer.pug  |   43 +
 .../states/configuration/igfs/general.jade      |   57 -
 .../states/configuration/igfs/general.pug       |   57 +
 .../modules/states/configuration/igfs/ipc.jade  |   60 -
 .../modules/states/configuration/igfs/ipc.pug   |   60 +
 .../modules/states/configuration/igfs/misc.jade |  108 -
 .../modules/states/configuration/igfs/misc.pug  |  123 +
 .../states/configuration/igfs/secondary.jade    |   45 -
 .../states/configuration/igfs/secondary.pug     |   45 +
 .../summary/summary-zipper.service.js           |    4 +-
 .../configuration/summary/summary.controller.js |   12 +-
 .../configuration/summary/summary.worker.js     |   67 +-
 .../frontend/app/modules/states/errors.state.js |    4 +-
 .../app/modules/states/password.state.js        |    6 +-
 .../app/modules/states/profile.state.js         |    6 +-
 .../frontend/app/modules/states/signin.state.js |    6 +-
 .../frontend/app/modules/user/Auth.service.js   |    8 +-
 .../frontend/app/primitives/badge/index.scss    |   41 +
 .../frontend/app/primitives/btn-group/index.pug |   39 +
 .../frontend/app/primitives/btn/index.scss      |  321 ++
 .../app/primitives/datepicker/index.pug         |   61 +
 .../app/primitives/datepicker/index.scss        |   94 +
 .../frontend/app/primitives/dropdown/index.pug  |   41 +
 .../frontend/app/primitives/dropdown/index.scss |  143 +
 .../frontend/app/primitives/file/index.pug      |   37 +
 .../frontend/app/primitives/file/index.scss     |   60 +
 .../app/primitives/form-field/index.scss        |   76 +
 .../frontend/app/primitives/grid/index.scss     |   42 +
 .../frontend/app/primitives/index.js            |   34 +
 .../frontend/app/primitives/modal/index.scss    |  215 +
 .../frontend/app/primitives/page/index.scss     |   35 +
 .../frontend/app/primitives/panel/index.scss    |   52 +
 .../frontend/app/primitives/switch/index.pug    |   34 +
 .../frontend/app/primitives/switch/index.scss   |   87 +
 .../frontend/app/primitives/switcher/index.pug  |   20 +
 .../frontend/app/primitives/switcher/index.scss |   80 +
 .../frontend/app/primitives/table/index.scss    |   91 +
 .../frontend/app/primitives/tabs/index.scss     |   88 +
 .../app/primitives/timepicker/index.pug         |   59 +
 .../app/primitives/timepicker/index.scss        |  137 +
 .../frontend/app/primitives/tooltip/index.pug   |   26 +
 .../app/primitives/typography/index.scss        |   36 +
 .../app/primitives/ui-grid-header/index.scss    |   99 +
 .../app/primitives/ui-grid-header/index.tpl.pug |   37 +
 .../app/primitives/ui-grid-settings/index.scss  |  256 +
 .../frontend/app/primitives/ui-grid/index.scss  |  521 ++
 .../web-console/frontend/app/services/Caches.js |   28 +
 .../frontend/app/services/Clone.service.js      |   64 -
 .../frontend/app/services/Clusters.js           |   95 +
 .../frontend/app/services/Clusters.spec.js      |   55 +
 .../frontend/app/services/Confirm.service.js    |    4 +-
 .../app/services/ConfirmBatch.service.js        |    5 +-
 .../frontend/controllers/caches-controller.js   |   54 +-
 .../frontend/controllers/clusters-controller.js |  209 +-
 .../frontend/controllers/domains-controller.js  |  233 +-
 .../frontend/controllers/igfs-controller.js     |   17 +-
 .../frontend/gulpfile.babel.js/index.js         |   26 -
 .../frontend/gulpfile.babel.js/paths.js         |   83 -
 .../frontend/gulpfile.babel.js/tasks/build.js   |   21 -
 .../frontend/gulpfile.babel.js/tasks/bundle.js  |   32 -
 .../frontend/gulpfile.babel.js/tasks/clean.js   |   32 -
 .../frontend/gulpfile.babel.js/tasks/copy.js    |   33 -
 .../gulpfile.babel.js/tasks/ignite-modules.js   |   55 -
 .../frontend/gulpfile.babel.js/tasks/jade.js    |   49 -
 .../frontend/gulpfile.babel.js/tasks/watch.js   |   33 -
 .../gulpfile.babel.js/webpack/common.js         |  191 -
 .../webpack/environments/development.js         |   79 -
 .../webpack/environments/production.js          |   44 -
 .../webpack/environments/test.js                |   52 -
 .../frontend/gulpfile.babel.js/webpack/index.js |   34 -
 .../frontend/ignite_modules/index.js            |    7 +-
 modules/web-console/frontend/package.json       |  184 +-
 .../frontend/public/images/icons/clock.svg      |    1 +
 .../frontend/public/images/icons/cross.svg      |    1 +
 .../frontend/public/images/icons/csv.svg        |    1 +
 .../frontend/public/images/icons/download.svg   |    2 +
 .../frontend/public/images/icons/filter.svg     |    1 +
 .../frontend/public/images/icons/gear.svg       |    1 +
 .../frontend/public/images/icons/index.js       |   25 +
 .../frontend/public/images/icons/manual.svg     |    1 +
 .../frontend/public/images/icons/search.svg     |    1 +
 .../frontend/public/images/ignite-logo.png      |  Bin 1982 -> 0 bytes
 .../frontend/public/images/ignite-logo.svg      |   17 +
 .../frontend/public/images/ignite-logo@2x.png   |  Bin 3325 -> 0 bytes
 .../stylesheets/_bootstrap-variables.scss       |   10 +-
 .../stylesheets/_font-awesome-custom.scss       |    5 +
 .../frontend/public/stylesheets/style.scss      |  373 +-
 .../frontend/public/stylesheets/variables.scss  |    8 +-
 .../frontend/test/e2e/exampe.test.js            |    4 +-
 .../frontend/test/karma.conf.babel.js           |   18 +-
 modules/web-console/frontend/test/karma.conf.js |    2 +-
 .../frontend/test/protractor.conf.js            |   16 +-
 .../frontend/test/unit/JavaTransformer.test.js  |    2 +-
 .../frontend/test/unit/JavaTypes.test.js        |  137 +-
 .../frontend/test/unit/SharpTransformer.test.js |    2 +-
 .../test/unit/SpringTransformer.test.js         |    2 +-
 .../frontend/test/unit/SqlTypes.test.js         |    7 +-
 .../frontend/test/unit/UserAuth.test.js         |    4 +-
 .../frontend/test/unit/Version.test.js          |   98 +-
 .../test/unit/defaultName.filter.test.js        |   27 +-
 modules/web-console/frontend/views/403.jade     |   22 -
 modules/web-console/frontend/views/403.tpl.pug  |   24 +
 modules/web-console/frontend/views/404.jade     |   22 -
 modules/web-console/frontend/views/404.tpl.pug  |   24 +
 modules/web-console/frontend/views/base.jade    |   22 -
 modules/web-console/frontend/views/base.pug     |   26 +
 modules/web-console/frontend/views/base2.pug    |   26 +
 .../frontend/views/configuration/caches.jade    |   55 -
 .../frontend/views/configuration/caches.tpl.pug |   55 +
 .../frontend/views/configuration/clusters.jade  |   68 -
 .../views/configuration/clusters.tpl.pug        |   85 +
 .../views/configuration/domains-import.jade     |  170 -
 .../views/configuration/domains-import.tpl.pug  |  178 +
 .../frontend/views/configuration/domains.jade   |   66 -
 .../views/configuration/domains.tpl.pug         |   65 +
 .../frontend/views/configuration/igfs.jade      |   51 -
 .../frontend/views/configuration/igfs.tpl.pug   |   54 +
 .../frontend/views/configuration/sidebar.jade   |   29 -
 .../summary-project-structure.jade              |   27 -
 .../summary-project-structure.tpl.pug           |   28 +
 .../views/configuration/summary-tabs.jade       |   25 -
 .../views/configuration/summary-tabs.pug        |   25 +
 .../frontend/views/configuration/summary.jade   |   90 -
 .../views/configuration/summary.tpl.pug         |   90 +
 .../frontend/views/includes/footer.jade         |   23 -
 .../frontend/views/includes/header-left.pug     |   56 +
 .../frontend/views/includes/header-right.pug    |   39 +
 .../frontend/views/includes/header.jade         |   52 -
 modules/web-console/frontend/views/index.jade   |   47 -
 modules/web-console/frontend/views/index.pug    |   48 +
 modules/web-console/frontend/views/reset.jade   |   48 -
 .../web-console/frontend/views/reset.tpl.pug    |   44 +
 .../frontend/views/settings/admin.jade          |   25 -
 .../frontend/views/settings/admin.tpl.pug       |   28 +
 .../frontend/views/settings/profile.jade        |   76 -
 .../frontend/views/settings/profile.tpl.pug     |   76 +
 modules/web-console/frontend/views/signin.jade  |  163 -
 .../web-console/frontend/views/signin.tpl.pug   |  159 +
 .../frontend/views/sql/cache-metadata.jade      |   40 -
 .../frontend/views/sql/cache-metadata.tpl.pug   |   40 +
 .../frontend/views/sql/chart-settings.jade      |   40 -
 .../frontend/views/sql/chart-settings.tpl.pug   |   40 +
 .../frontend/views/sql/notebook-new.jade        |   33 -
 .../frontend/views/sql/notebook-new.tpl.pug     |   33 +
 .../frontend/views/sql/paragraph-rate.jade      |   31 -
 .../frontend/views/sql/paragraph-rate.tpl.pug   |   31 +
 modules/web-console/frontend/views/sql/sql.jade |  278 -
 .../web-console/frontend/views/sql/sql.tpl.pug  |  288 ++
 .../views/templates/agent-download.jade         |   50 -
 .../views/templates/agent-download.tpl.pug      |   61 +
 .../frontend/views/templates/alert.jade         |   21 -
 .../frontend/views/templates/alert.tpl.pug      |   21 +
 .../frontend/views/templates/batch-confirm.jade |   34 -
 .../views/templates/batch-confirm.tpl.pug       |   34 +
 .../frontend/views/templates/clone.jade         |   39 -
 .../frontend/views/templates/confirm.jade       |   33 -
 .../frontend/views/templates/confirm.tpl.pug    |   34 +
 .../frontend/views/templates/demo-info.jade     |   47 -
 .../frontend/views/templates/demo-info.tpl.pug  |   47 +
 .../frontend/views/templates/dropdown.jade      |   24 -
 .../frontend/views/templates/dropdown.tpl.pug   |   24 +
 .../views/templates/getting-started.jade        |   34 -
 .../views/templates/getting-started.tpl.pug     |   34 +
 .../frontend/views/templates/message.jade       |   28 -
 .../frontend/views/templates/message.tpl.pug    |   28 +
 .../frontend/views/templates/pagination.jade    |   32 -
 .../frontend/views/templates/select.jade        |   26 -
 .../views/templates/validation-error.jade       |   25 -
 .../views/templates/validation-error.tpl.pug    |   25 +
 .../frontend/webpack/webpack.common.js          |  193 +
 .../frontend/webpack/webpack.dev.babel.js       |   97 +
 .../frontend/webpack/webpack.prod.babel.js      |   64 +
 .../frontend/webpack/webpack.test.js            |   33 +
 modules/web-console/licenses/cc-by-3.0.txt      |  319 ++
 modules/web-console/pom.xml                     |    2 +-
 modules/web-console/web-agent/README.txt        |    2 +-
 modules/web-console/web-agent/pom.xml           |   38 +-
 .../console/agent/AgentConfiguration.java       |   47 +-
 .../ignite/console/agent/AgentLauncher.java     |  329 +-
 .../apache/ignite/console/agent/AgentUtils.java |   74 +
 .../ignite/console/agent/db/DbColumn.java       |   95 +
 .../console/agent/db/DbMetadataReader.java      |  142 +
 .../ignite/console/agent/db/DbSchema.java       |   60 +
 .../apache/ignite/console/agent/db/DbTable.java |   87 +
 .../agent/db/dialect/DB2MetadataDialect.java    |   33 +
 .../db/dialect/DatabaseMetadataDialect.java     |  127 +
 .../agent/db/dialect/JdbcMetadataDialect.java   |  246 +
 .../agent/db/dialect/MySQLMetadataDialect.java  |   88 +
 .../agent/db/dialect/OracleMetadataDialect.java |  429 ++
 .../console/agent/handlers/AbstractHandler.java |  110 -
 .../agent/handlers/AbstractListener.java        |  104 +
 .../console/agent/handlers/ClusterListener.java |  327 ++
 .../console/agent/handlers/DatabaseHandler.java |  298 --
 .../agent/handlers/DatabaseListener.java        |  332 ++
 .../console/agent/handlers/DemoListener.java    |  131 +
 .../console/agent/handlers/RestHandler.java     |  276 -
 .../console/agent/handlers/RestListener.java    |   81 +
 .../ignite/console/agent/rest/RestExecutor.java |  224 +
 .../ignite/console/agent/rest/RestResult.java   |   81 +
 .../ignite/console/demo/AgentClusterDemo.java   |  622 +--
 .../ignite/console/demo/AgentDemoUtils.java     |   79 +
 .../demo/service/DemoCachesLoadService.java     |  480 ++
 .../demo/service/DemoComputeLoadService.java    |   79 +
 .../service/DemoRandomCacheLoadService.java     |  119 +
 .../service/DemoServiceClusterSingleton.java    |   41 +
 .../demo/service/DemoServiceKeyAffinity.java    |   41 +
 .../service/DemoServiceMultipleInstances.java   |   41 +
 .../demo/service/DemoServiceNodeSingleton.java  |   41 +
 .../console/demo/task/DemoCancellableTask.java  |   92 +
 .../console/demo/task/DemoComputeTask.java      |  105 +
 .../src/main/resources/log4j.properties         |    7 +-
 modules/web/ignite-appserver-test/pom.xml       |    2 +-
 modules/web/ignite-websphere-test/pom.xml       |    2 +-
 modules/web/pom.xml                             |    2 +-
 .../cache/websession/WebSessionFilter.java      |   34 +-
 .../servlet/ServletContextListenerStartup.java  |   12 +-
 .../ignite/startup/servlet/ServletStartup.java  |    8 +-
 .../internal/websession/WebSessionSelfTest.java |   25 +-
 .../webapp2/META-INF/ignite-webapp-config.xml   |   28 +-
 modules/yardstick/DEVNOTES-standalone.txt       |   16 +
 modules/yardstick/DEVNOTES.txt                  |   20 +-
 modules/yardstick/README.txt                    |  145 +-
 .../config/benchmark-atomic-win.properties      |   10 +-
 .../config/benchmark-atomic.properties          |   50 +-
 .../config/benchmark-bin-identity.properties    |   24 +-
 .../config/benchmark-cache-load-win.properties  |    2 +-
 .../config/benchmark-cache-load.properties      |   19 +-
 .../config/benchmark-client-mode.properties     |   67 +-
 .../config/benchmark-compute-win.properties     |    2 +-
 .../config/benchmark-compute.properties         |   51 +-
 .../config/benchmark-failover.properties        |   24 +-
 .../yardstick/config/benchmark-full.properties  |   80 +-
 .../yardstick/config/benchmark-h2.properties    |   56 +
 .../config/benchmark-multicast.properties       |  106 +-
 .../yardstick/config/benchmark-mysql.properties |   56 +
 .../yardstick/config/benchmark-pgsql.properties |   57 +
 .../config/benchmark-put-indexed-val.properties |   44 +-
 .../benchmark-query-put-separated.properties    |   11 +-
 .../config/benchmark-query-win.properties       |    7 +-
 .../yardstick/config/benchmark-query.properties |   51 +-
 .../config/benchmark-remote-sample.properties   |   81 +
 .../config/benchmark-remote.properties          |  115 +
 .../config/benchmark-sample.properties          |   81 +
 .../config/benchmark-sql-dml.properties         |   57 +-
 .../yardstick/config/benchmark-store.properties |   41 +-
 .../config/benchmark-tx-win.properties          |    8 +-
 .../yardstick/config/benchmark-tx.properties    |   50 +-
 .../yardstick/config/benchmark-win.properties   |    2 +-
 modules/yardstick/config/benchmark.properties   |   85 +-
 modules/yardstick/config/h2-schema.sql          |   11 +
 modules/yardstick/config/ignite-base-config.xml |   85 +-
 .../config/ignite-base-load-config.xml          |   98 +-
 .../config/ignite-cache-load-config.xml         |    5 -
 .../yardstick/config/ignite-db-base-config.xml  |  118 +
 .../config/ignite-db-localhost-config.xml       |   53 +
 .../config/ignite-failover-base-config.xml      |   43 -
 .../ignite-int-max-values-offheap-config.xml    |   89 -
 .../ignite-int-max-values-onheap-config.xml     |    5 +-
 .../ignite-int-max-values-swap-config.xml       |   93 -
 modules/yardstick/config/ignite-jdbc-config.xml |    4 +-
 .../config/ignite-localhost-config.xml          |    6 +
 .../yardstick/config/ignite-rdbms-config.xml    |   70 +
 .../yardstick/config/ignite-remote-config.xml   |   47 +
 .../yardstick/config/ignite-store-config.xml    |    2 -
 modules/yardstick/config/mysql-schema.sql       |   11 +
 modules/yardstick/config/pgsql-schema.sql       |   18 +
 .../config/sql/benchmark-jdbc-ignite.properties |   64 +
 .../sql/benchmark-mysql-ignite.properties       |   66 +
 .../config/sql/benchmark-native.properties      |   62 +
 .../sql/benchmark-pgsql-ignite.properties       |   66 +
 modules/yardstick/config/sql/db-h2-queries.sql  |    9 +
 .../yardstick/config/sql/db-mysql-queries.sql   |   14 +
 .../yardstick/config/sql/db-pgsql-queries.sql   |   14 +
 .../yardstick/config/sql/h2-insert-query.sql    |    1 +
 .../yardstick/config/sql/rdbms-mysql-schema.sql |    5 +
 .../yardstick/config/sql/rdbms-pgsql-schema.sql |   10 +
 .../test-max-int-values-offheap.properties      |   70 -
 .../test-max-int-values-onheap.properties       |   31 +-
 .../config/test-max-int-values-swap.properties  |   69 -
 modules/yardstick/pom-standalone.xml            |    2 +-
 modules/yardstick/pom.xml                       |   20 +-
 .../yardstick/IgniteAbstractBenchmark.java      |    8 +-
 .../yardstick/IgniteBenchmarkArguments.java     |  242 +-
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |    9 +-
 .../org/apache/ignite/yardstick/IgniteNode.java |   46 +-
 .../cache/IgniteBinaryIdentityBenchmark.java    |   21 -
 .../cache/IgniteCacheAbstractBenchmark.java     |  158 +
 .../yardstick/cache/IgniteGetAllBenchmark.java  |   45 +
 .../cache/IgniteGetAllOffHeapBenchmark.java     |   30 +
 .../cache/IgniteGetAllPutAllTxBenchmark.java    |    2 +
 .../cache/IgniteGetAndPutBenchmark.java         |    4 +-
 .../cache/IgniteGetAndPutTxBenchmark.java       |    4 +-
 .../yardstick/cache/IgniteGetBenchmark.java     |   19 +
 .../IgniteGetEntriesPutAllTxBenchmark.java      |    2 +
 .../cache/IgniteGetFromComputeBenchmark.java    |  167 +
 .../cache/IgniteGetOffHeapBenchmark.java        |   32 -
 .../cache/IgniteGetOffHeapValuesBenchmark.java  |   32 -
 .../yardstick/cache/IgniteInvokeBenchmark.java  |    2 +
 .../cache/IgniteInvokeTxBenchmark.java          |    2 +
 .../IgniteInvokeWithInjectionBenchmark.java     |    2 +
 .../cache/IgniteIoTestAbstractBenchmark.java    |   61 +
 .../cache/IgniteIoTestSendAllBenchmark.java     |   32 +
 .../cache/IgniteIoTestSendRandomBenchmark.java  |   35 +
 .../IgniteLegacyBinaryIdentityGetBenchmark.java |   30 -
 .../IgniteLegacyBinaryIdentityPutBenchmark.java |   30 -
 .../yardstick/cache/IgnitePutAllBenchmark.java  |   97 +-
 .../cache/IgnitePutAllOffHeapBenchmark.java     |   30 +
 .../IgnitePutAllSerializableTxBenchmark.java    |    2 +
 .../cache/IgnitePutAllTxBenchmark.java          |   43 +-
 .../cache/IgnitePutAllTxOffHeapBenchmark.java   |   30 +
 .../yardstick/cache/IgnitePutBenchmark.java     |    2 +
 .../cache/IgnitePutGetBatchBenchmark.java       |    2 +
 .../yardstick/cache/IgnitePutGetBenchmark.java  |    2 +
 .../cache/IgnitePutGetEntryBenchmark.java       |    2 +
 .../cache/IgnitePutGetEntryTxBenchmark.java     |    2 +
 .../cache/IgnitePutGetOffHeapBenchmark.java     |   32 -
 .../IgnitePutGetOffHeapValuesBenchmark.java     |   32 -
 .../cache/IgnitePutGetTxBatchBenchmark.java     |    2 +
 .../cache/IgnitePutGetTxBenchmark.java          |    2 +
 .../cache/IgnitePutGetTxOffHeapBenchmark.java   |   32 -
 .../IgnitePutGetTxOffHeapValuesBenchmark.java   |   32 -
 ...IgnitePutIfAbsentIndexedValue1Benchmark.java |    2 +
 .../cache/IgnitePutIndexedValue1Benchmark.java  |    2 +
 .../cache/IgnitePutIndexedValue2Benchmark.java  |    2 +
 .../cache/IgnitePutIndexedValue8Benchmark.java  |    2 +
 .../cache/IgnitePutOffHeapBenchmark.java        |   32 -
 .../IgnitePutOffHeapIndexedValue8Benchmark.java |   30 +
 .../cache/IgnitePutOffHeapValuesBenchmark.java  |   32 -
 .../IgnitePutRandomValueSizeBenchmark.java      |   43 +
 .../cache/IgnitePutRemoveBenchmark.java         |    2 +
 .../yardstick/cache/IgnitePutTxBenchmark.java   |    2 +
 .../cache/IgnitePutTxImplicitBenchmark.java     |    2 +
 .../cache/IgnitePutTxOffHeapBenchmark.java      |   32 -
 .../IgnitePutTxOffHeapValuesBenchmark.java      |   32 -
 .../cache/IgnitePutTxPrimaryOnlyBenchmark.java  |    2 +
 .../IgnitePutTxSkipLocalBackupBenchmark.java    |    2 +
 .../cache/IgnitePutValue8Benchmark.java         |   44 +
 .../IgniteReplaceIndexedValue1Benchmark.java    |    2 +
 .../cache/IgniteScanQueryBenchmark.java         |   88 +
 .../cache/IgniteSqlQueryBenchmark.java          |   16 +-
 .../IgniteSqlQueryDistributedJoinBenchmark.java |   24 +-
 .../cache/IgniteSqlQueryJoinBenchmark.java      |   14 +-
 .../IgniteSqlQueryJoinOffHeapBenchmark.java     |   32 -
 .../cache/IgniteSqlQueryOffHeapBenchmark.java   |   32 -
 .../cache/IgniteSqlQueryPutBenchmark.java       |    8 +-
 .../IgniteSqlQueryPutOffHeapBenchmark.java      |   32 -
 .../IgniteSqlQueryPutSeparatedBenchmark.java    |    4 +
 .../cache/WaitMapExchangeFinishCallable.java    |    4 +-
 .../dml/IgniteSqlDeleteFilteredBenchmark.java   |    5 +-
 .../dml/IgniteSqlUpdateFilteredBenchmark.java   |    5 +-
 .../IgniteAtomicInvokeRetryBenchmark.java       |   12 +-
 ...IgniteAtomicOffHeapInvokeRetryBenchmark.java |   31 -
 .../IgniteAtomicOffHeapRetriesBenchmark.java    |   31 -
 .../failover/IgniteAtomicRetriesBenchmark.java  |   12 +-
 .../IgniteFailoverAbstractBenchmark.java        |   22 +-
 ...IgniteTransactionalInvokeRetryBenchmark.java |   10 +-
 ...ransactionalOffHeapInvokeRetryBenchmark.java |   33 -
 ...ransactionalOffHeapWriteInvokeBenchmark.java |   37 -
 ...eTransactionalOffHeapWriteReadBenchmark.java |   32 -
 ...IgniteTransactionalWriteInvokeBenchmark.java |   16 +-
 .../IgniteTransactionalWriteReadBenchmark.java  |   12 +-
 .../cache/jdbc/IgniteNativeTxBenchmark.java     |  169 +
 .../cache/jdbc/JdbcAbstractBenchmark.java       |  202 +
 .../yardstick/cache/jdbc/JdbcPutBenchmark.java  |   86 +
 .../cache/jdbc/JdbcPutGetBenchmark.java         |   62 +
 .../jdbc/JdbcPutIndexedValue8Benchmark.java     |  104 +
 .../cache/jdbc/JdbcSqlQueryBenchmark.java       |   94 +
 .../cache/jdbc/JdbcSqlQueryJoinBenchmark.java   |  111 +
 .../yardstick/cache/jdbc/RdbmsBenchmark.java    |  220 +
 .../IgniteCacheRandomOperationBenchmark.java    |  127 +-
 .../yardstick/cache/load/model/ModelUtil.java   |    6 +-
 .../ignite/yardstick/cache/model/Accounts.java  |   62 +
 .../ignite/yardstick/cache/model/Branches.java  |   62 +
 .../ignite/yardstick/cache/model/History.java   |   69 +
 .../ignite/yardstick/cache/model/Tellers.java   |   62 +
 .../apache/ignite/yardstick/io/FileUtils.java   |   64 +
 modules/yarn/pom.xml                            |    2 +-
 .../apache/ignite/yarn/ClusterProperties.java   |    1 +
 .../yarn/IgniteApplicationMasterSelfTest.java   |   13 +
 modules/zeromq/README.txt                       |   37 +
 modules/zeromq/licenses/apache-2.0.txt          |  202 +
 modules/zeromq/pom.xml                          |   75 +
 .../stream/zeromq/IgniteZeroMqStreamer.java     |  146 +
 .../ignite/stream/zeromq/ZeroMqTypeSocket.java  |   56 +
 .../ignite/stream/zeromq/package-info.java      |   21 +
 .../stream/zeromq/IgniteZeroMqStreamerTest.java |  235 +
 .../zeromq/IgniteZeroMqStreamerTestSuite.java   |   37 +
 .../ZeroMqStringSingleTupleExtractor.java       |   46 +
 .../ignite/stream/zeromq/package-info.java      |   21 +
 modules/zookeeper/pom.xml                       |    2 +-
 .../zk/TcpDiscoveryZookeeperIpFinder.java       |   37 +-
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  |   15 +-
 parent/pom.xml                                  |   43 +-
 pom.xml                                         |  438 +-
 scripts/git-remoteless-branches.sh              |   35 +
 6073 files changed, 418817 insertions(+), 181770 deletions(-)
----------------------------------------------------------------------



[23/31] ignite git commit: Native batching prototype.

Posted by vo...@apache.org.
Native batching prototype.


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

Branch: refs/heads/ignite-6022-proto
Commit: 03f9fe71ed1e41353e983cc8993b7b369cd91936
Parents: dd70a84
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 14:15:41 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 14:15:41 2017 +0300

----------------------------------------------------------------------
 .../cache/query/SqlFieldsQueryEx.java           |  44 ++++-
 .../odbc/jdbc/JdbcRequestHandler.java           |  42 +++--
 .../query/h2/DmlStatementsProcessor.java        | 178 ++++++++++++++++++-
 .../processors/query/h2/IgniteH2Indexing.java   |  17 +-
 .../processors/query/h2/dml/DmlUtils.java       |  19 ++
 .../processors/query/h2/dml/UpdatePlan.java     |  34 +++-
 .../query/h2/dml/UpdatePlanBuilder.java         |   2 +-
 7 files changed, 303 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/03f9fe71/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java
index c5f786e..fb098a7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 
@@ -33,13 +35,23 @@ public final class SqlFieldsQueryEx extends SqlFieldsQuery {
     /** Whether server side DML should be enabled. */
     private boolean skipReducerOnUpdate;
 
+    /** Batched arguments. */
+    private List<Object[]> batchedArgs;
+
+    public SqlFieldsQueryEx(String sql, Boolean isQry) {
+        super(sql);
+
+        this.isQry = isQry;
+    }
+
     /**
      * @param sql SQL query.
      * @param isQry Flag indicating whether this object denotes a query or an update operation.
      */
-    public SqlFieldsQueryEx(String sql, Boolean isQry) {
-        super(sql);
-        this.isQry = isQry;
+    public SqlFieldsQueryEx(String sql, Boolean isQry, int batchedArgsSize) {
+        this(sql, isQry);
+
+        this.batchedArgs = new ArrayList<>(batchedArgsSize);
     }
 
     /**
@@ -50,6 +62,7 @@ public final class SqlFieldsQueryEx extends SqlFieldsQuery {
 
         this.isQry = qry.isQry;
         this.skipReducerOnUpdate = qry.skipReducerOnUpdate;
+        this.batchedArgs = qry.batchedArgs;
     }
 
     /**
@@ -151,6 +164,31 @@ public final class SqlFieldsQueryEx extends SqlFieldsQuery {
         return skipReducerOnUpdate;
     }
 
+    /**
+     * Add batched arguments.
+     *
+     * @param args Arguments.
+     */
+    public void addBatchedArgs(Object[] args) {
+        if (batchedArgs == null)
+            batchedArgs = new ArrayList<>();
+
+        batchedArgs.add(args);
+    }
+
+    /**
+     * Batched arguments.
+     *
+     * @return Batched arguments.
+     */
+    public List<Object[]> batchedArgs() {
+        return batchedArgs;
+    }
+
+    public void clearBatchedArgs() {
+        batchedArgs = null;
+    }
+
     /** {@inheritDoc} */
     @Override public SqlFieldsQuery copy() {
         return new SqlFieldsQueryEx(this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/03f9fe71/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index e3b6f5b..2530360 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -475,28 +475,45 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
         if (F.isEmpty(schemaName))
             schemaName = QueryUtils.DFLT_SCHEMA;
 
+        int qryCnt = req.queries().size();
+
         int successQueries = 0;
-        int updCnts[] = new int[req.queries().size()];
+        int updCnts[] = new int[qryCnt];
 
         try {
-            String sql = null;
+            SqlFieldsQueryEx qry = null;
 
             for (JdbcQuery q : req.queries()) {
-                if (q.sql() != null)
-                    sql = q.sql();
+                if (q.sql() != null) {
+                    if (qry != null) {
+                        QueryCursorImpl<List<?>> qryCur = (QueryCursorImpl<List<?>>)ctx.query()
+                            .querySqlFieldsNoCache(qry, true, true).get(0);
+
+                        assert !qryCur.isQuery();
+
+                        List<List<?>> items = qryCur.getAll();
+
+                        // TODO: Set on correct positions.
+                        updCnts[successQueries++] = ((Long)items.get(0).get(0)).intValue();
+                    }
+
+                    qry = new SqlFieldsQueryEx(q.sql(), false, qryCnt);
 
-                SqlFieldsQuery qry = new SqlFieldsQueryEx(sql, false);
+                    qry.setDistributedJoins(distributedJoins);
+                    qry.setEnforceJoinOrder(enforceJoinOrder);
+                    qry.setCollocated(collocated);
+                    qry.setReplicatedOnly(replicatedOnly);
+                    qry.setLazy(lazy);
 
-                qry.setArgs(q.args());
+                    qry.setSchema(schemaName);
+                }
 
-                qry.setDistributedJoins(distributedJoins);
-                qry.setEnforceJoinOrder(enforceJoinOrder);
-                qry.setCollocated(collocated);
-                qry.setReplicatedOnly(replicatedOnly);
-                qry.setLazy(lazy);
+                assert qry != null;
 
-                qry.setSchema(schemaName);
+                qry.addBatchedArgs(q.args());
+            }
 
+            if (qry != null) {
                 QueryCursorImpl<List<?>> qryCur = (QueryCursorImpl<List<?>>)ctx.query()
                     .querySqlFieldsNoCache(qry, true, true).get(0);
 
@@ -504,6 +521,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
 
                 List<List<?>> items = qryCur.getAll();
 
+                // TODO: Set on correct positions.
                 updCnts[successQueries++] = ((Long)items.get(0).get(0)).intValue();
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/03f9fe71/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 9a6b0af..df14c85 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -22,6 +22,7 @@ import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -42,6 +43,7 @@ import org.apache.ignite.internal.processors.cache.CacheOperationContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx;
 import org.apache.ignite.internal.processors.odbc.SqlStateCode;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
@@ -50,6 +52,7 @@ import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlBatchSender;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlDistributedPlanInfo;
+import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
@@ -189,6 +192,80 @@ public class DmlStatementsProcessor {
     }
 
     /**
+     * Execute DML statement, possibly with few re-attempts in case of concurrent data modifications.
+     *
+     * @param schemaName Schema.
+     * @param conn Connection.
+     * @param prepared Prepared statement.
+     * @param fieldsQry Original query.
+     * @param loc Query locality flag.
+     * @param filters Cache name and key filter.
+     * @param cancel Cancel.
+     * @return Update result (modified items count and failed keys).
+     * @throws IgniteCheckedException if failed.
+     */
+    private Collection<UpdateResult> updateSqlFieldsBatched(String schemaName, Connection conn, Prepared prepared,
+        SqlFieldsQueryEx fieldsQry, boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel)
+        throws IgniteCheckedException {
+        List<Object[]> argss = fieldsQry.batchedArgs();
+
+        UpdatePlan plan = getPlanForStatement(schemaName, conn, prepared, fieldsQry, loc, null);
+
+        if (plan.hasRows()) {
+            GridCacheContext<?, ?> cctx = plan.cacheContext();
+
+            CacheOperationContext opCtx = cctx.operationContextPerCall();
+
+            // Force keepBinary for operation context to avoid binary deserialization inside entry processor
+            if (cctx.binaryMarshaller()) {
+                CacheOperationContext newOpCtx = null;
+
+                if (opCtx == null)
+                    // Mimics behavior of GridCacheAdapter#keepBinary and GridCacheProxyImpl#keepBinary
+                    newOpCtx = new CacheOperationContext(false, null, true, null, false, null, false);
+                else if (!opCtx.isKeepBinary())
+                    newOpCtx = opCtx.keepBinary();
+
+                if (newOpCtx != null)
+                    cctx.operationContextPerCall(newOpCtx);
+            }
+
+            try {
+                List<List<?>> cur = plan.createRows(argss);
+
+                UpdateResult res = processDmlSelectResultBatched(cctx, plan, cur, fieldsQry.getPageSize());
+
+                Collection<UpdateResult> ress = new ArrayList<>(1);
+
+                // TODO: Wrong!
+                ress.add(res);
+
+                return ress;
+            }
+            finally {
+                cctx.operationContextPerCall(opCtx);
+            }
+        }
+        else {
+            // Fallback to previous mode.
+            Collection<UpdateResult> ress = new ArrayList<>(argss.size());
+
+            for (Object[] args : argss) {
+                SqlFieldsQueryEx qry0 = (SqlFieldsQueryEx)fieldsQry.copy();
+
+                qry0.clearBatchedArgs();
+                qry0.setArgs(args);
+
+                UpdateResult res = updateSqlFields(schemaName, conn, prepared, qry0, loc, filters, cancel);
+
+                ress.add(res);
+            }
+
+            return ress;
+        }
+    }
+
+    /**
      * @param schemaName Schema.
      * @param c Connection.
      * @param p Prepared statement.
@@ -198,18 +275,44 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    QueryCursorImpl<List<?>> updateSqlFieldsDistributed(String schemaName, Connection c, Prepared p,
+    List<QueryCursorImpl<List<?>>> updateSqlFieldsDistributed(String schemaName, Connection c, Prepared p,
         SqlFieldsQuery fieldsQry, GridQueryCancel cancel) throws IgniteCheckedException {
-        UpdateResult res = updateSqlFields(schemaName, c, p, fieldsQry, false, null, cancel);
+        if (DmlUtils.isBatched(fieldsQry)) {
+            // TODO: Refactor.
+            Collection<UpdateResult> ress = updateSqlFieldsBatched(schemaName, c, p, (SqlFieldsQueryEx)fieldsQry,
+                false, null, cancel);
+
+            ArrayList<QueryCursorImpl<List<?>>> resCurs = new ArrayList<>(ress.size());
 
-        checkUpdateResult(res);
+            for (UpdateResult res : ress) {
+                checkUpdateResult(res);
 
-        QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
-            (Collections.singletonList(res.counter())), cancel, false);
+                QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
+                    (Collections.singletonList(res.counter())), cancel, false);
 
-        resCur.fieldsMeta(UPDATE_RESULT_META);
+                resCur.fieldsMeta(UPDATE_RESULT_META);
 
-        return resCur;
+                resCurs.add(resCur);
+            }
+
+            return resCurs;
+        }
+        else {
+            UpdateResult res = updateSqlFields(schemaName, c, p, fieldsQry, false, null, cancel);
+
+            ArrayList<QueryCursorImpl<List<?>>> resCurs = new ArrayList<>(1);
+
+            checkUpdateResult(res);
+
+            QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
+                (Collections.singletonList(res.counter())), cancel, false);
+
+            resCur.fieldsMeta(UPDATE_RESULT_META);
+
+            resCurs.add(resCur);
+
+            return resCurs;
+        }
     }
 
     /**
@@ -399,6 +502,22 @@ public class DmlStatementsProcessor {
         return processDmlSelectResult(cctx, plan, cur, pageSize);
     }
 
+    private UpdateResult processDmlSelectResultBatched(GridCacheContext cctx, UpdatePlan plan, Collection<List<?>> rows,
+        int pageSize) throws IgniteCheckedException {
+        switch (plan.mode()) {
+            case MERGE:
+                // TODO
+                throw new IgniteCheckedException("Unsupported, fix");
+
+            case INSERT:
+                return new UpdateResult(doInsertBatched(plan, rows, pageSize), X.EMPTY_OBJECT_ARRAY);
+
+            default:
+                throw new IgniteSQLException("Unexpected DML operation [mode=" + plan.mode() + ']',
+                    IgniteQueryErrorCode.UNEXPECTED_OPERATION);
+        }
+    }
+
     /**
      * @param cctx Cache context.
      * @param plan Update plan.
@@ -673,6 +792,50 @@ public class DmlStatementsProcessor {
     }
 
     /**
+     * Execute INSERT statement plan.
+     * @param cursor Cursor to take inserted data from.
+     * @param pageSize Batch size for streaming, anything <= 0 for single page operations.
+     * @return Number of items affected.
+     * @throws IgniteCheckedException if failed, particularly in case of duplicate keys.
+     */
+    @SuppressWarnings({"unchecked", "ConstantConditions"})
+    private long doInsertBatched(UpdatePlan plan, Collection<List<?>> cursor, int pageSize)
+        throws IgniteCheckedException {
+        GridCacheContext cctx = plan.cacheContext();
+
+        // Keys that failed to INSERT due to duplication.
+        DmlBatchSender sender = new DmlBatchSender(cctx, pageSize);
+
+        for (List<?> row : cursor) {
+            final IgniteBiTuple keyValPair = plan.processRow(row);
+
+            sender.add(keyValPair.getKey(), new InsertEntryProcessor(keyValPair.getValue()));
+        }
+
+        // TODO: Tale page size in count?
+        sender.flush();
+
+        SQLException resEx = sender.error();
+
+        if (!F.isEmpty(sender.failedKeys())) {
+            String msg = "Failed to INSERT some keys because they are already in cache " +
+                "[keys=" + sender.failedKeys() + ']';
+
+            SQLException dupEx = new SQLException(msg, SqlStateCode.CONSTRAINT_VIOLATION);
+
+            if (resEx == null)
+                resEx = dupEx;
+            else
+                resEx.setNextException(dupEx);
+        }
+
+        if (resEx != null)
+            throw new IgniteSQLException(resEx);
+
+        return sender.updateCount();
+    }
+
+    /**
      *
      * @param schemaName Schema name.
      * @param stmt Prepared statement.
@@ -810,5 +973,4 @@ public class DmlStatementsProcessor {
             throw new IgniteSQLException(conEx);
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/03f9fe71/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 6fdcd27..31a6645 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
@@ -1507,14 +1507,17 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     int paramsCnt = prepared.getParameters().size();
 
                     if (paramsCnt > 0) {
-                        if (argsOrig == null || argsOrig.length < firstArg + paramsCnt) {
-                            throw new IgniteException("Invalid number of query parameters. " +
-                                "Cannot find " + (argsOrig.length + 1 - firstArg) + " parameter.");
-                        }
+                        // TODO: Check remainer.
+                        if (prepared.isQuery()) {
+                            if (argsOrig == null || argsOrig.length < firstArg + paramsCnt) {
+                                throw new IgniteException("Invalid number of query parameters. " +
+                                    "Cannot find " + (argsOrig.length + 1 - firstArg) + " parameter.");
+                            }
 
-                        args = Arrays.copyOfRange(argsOrig, firstArg, firstArg + paramsCnt);
+                            args = Arrays.copyOfRange(argsOrig, firstArg, firstArg + paramsCnt);
 
-                        firstArg += paramsCnt;
+                            firstArg += paramsCnt;
+                        }
                     }
 
                     cachedQryKey = new H2TwoStepCachedQueryKey(schemaName, sqlQry, grpByCollocated,
@@ -1555,7 +1558,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 if (twoStepQry == null) {
                     if (DmlStatementsProcessor.isDmlStatement(prepared)) {
                         try {
-                            res.add(dmlProc.updateSqlFieldsDistributed(schemaName, c, prepared,
+                            res.addAll(dmlProc.updateSqlFieldsDistributed(schemaName, c, prepared,
                                 qry.copy().setSql(sqlQry).setArgs(args), cancel));
 
                             continue;

http://git-wip-us.apache.org/repos/asf/ignite/blob/03f9fe71/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
index 8d4861e..e4b52a6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
@@ -22,10 +22,13 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Date;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.H2Utils;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.h2.util.DateTimeUtils;
 import org.h2.util.LocalDateTimeUtils;
@@ -117,6 +120,22 @@ public class DmlUtils {
     }
 
     /**
+     * Check whether query is batched.
+     *
+     * @param qry Query.
+     * @return {@code True} if batched.
+     */
+    public static boolean isBatched(SqlFieldsQuery qry) {
+        if (qry instanceof SqlFieldsQueryEx) {
+            SqlFieldsQueryEx qry0 = (SqlFieldsQueryEx)qry;
+
+            return !F.isEmpty(qry0.batchedArgs());
+        }
+
+        return false;
+    }
+
+    /**
      * Private constructor.
      */
     private DmlUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/03f9fe71/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 6a45c3c..248041a 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
@@ -376,10 +376,10 @@ public final class UpdatePlan {
     public List<List<?>> createRows(Object[] args) throws IgniteCheckedException {
         assert rowsNum > 0 && !F.isEmpty(colNames);
 
-        List<List<?>> res = new ArrayList<>(rowsNum);
-
         GridH2RowDescriptor desc = tbl.rowDescriptor();
 
+        List<List<?>> res = new ArrayList<>(rowsNum);
+
         for (List<DmlArgument> row : rows) {
             List<Object> resRow = new ArrayList<>();
 
@@ -401,6 +401,36 @@ public final class UpdatePlan {
         return res;
     }
 
+    public List<List<?>> createRows(List<Object[]> argss) throws IgniteCheckedException {
+        assert rowsNum > 0 && !F.isEmpty(colNames);
+
+        GridH2RowDescriptor desc = tbl.rowDescriptor();
+
+        List<List<?>> res = new ArrayList<>(rowsNum * argss.size());
+
+        for (Object[] args : argss) {
+            for (List<DmlArgument> row : rows) {
+                List<Object> resRow = new ArrayList<>();
+
+                for (int j = 0; j < colNames.length; j++) {
+                    Object colVal = row.get(j).get(args);
+
+                    if (j == keyColIdx || j == valColIdx) {
+                        Class<?> colCls = j == keyColIdx ? desc.type().keyClass() : desc.type().valueClass();
+
+                        colVal = DmlUtils.convert(colVal, desc, colCls, colTypes[j]);
+                    }
+
+                    resRow.add(colVal);
+                }
+
+                res.add(resRow);
+            }
+        }
+
+        return res;
+    }
+
     /**
      * @return Update mode.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/03f9fe71/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index 5ffd264..3305b00 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -619,7 +619,7 @@ public final class UpdatePlanBuilder {
         Connection conn, SqlFieldsQuery fieldsQry, boolean loc, String selectQry, String cacheName)
         throws IgniteCheckedException {
 
-        if (loc || !isSkipReducerOnUpdateQuery(fieldsQry))
+        if (loc || !isSkipReducerOnUpdateQuery(fieldsQry) || DmlUtils.isBatched(fieldsQry))
             return null;
 
         assert conn != null;


[30/31] ignite git commit: Merge branch 'master' into ignite-4490-batch

Posted by vo...@apache.org.
Merge branch 'master' into ignite-4490-batch

# Conflicts:
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArguments.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java


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

Branch: refs/heads/ignite-6022-proto
Commit: eeb116bf54aa72e5fabbe419383bec3d8a2e8bb4
Parents: b87d6c3 7782d38
Author: devozerov <vo...@gridgain.com>
Authored: Tue Dec 19 15:15:29 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Dec 19 15:15:29 2017 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsDataManager.java        |  4 +-
 ...zySecondaryFileSystemPositionedReadable.java | 15 +++--
 .../query/PlatformAbstractQueryCursor.java      |  9 +++
 .../cache/query/PlatformFieldsQueryCursor.java  | 24 ++++++-
 .../ignite/internal/util/nio/GridNioServer.java | 59 ++++++++++-------
 ...fsSecondaryFileSystemPositionedReadable.java | 41 +++++++-----
 .../processors/query/h2/dml/DmlArguments.java   |  1 -
 .../regression/KNNMultipleLinearRegression.java |  2 +-
 .../ExpiryCacheHolderTest.cs                    |  5 ++
 .../Binary/BinaryFooterTest.cs                  |  2 +-
 .../Binary/Serializable/SqlDmlTest.cs           |  6 +-
 .../Cache/CacheTestAsyncWrapper.cs              |  7 ++
 .../Cache/Query/CacheDmlQueriesTest.cs          | 18 ++---
 .../Query/CacheQueriesCodeConfigurationTest.cs  |  2 +-
 .../Cache/Query/CacheQueriesTest.cs             | 69 +++++++++++++++-----
 .../Query/Linq/CacheLinqTest.Introspection.cs   |  2 +-
 .../Client/Cache/SqlQueryTest.cs                |  4 +-
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |  8 +++
 .../Impl/Binary/BinaryReaderExtensions.cs       |  2 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  | 29 +++++---
 .../Impl/Cache/ICacheInternal.cs                |  2 +-
 .../Impl/Cache/Query/FieldsQueryCursor.cs       | 39 +++++++++++
 .../Impl/Cache/Query/PlatformQueryQursorBase.cs |  8 +++
 .../Impl/Client/Cache/CacheClient.cs            |  2 +-
 .../Cache/Query/ClientFieldsQueryCursor.cs      | 12 +---
 .../NuGet/LINQPad/BinaryModeExample.linq        |  2 +-
 .../NuGet/LINQPad/QueryExample.linq             |  2 +-
 .../Apache.Ignite.Linq/CacheLinqExtensions.cs   | 10 +--
 .../Impl/CacheFieldsQueryExecutor.cs            | 10 +--
 modules/platforms/dotnet/Apache.Ignite.ndproj   |  2 +-
 .../Datagrid/BinaryModeExample.cs               |  5 +-
 .../Datagrid/QueryDmlExample.cs                 | 18 ++---
 .../Datagrid/QueryExample.cs                    |  5 +-
 .../dotnet/examples/dotnetcore/App.config       |  5 +-
 .../dotnet/examples/dotnetcore/SqlExample.cs    |  2 +-
 35 files changed, 298 insertions(+), 135 deletions(-)
----------------------------------------------------------------------



[28/31] ignite git commit: Add benchmark.

Posted by vo...@apache.org.
Add benchmark.


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

Branch: refs/heads/ignite-6022-proto
Commit: 2df2c1f6af2650247a3e717f4ca10a9d817b376b
Parents: 392e8ee
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 15:52:54 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 15:52:54 2017 +0300

----------------------------------------------------------------------
 .../odbc/jdbc/JdbcRequestHandler.java           |   2 +-
 modules/indexing/pom.xml                        |   6 +
 .../query/h2/opt/JdbcBatchLoader.java           | 167 +++++++++++++++++++
 3 files changed, 174 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2df2c1f6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index c28c831..7e508a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -465,7 +465,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
         }
     }
 
-    public static volatile boolean STREAMER = false;
+    public static volatile boolean STREAMER = true;
 
     /**
      * @param req Request.

http://git-wip-us.apache.org/repos/asf/ignite/blob/2df2c1f6/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index fc965c5..1edaac0 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -123,6 +123,12 @@
             <version>${spring.version}</version>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>com.mchange</groupId>
+            <artifactId>c3p0</artifactId>
+            <version>0.9.5.2</version>
+        </dependency>
     </dependencies>
 
     <profiles>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2df2c1f6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBatchLoader.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBatchLoader.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBatchLoader.java
new file mode 100644
index 0000000..f145f58
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBatchLoader.java
@@ -0,0 +1,167 @@
+package org.apache.ignite.internal.processors.query.h2.opt;
+
+import com.mchange.v2.c3p0.ComboPooledDataSource;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.Statement;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+public class JdbcBatchLoader {
+    /** */
+    private static final String SQL_CREATE = "CREATE TABLE IF NOT EXISTS Person(" +
+        " id integer PRIMARY KEY," +
+        " name varchar(50)," +
+        " age integer," +
+        " salary integer" +
+        ")";
+
+    /** */
+    private static final String SQL_INSERT = "INSERT INTO Person(id, name, age, salary) VALUES (?, ?, ?, ?)";
+
+    /**
+     * @param msg Message to log.
+     */
+    private static void log(String msg) {
+        U.debug(msg);
+    }
+
+    /**
+     * Main entry point.
+     *
+     * @param args Command line arguments.
+     */
+    public static void main(String[] args) {
+        IgniteConfiguration cfg = new IgniteConfiguration().setLocalHost("127.0.0.1");
+
+        try (Ignite node = Ignition.start(cfg)) {
+            try {
+                JdbcBatchLoader ldr = new JdbcBatchLoader();
+
+                ldr.load(10_000_000, 10_000, 8, "127.0.0.1");
+            }
+            catch (Exception e) {
+                log("Failed to load data into cloud");
+
+                e.printStackTrace();
+            }
+        }
+    }
+
+    /**
+     * Load data into cloud.
+     *
+     * @param total Total number of rows to lad.
+     * @param batch Batch size.
+     * @param threads How many threads to use.
+     * @param addr JDBC endpoint address.
+     * @throws Exception If failed to load data to cloud.
+     */
+    public void load(int total, int batch, int threads, String addr) throws Exception {
+        log("Connecting to IGNITE...");
+
+        ComboPooledDataSource dataSrc = new ComboPooledDataSource();
+
+        dataSrc.setDriverClass("org.apache.ignite.IgniteJdbcThinDriver");
+        dataSrc.setJdbcUrl("jdbc:ignite:thin://" + addr);
+
+        try(Connection conn = dataSrc.getConnection()) {
+            Statement stmt = conn.createStatement();
+
+            stmt.execute(SQL_CREATE);
+
+            U.closeQuiet(stmt);
+        }
+
+        int cnt = total / batch;
+
+        CountDownLatch latch = new CountDownLatch(cnt);
+
+        ExecutorService exec = Executors.newFixedThreadPool(threads);
+
+        log("Start loading of " + total + " records...");
+
+        long start = System.currentTimeMillis();
+
+        for (int i = 0; i < cnt; i++)
+            exec.execute(new Worker(dataSrc, i, batch, latch));
+
+        latch.await();
+
+        log("Loading time: " + (System.currentTimeMillis() - start) / 1000 + "seconds");
+        log("Loading finished!");
+
+        U.shutdownNow(JdbcBatchLoader.class, exec, null);
+        dataSrc.close();
+    }
+
+    /**
+     * Class that execute batch loading.
+     */
+    private static class Worker implements Runnable {
+        /** */
+        private final ComboPooledDataSource dataSrc;
+
+        /** */
+        private final int packet;
+
+        /** */
+        private final CountDownLatch latch;
+
+        /** */
+        private final int start;
+
+        /** */
+        private final int finish;
+
+        /**
+         *
+         * @param dataSrc Data source.
+         * @param packet Packet ID.
+         * @param batch Batch size.
+         * @param latch Control latch to complete loading.
+         */
+        private Worker(ComboPooledDataSource dataSrc, int packet, int batch, CountDownLatch latch) {
+            this.dataSrc = dataSrc;
+            this.packet = packet;
+            this.latch = latch;
+
+            start = packet * batch;
+            finish = start + batch;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try(Connection conn = dataSrc.getConnection()) {
+                PreparedStatement pstmt = conn.prepareStatement(SQL_INSERT);
+
+                for (int i = start; i < finish; i++) {
+                    pstmt.setInt(1, i);
+                    pstmt.setString(2, "Some name" + i);
+                    pstmt.setInt(3, 100);
+                    pstmt.setInt(4, 200);
+
+                    pstmt.addBatch();
+                }
+
+                pstmt.executeBatch();
+            }
+            catch (Throwable e) {
+                log("Failed to load packet: [packet=" + packet + ", err=" + e.getMessage() + "]");
+
+                e.printStackTrace();
+            }
+            finally {
+                latch.countDown();
+
+//                log("Processed packed: " + packet);
+            }
+        }
+    }
+}


[13/31] ignite git commit: Merge remote-tracking branch 'origin/ignite-4490' into ignite-4490

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/ignite-4490' into ignite-4490


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

Branch: refs/heads/ignite-6022-proto
Commit: b30729e5ace530d2c05aea5ebb54abb137aefe79
Parents: e11fee9 fdf388b
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Dec 15 18:55:28 2017 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Fri Dec 15 18:55:28 2017 +0300

----------------------------------------------------------------------
 .../query/h2/dml/FastUpdateArguments.java       | 20 ++++++++++++++++----
 1 file changed, 16 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[27/31] ignite git commit: Benchmark with client.

Posted by vo...@apache.org.
Benchmark with client.


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

Branch: refs/heads/ignite-6022-proto
Commit: 392e8eec5cfebf449ea8852161649737096a7bc0
Parents: d23bdd9
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 14:45:42 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 14:45:42 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/query/h2/opt/JdbcBenchmarkRunner.java      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/392e8eec/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
index 18dc575..6b6dc21 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
@@ -16,7 +16,7 @@ import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.atomic.LongAdder;
 
 public class JdbcBenchmarkRunner {
-    private static final long KEY_CNT = 5_000_000;
+    private static final long KEY_CNT = 50_000_000;
 
     private static final int THREAD_CNT = 1;
 


[18/31] ignite git commit: Old code to new approach.

Posted by vo...@apache.org.
Old code to new approach.


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

Branch: refs/heads/ignite-6022-proto
Commit: 94aabdab98888a3783196486a1c08fb1ab1898ad
Parents: 58ffec9
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 11:24:44 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 11:24:44 2017 +0300

----------------------------------------------------------------------
 .../processors/query/h2/dml/DmlArgument.java    |  31 ++++++
 .../processors/query/h2/dml/DmlArguments.java   | 105 +++++++++++++++++++
 .../processors/query/h2/dml/FastUpdate.java     |  91 ++--------------
 .../query/h2/dml/FastUpdateArguments.java       |  10 +-
 4 files changed, 147 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/94aabdab/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArgument.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArgument.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArgument.java
new file mode 100644
index 0000000..b3c3dce
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArgument.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.query.h2.dml;
+
+/**
+ * DML argument
+ */
+public interface DmlArgument {
+    /**
+     * Get argument from parameter list.
+     *
+     * @param params Query input parameters.
+     * @return value.
+     */
+    Object get(Object[] params);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/94aabdab/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArguments.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArguments.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArguments.java
new file mode 100644
index 0000000..87fc588
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArguments.java
@@ -0,0 +1,105 @@
+/*
+ * 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.dml;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlConst;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlParameter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * DML arguments factory.
+ */
+public class DmlArguments {
+    /** Operand that always evaluates as {@code null}. */
+    private final static DmlArgument NULL_ARG = new ConstantArgument(null);
+
+    /**
+     * Create argument from AST element.
+     *
+     * @param el Element.
+     * @return DML argument.
+     */
+    public static DmlArgument create(@Nullable GridSqlElement el) {
+        assert el == null ^ (el instanceof GridSqlConst || el instanceof GridSqlParameter);
+
+        if (el == null)
+            return NULL_ARG;
+
+        if (el instanceof GridSqlConst)
+            return new ConstantArgument(((GridSqlConst)el).value().getObject());
+        else
+            return new ParamArgument(((GridSqlParameter)el).index());
+    }
+
+    /**
+     * Private constructor.
+     */
+    private DmlArguments() {
+        // No-op.
+    }
+
+    /**
+     * Value argument.
+     */
+    private static class ConstantArgument implements DmlArgument {
+        /** Value to return. */
+        private final Object val;
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        private ConstantArgument(Object val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        public Object get(Object[] params) {
+            return val;
+        }
+    }
+
+    /**
+     * Parameter argument.
+     */
+    private static class ParamArgument implements DmlArgument {
+        /** Value to return. */
+        private final int paramIdx;
+
+        /**
+         * Constructor.
+         *
+         * @param paramIdx Parameter index.
+         */
+        private ParamArgument(int paramIdx) {
+            assert paramIdx >= 0;
+
+            this.paramIdx = paramIdx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get(Object[] params) {
+            assert params.length > paramIdx;
+
+            return params[paramIdx];
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/94aabdab/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
index e662245..dcceff3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
@@ -20,26 +20,21 @@ package org.apache.ignite.internal.processors.query.h2.dml;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.query.h2.UpdateResult;
-import org.apache.ignite.internal.processors.query.h2.sql.GridSqlConst;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
-import org.apache.ignite.internal.processors.query.h2.sql.GridSqlParameter;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * Arguments for fast, query-less UPDATE or DELETE - key and, optionally, value and new value.
  */
 public final class FastUpdate {
-    /** Operand that always evaluates as {@code null}. */
-    private final static FastUpdateArgument NULL_ARG = new ConstantArgument(null);
-
     /** Operand to compute key. */
-    private final FastUpdateArgument keyArg;
+    private final DmlArgument keyArg;
 
     /** Operand to compute value. */
-    private final FastUpdateArgument valArg;
+    private final DmlArgument valArg;
 
     /** Operand to compute new value. */
-    private final FastUpdateArgument newValArg;
+    private final DmlArgument newValArg;
 
     /**
      * Create fast update instance.
@@ -50,9 +45,9 @@ public final class FastUpdate {
      * @return Fast update.
      */
     public static FastUpdate create(GridSqlElement key, GridSqlElement val, @Nullable GridSqlElement newVal) {
-        FastUpdateArgument keyArg = argument(key);
-        FastUpdateArgument valArg = argument(val);
-        FastUpdateArgument newValArg = argument(newVal);
+        DmlArgument keyArg = DmlArguments.create(key);
+        DmlArgument valArg = DmlArguments.create(val);
+        DmlArgument newValArg = DmlArguments.create(newVal);
 
         return new FastUpdate(keyArg, valArg, newValArg);
     }
@@ -64,7 +59,7 @@ public final class FastUpdate {
      * @param valArg Value argument.
      * @param newValArg New value argument.
      */
-    private FastUpdate(FastUpdateArgument keyArg, FastUpdateArgument valArg, FastUpdateArgument newValArg) {
+    private FastUpdate(DmlArgument keyArg, DmlArgument valArg, DmlArgument newValArg) {
         this.keyArg = keyArg;
         this.valArg = valArg;
         this.newValArg = newValArg;
@@ -80,12 +75,12 @@ public final class FastUpdate {
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
     public UpdateResult execute(GridCacheAdapter cache, Object[] args) throws IgniteCheckedException {
-        Object key = keyArg.apply(args);
+        Object key = keyArg.get(args);
 
         assert key != null;
 
-        Object val = valArg.apply(args);
-        Object newVal = newValArg.apply(args);
+        Object val = valArg.get(args);
+        Object newVal = newValArg.get(args);
 
         boolean res;
 
@@ -106,70 +101,4 @@ public final class FastUpdate {
 
         return res ? UpdateResult.ONE : UpdateResult.ZERO;
     }
-
-    /**
-     * Create argument for AST element.
-     *
-     * @param el Element.
-     * @return Argument.
-     */
-    private static FastUpdateArgument argument(@Nullable GridSqlElement el) {
-        assert el == null ^ (el instanceof GridSqlConst || el instanceof GridSqlParameter);
-
-        if (el == null)
-            return NULL_ARG;
-
-        if (el instanceof GridSqlConst)
-            return new ConstantArgument(((GridSqlConst)el).value().getObject());
-        else
-            return new ParamArgument(((GridSqlParameter)el).index());
-    }
-
-    /**
-     * Value argument.
-     */
-    private static class ConstantArgument implements FastUpdateArgument {
-        /** Value to return. */
-        private final Object val;
-
-        /**
-         * Constructor.
-         *
-         * @param val Value.
-         */
-        private ConstantArgument(Object val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
-            return val;
-        }
-    }
-
-    /**
-     * Parameter argument.
-     */
-    private static class ParamArgument implements FastUpdateArgument {
-        /** Value to return. */
-        private final int paramIdx;
-
-        /**
-         * Constructor.
-         *
-         * @param paramIdx Parameter index.
-         */
-        private ParamArgument(int paramIdx) {
-            assert paramIdx >= 0;
-
-            this.paramIdx = paramIdx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
-            assert arg.length > paramIdx;
-
-            return arg[paramIdx];
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/94aabdab/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
index 9ba66f1..c7a45a3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
@@ -34,7 +34,7 @@ public final class FastUpdateArguments {
 
     /** */
     public FastUpdateArguments(FastUpdateArgument key, FastUpdateArgument val, FastUpdateArgument newVal) {
-        assert key != null && key != NULL_ARGUMENT;
+        assert key != null;
         assert val != null;
         assert newVal != null;
 
@@ -43,14 +43,6 @@ public final class FastUpdateArguments {
         this.newVal = newVal;
     }
 
-    /** Operand that always evaluates as {@code null}. */
-    public final static FastUpdateArgument NULL_ARGUMENT = new FastUpdateArgument() {
-        /** {@inheritDoc} */
-        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
-            return null;
-        }
-    };
-
     /**
      * Simple constant value based operand.
      */


[25/31] ignite git commit: Benchmark.

Posted by vo...@apache.org.
Benchmark.


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

Branch: refs/heads/ignite-6022-proto
Commit: 5f888e35da89dcb0426bae9df75f37b962020b45
Parents: d57d406
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 14:34:26 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 14:34:26 2017 +0300

----------------------------------------------------------------------
 .../query/h2/opt/JdbcBenchmarkRunner.java       | 181 +++++++++++++++++++
 1 file changed, 181 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5f888e35/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
new file mode 100644
index 0000000..a09cee4
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
@@ -0,0 +1,181 @@
+package org.apache.ignite.internal.processors.query.h2.opt;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequestHandler;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import java.io.File;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.Statement;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.LongAdder;
+
+public class JdbcBenchmarkRunner {
+    private static final long KEY_CNT = 5_000_000;
+
+    private static final int THREAD_CNT = 1;
+
+    private static final int BATCH_SIZE = 10000;
+
+    private static final LongAdder OPS = new LongAdder();
+
+    private static volatile boolean done;
+
+    public static void main(String[] args) throws Exception {
+        U.delete(new File("C:\\Personal\\code\\incubator-ignite\\work"));
+
+        IgniteConfiguration cfg = new IgniteConfiguration().setLocalHost("127.0.0.1");
+
+//        DataStorageConfiguration dsCfg = new DataStorageConfiguration().setWalMode(WALMode.LOG_ONLY);
+//
+//        dsCfg.getDefaultDataRegionConfiguration().setPersistenceEnabled(true);
+//        dsCfg.getDefaultDataRegionConfiguration().setMaxSize(4 * 1024 * 1024 * 1024L);
+//
+//        dsCfg.setCheckpointFrequency(Long.MAX_VALUE);
+//
+//        cfg.setDataStorageConfiguration(dsCfg);
+
+        try (Ignite node = Ignition.start(cfg)) {
+            node.active(true);
+
+            try (Connection conn = connect()) {
+                execute(conn, "CREATE TABLE tbl (id BIGINT PRIMARY KEY, v1 BIGINT, v2 BIGINT, v3 BIGINT, v4 BIGINT)");
+            }
+
+            new Thread(new Runnable() {
+                @Override public void run() {
+                    while (!done) {
+                        long startTime = System.currentTimeMillis();
+                        long startOps = OPS.longValue();
+
+                        try {
+                            Thread.sleep(3000L);
+                        }
+                        catch (InterruptedException e) {
+                            break;
+                        }
+
+                        long endTime = System.currentTimeMillis();
+                        long endOps = OPS.longValue();
+
+                        double t = 1000 * (double)(endOps - startOps) / (double)(endTime - startTime);
+
+                        if (!done)
+                            System.out.println("Throughput: " + String.format("%1$,.2f", t) + " ops/sec");
+                    }
+                }
+            }).start();
+
+            JdbcRequestHandler.STREAMER = true;
+
+            long start = System.currentTimeMillis();
+
+            CyclicBarrier startBarrier = new CyclicBarrier(THREAD_CNT);
+            CountDownLatch stopLatch = new CountDownLatch(THREAD_CNT);
+
+            for (int i = 0; i < THREAD_CNT; i++) {
+                final int i0 = i;
+
+                new Thread(new Runnable() {
+                    @SuppressWarnings("InfiniteLoopStatement")
+                    @Override public void run() {
+                        try (Connection conn = connect()) {
+                            startBarrier.await();
+
+                            doUpdate(conn, i0);
+
+                            execute(conn, "FLUSH");
+                        }
+                        catch (Exception e) {
+                            System.out.println("ERROR: " + e);
+                        }
+                        finally {
+                            stopLatch.countDown();
+                        }
+                    }
+                }).start();
+            }
+
+            stopLatch.await();
+
+            done = true;
+
+            long end = System.currentTimeMillis();
+
+            float dur = (float)((double)(end - start) / 1000);
+
+            System.out.println("TOTAL DURATION: " + dur);
+        }
+    }
+
+    @SuppressWarnings("InfiniteLoopStatement")
+    private static void doUpdate(Connection conn, int idx) throws Exception {
+        long keyCnt = KEY_CNT / THREAD_CNT;
+
+        long startIdx = keyCnt * idx;
+        long endIdx = startIdx + keyCnt;
+
+        System.out.println("INSERT interval [" + startIdx + " -> " + endIdx + ')');
+
+        try (PreparedStatement stmt = conn.prepareStatement("INSERT INTO tbl (id, v1, v2, v3, v4) VALUES (?, ?, ?, ?, ?)")) {
+            if (BATCH_SIZE == 0) {
+                for (long i = startIdx; i < endIdx; i++) {
+                    stmt.setLong(1, i);
+                    stmt.setLong(2, i);
+                    stmt.setLong(3, i);
+                    stmt.setLong(4, i);
+                    stmt.setLong(5, i);
+
+                    stmt.execute();
+
+                    OPS.increment();
+                }
+            }
+            else {
+                int curSize = 0;
+
+                for (long i = startIdx; i < endIdx; i++) {
+                    stmt.setLong(1, i);
+                    stmt.setLong(2, i);
+                    stmt.setLong(3, i);
+                    stmt.setLong(4, i);
+                    stmt.setLong(5, i);
+
+                    stmt.addBatch();
+
+                    curSize++;
+
+                    if (curSize == BATCH_SIZE) {
+                        stmt.executeBatch();
+
+                        OPS.add(curSize);
+
+                        curSize = 0;
+                    }
+                }
+
+                if (curSize > 0) {
+                    stmt.executeBatch();
+
+                    OPS.add(curSize);
+                }
+            }
+
+        }
+    }
+
+    private static Connection connect() throws Exception {
+        return DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1");
+    }
+
+    private static void execute(Connection conn, String sql) throws Exception {
+        try (Statement stmt = conn.createStatement()) {
+            stmt.executeUpdate(sql);
+        }
+    }
+}


[07/31] ignite git commit: Merge branch 'master' into ignite-4490

Posted by vo...@apache.org.
Merge branch 'master' into ignite-4490

# Conflicts:
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java


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

Branch: refs/heads/ignite-6022-proto
Commit: e66b664213633bcd48fe21cd8784135ac5080504
Parents: 336ad01 3919d80
Author: Alexander Paschenko <al...@gmail.com>
Authored: Thu Aug 10 20:44:54 2017 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Thu Aug 10 20:48:51 2017 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   18 +-
 LICENSE                                         |    7 +
 RELEASE_NOTES.txt                               |   33 +
 examples/config/example-default.xml             |    7 -
 examples/config/example-memory-policies.xml     |    7 +-
 examples/pom.xml                                |    9 +-
 .../datagrid/CacheClientBinaryQueryExample.java |    3 +
 .../store/auto/CacheBinaryAutoStoreExample.java |  170 --
 .../datagrid/store/auto/package-info.java       |   22 -
 .../examples/datagrid/CacheQueryDdlExample.java |  118 +
 .../ignite/examples/datagrid/JdbcExample.java   |  135 +
 .../ml/math/matrix/ExampleMatrixStorage.java    |    6 +-
 .../CacheExamplesMultiNodeSelfTest.java         |    9 +
 .../ignite/examples/CacheExamplesSelfTest.java  |    8 +
 modules/aop/pom.xml                             |    2 +-
 modules/apache-license-gen/pom.xml              |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/benchmarks/pom.xml                      |    2 +-
 .../jmh/cache/JmhCacheLocksBenchmark.java       |  119 +
 modules/camel/pom.xml                           |    2 +-
 modules/cassandra/pom.xml                       |    2 +-
 modules/cassandra/serializers/pom.xml           |    4 +-
 modules/cassandra/store/pom.xml                 |    4 +-
 .../ignite/tests/utils/TestTransaction.java     |   10 +
 modules/clients/pom.xml                         |    2 +-
 .../client/suite/IgniteClientTestSuite.java     |    2 +
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |   12 +
 .../jdbc2/JdbcDefaultNoOpCacheTest.java         |   33 +
 .../jdbc2/JdbcDynamicIndexAbstractSelfTest.java |    2 -
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |   35 +
 .../JettyRestProcessorAbstractSelfTest.java     |   70 +-
 .../tcp/redis/RedisProtocolServerSelfTest.java  |  110 +
 .../tcp/redis/RedisProtocolStringSelfTest.java  |   45 +-
 .../ignite/jdbc/JdbcDefaultNoOpCacheTest.java   |   35 +
 .../ignite/jdbc/JdbcNoDefaultCacheTest.java     |   50 +-
 .../jdbc/JdbcPreparedStatementSelfTest.java     |   35 +
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    7 +-
 .../ignite/jdbc/thin/JdbcThinBatchSelfTest.java |  333 +++
 .../JdbcThinDynamicIndexAbstractSelfTest.java   |    2 -
 .../thin/JdbcThinPreparedStatementSelfTest.java |   35 +
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 modules/core/pom.xml                            |    9 +-
 .../java/org/apache/ignite/IgniteCache.java     |    1 -
 .../org/apache/ignite/IgniteDataStreamer.java   |   26 +-
 .../apache/ignite/IgniteSystemProperties.java   |   44 +-
 .../org/apache/ignite/cache/CacheManager.java   |   14 +-
 .../org/apache/ignite/cache/QueryEntity.java    |  229 ++
 .../apache/ignite/cache/affinity/Affinity.java  |   16 +-
 .../rendezvous/RendezvousAffinityFunction.java  |    8 +-
 .../ignite/compute/ComputeJobAdapter.java       |    2 +-
 .../configuration/CacheConfiguration.java       |  620 +---
 .../configuration/IgniteConfiguration.java      |   15 +-
 .../configuration/MemoryConfiguration.java      |    4 +-
 .../MemoryPolicyConfiguration.java              |    4 +-
 .../PersistentStoreConfiguration.java           |  124 +-
 .../org/apache/ignite/events/EventType.java     |   12 +
 .../ignite/events/WalSegmentArchivedEvent.java  |   62 +
 .../apache/ignite/internal/GridComponent.java   |    4 +-
 .../ignite/internal/GridKernalContext.java      |    2 +-
 .../ignite/internal/GridKernalContextImpl.java  |    6 +-
 .../ignite/internal/GridPluginComponent.java    |    2 +-
 .../internal/IgniteDiagnosticMessage.java       |   12 +-
 .../IgniteDiagnosticPrepareContext.java         |    6 +-
 .../apache/ignite/internal/IgniteKernal.java    |  120 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    3 +
 .../org/apache/ignite/internal/IgnitionEx.java  |   66 +-
 .../ignite/internal/MarshallerContextImpl.java  |   14 +-
 .../ignite/internal/binary/BinaryContext.java   |    1 +
 .../ignite/internal/binary/BinaryUtils.java     |   38 +-
 .../internal/binary/BinaryWriterExImpl.java     |    4 +-
 .../internal/binary/GridBinaryMarshaller.java   |    3 +
 .../binary/builder/BinaryBuilderSerializer.java |    6 +-
 .../client/router/impl/GridTcpRouterImpl.java   |   57 +-
 .../internal/jdbc/JdbcPreparedStatement.java    |    6 +-
 .../internal/jdbc/thin/JdbcThinConnection.java  |    7 +-
 .../jdbc/thin/JdbcThinPreparedStatement.java    |   18 +-
 .../internal/jdbc/thin/JdbcThinStatement.java   |   46 +-
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |   20 +
 .../ignite/internal/jdbc2/JdbcConnection.java   |    5 +-
 .../internal/jdbc2/JdbcPreparedStatement.java   |   12 +-
 .../internal/managers/GridManagerAdapter.java   |    2 +-
 .../managers/communication/GridIoManager.java   |    6 +
 .../managers/communication/GridIoMessage.java   |    3 +
 .../communication/GridIoMessageFactory.java     |    2 +-
 .../managers/communication/GridIoPolicy.java    |    3 +
 .../internal/managers/discovery/DiscoCache.java |   17 +-
 .../discovery/DiscoveryLocalJoinData.java       |  104 +
 .../discovery/GridDiscoveryManager.java         |  174 +-
 .../eventstorage/GridEventStorageManager.java   |  323 +-
 .../eventstorage/HighPriorityListener.java      |   28 +
 .../mem/file/MappedFileMemoryProvider.java      |    3 +-
 .../ignite/internal/pagemem/FullPageId.java     |   38 +-
 .../ignite/internal/pagemem/PageIdUtils.java    |   14 +-
 .../ignite/internal/pagemem/PageSupport.java    |   36 +-
 ...ishSnapshotOperationAckDiscoveryMessage.java |   77 -
 .../pagemem/snapshot/SnapshotOperation.java     |  202 --
 .../pagemem/snapshot/SnapshotOperationType.java |   49 -
 ...artSnapshotOperationAckDiscoveryMessage.java |  149 -
 .../StartSnapshotOperationDiscoveryMessage.java |  192 --
 .../pagemem/store/IgnitePageStoreManager.java   |   44 +-
 .../internal/pagemem/store/PageStore.java       |    7 +
 .../internal/pagemem/wal/record/WALRecord.java  |   11 +-
 .../delta/DataPageInsertFragmentRecord.java     |    6 +-
 .../wal/record/delta/DataPageInsertRecord.java  |    6 +-
 .../wal/record/delta/DataPageRemoveRecord.java  |    6 +-
 .../delta/DataPageSetFreeListPageRecord.java    |    6 +-
 .../wal/record/delta/DataPageUpdateRecord.java  |    6 +-
 .../wal/record/delta/FixCountRecord.java        |    6 +-
 .../record/delta/FixLeftmostChildRecord.java    |    6 +-
 .../pagemem/wal/record/delta/FixRemoveId.java   |    6 +-
 .../wal/record/delta/InitNewPageRecord.java     |    6 +-
 .../wal/record/delta/InnerReplaceRecord.java    |    6 +-
 .../pagemem/wal/record/delta/InsertRecord.java  |    6 +-
 .../pagemem/wal/record/delta/MergeRecord.java   |    6 +-
 .../wal/record/delta/MetaPageAddRootRecord.java |    6 +-
 .../wal/record/delta/MetaPageCutRootRecord.java |    6 +-
 .../wal/record/delta/MetaPageInitRecord.java    |    6 +-
 .../delta/MetaPageInitRootInlineRecord.java     |    6 +-
 .../record/delta/MetaPageInitRootRecord.java    |    4 +-
 .../delta/MetaPageUpdateLastAllocatedIndex.java |    6 +-
 ...aPageUpdateLastSuccessfulFullSnapshotId.java |    4 +-
 .../MetaPageUpdateLastSuccessfulSnapshotId.java |    4 +-
 .../delta/MetaPageUpdateNextSnapshotId.java     |    4 +-
 .../MetaPageUpdatePartitionDataRecord.java      |    8 +-
 .../wal/record/delta/NewRootInitRecord.java     |    6 +-
 .../wal/record/delta/PageDeltaRecord.java       |   14 +-
 .../delta/PageListMetaResetCountRecord.java     |    6 +-
 .../record/delta/PagesListAddPageRecord.java    |    6 +-
 .../delta/PagesListInitNewPageRecord.java       |    6 +-
 .../record/delta/PagesListRemovePageRecord.java |    2 +-
 .../record/delta/PagesListSetNextRecord.java    |    6 +-
 .../delta/PagesListSetPreviousRecord.java       |    6 +-
 .../record/delta/PartitionDestroyRecord.java    |   20 +-
 .../record/delta/PartitionMetaStateRecord.java  |   16 +-
 .../pagemem/wal/record/delta/RecycleRecord.java |    6 +-
 .../pagemem/wal/record/delta/RemoveRecord.java  |    6 +-
 .../pagemem/wal/record/delta/ReplaceRecord.java |    6 +-
 .../record/delta/SplitExistingPageRecord.java   |    6 +-
 .../record/delta/SplitForwardPageRecord.java    |    6 +-
 .../record/delta/TrackingPageDeltaRecord.java   |    6 +-
 .../processors/GridProcessorAdapter.java        |    2 +-
 .../affinity/GridAffinityProcessor.java         |    2 +-
 .../cache/CacheAffinitySharedManager.java       |  365 ++-
 .../processors/cache/CacheGroupContext.java     |    4 +-
 .../processors/cache/CacheGroupData.java        |    4 +-
 .../processors/cache/CacheGroupDescriptor.java  |   20 +-
 .../processors/cache/CacheObjectUtils.java      |    4 +-
 .../cache/CacheObjectsReleaseFuture.java        |   60 +
 .../processors/cache/CacheOperationContext.java |   15 +
 .../cache/ChangeGlobalStateMessage.java         |  120 -
 .../processors/cache/ClusterCachesInfo.java     |  643 +++-
 .../internal/processors/cache/ClusterState.java |   38 -
 .../cache/DynamicCacheChangeRequest.java        |   52 +-
 .../cache/DynamicCacheDescriptor.java           |   13 -
 .../processors/cache/ExchangeActions.java       |   71 +-
 .../cache/GatewayProtectedCacheProxy.java       | 1754 +++++++++++
 .../processors/cache/GridCacheAdapter.java      |   78 +-
 .../processors/cache/GridCacheAttributes.java   |   42 +
 .../processors/cache/GridCacheEntryEx.java      |   17 +-
 .../processors/cache/GridCacheEventManager.java |    2 -
 .../cache/GridCacheEvictionManager.java         |    1 -
 .../cache/GridCacheExplicitLockSpan.java        |   10 +-
 .../processors/cache/GridCacheIoManager.java    |   45 +-
 .../processors/cache/GridCacheMapEntry.java     |   59 +-
 .../processors/cache/GridCacheMvccManager.java  |   30 +-
 .../GridCachePartitionExchangeManager.java      |  665 +++--
 .../processors/cache/GridCachePreloader.java    |    8 +-
 .../cache/GridCachePreloaderAdapter.java        |    4 +-
 .../processors/cache/GridCacheProcessor.java    |  332 ++-
 .../cache/GridCacheSharedContext.java           |  103 +-
 .../cache/GridCacheSharedManager.java           |    6 -
 .../cache/GridCacheSharedManagerAdapter.java    |   16 -
 .../processors/cache/GridCacheUtils.java        |    8 +-
 .../cache/GridDeferredAckMessageSender.java     |    3 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |   33 +-
 .../processors/cache/IgniteCacheProxy.java      | 2817 +-----------------
 .../processors/cache/IgniteCacheProxyImpl.java  | 1828 ++++++++++++
 .../processors/cache/PendingDiscoveryEvent.java |   61 +
 .../processors/cache/StateChangeRequest.java    |   77 +
 .../binary/CacheObjectBinaryProcessorImpl.java  |   12 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |    1 -
 .../GridDistributedTxRemoteAdapter.java         |   13 +-
 .../dht/GridClientPartitionTopology.java        |  101 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |    1 -
 .../distributed/dht/GridDhtCacheEntry.java      |    6 -
 .../cache/distributed/dht/GridDhtGetFuture.java |   15 +-
 .../distributed/dht/GridDhtGetSingleFuture.java |    2 -
 .../distributed/dht/GridDhtLocalPartition.java  |   45 -
 .../dht/GridDhtPartitionTopology.java           |   32 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  268 +-
 .../dht/GridDhtTopologyFutureAdapter.java       |    2 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |    9 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   37 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   13 -
 .../distributed/dht/GridDhtTxPrepareFuture.java |   49 +-
 .../dht/GridPartitionedSingleGetFuture.java     |    3 -
 .../dht/atomic/DhtAtomicUpdateResult.java       |  131 +
 .../dht/atomic/GridDhtAtomicCache.java          |  382 +--
 ...idNearAtomicAbstractSingleUpdateRequest.java |    9 +-
 .../GridNearAtomicAbstractUpdateFuture.java     |    3 +-
 ...GridNearAtomicSingleUpdateInvokeRequest.java |    6 +
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |    4 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |    2 +-
 .../preloader/ForceRebalanceExchangeTask.java   |   58 +
 .../dht/preloader/GridDhtForceKeysFuture.java   |    1 -
 .../dht/preloader/GridDhtPartitionDemander.java |   29 +-
 .../preloader/GridDhtPartitionExchangeId.java   |   63 +-
 .../dht/preloader/GridDhtPartitionFullMap.java  |   24 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    7 +-
 .../dht/preloader/GridDhtPartitionSupplier.java |    2 +-
 .../GridDhtPartitionSupplyMessage.java          |   20 +-
 .../GridDhtPartitionsExchangeFuture.java        |  581 ++--
 .../preloader/GridDhtPartitionsFullMessage.java |   48 +-
 .../GridDhtPartitionsSingleMessage.java         |   38 +-
 .../dht/preloader/GridDhtPreloader.java         |   35 +-
 .../preloader/GridDhtPreloaderAssignments.java  |   21 +-
 .../RebalanceReassignExchangeTask.java          |   44 +
 .../distributed/near/GridNearCacheEntry.java    |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 -
 ...arOptimisticSerializableTxPrepareFuture.java |   11 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   17 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   11 +-
 .../near/GridNearTxFinishFuture.java            |    6 +-
 .../cache/distributed/near/GridNearTxLocal.java |   57 +-
 .../near/GridNearTxPrepareRequest.java          |    9 +-
 .../cache/local/GridLocalLockFuture.java        |    5 -
 .../cache/persistence/DataStructure.java        |   46 +-
 .../cache/persistence/DbCheckpointListener.java |   21 +-
 .../FullPageIdIterableComparator.java           |   51 -
 .../GridCacheDatabaseSharedManager.java         |  279 +-
 .../persistence/GridCacheOffheapManager.java    |  102 +-
 .../IgniteCacheDatabaseSharedManager.java       |  161 +-
 .../persistence/IgniteCacheSnapshotManager.java |  140 -
 .../cache/persistence/MetadataStorage.java      |    2 +-
 .../cache/persistence/file/FileIO.java          |  154 +
 .../cache/persistence/file/FileIODecorator.java |   98 +
 .../cache/persistence/file/FileIOFactory.java   |   45 +
 .../cache/persistence/file/FilePageStore.java   |  107 +-
 .../persistence/file/FilePageStoreFactory.java  |   35 +
 .../persistence/file/FilePageStoreManager.java  |   66 +-
 .../cache/persistence/file/FilePageStoreV2.java |   53 +
 .../file/FileVersionCheckingFactory.java        |  116 +
 .../persistence/file/RandomAccessFileIO.java    |  110 +
 .../file/RandomAccessFileIOFactory.java         |   42 +
 .../persistence/freelist/FreeListImpl.java      |   39 +-
 .../cache/persistence/freelist/PagesList.java   |  103 +-
 .../freelist/io/PagesListMetaIO.java            |    2 +-
 .../persistence/pagemem/FullPageIdTable.java    |   54 +-
 .../cache/persistence/pagemem/PageMemoryEx.java |   12 +-
 .../persistence/pagemem/PageMemoryImpl.java     |  184 +-
 .../persistence/partstate/GroupPartitionId.java |  145 +
 .../partstate/PagesAllocationRange.java         |   68 +
 .../partstate/PartitionAllocationMap.java       |  113 +
 .../snapshot/IgniteCacheSnapshotManager.java    |  168 ++
 .../snapshot/SnapshotDiscoveryMessage.java      |   33 +
 .../persistence/snapshot/SnapshotOperation.java |   44 +
 .../cache/persistence/tree/BPlusTree.java       |   16 +-
 .../cache/persistence/tree/io/PageMetaIO.java   |   27 +-
 .../tree/io/PagePartitionCountersIO.java        |    2 +-
 .../persistence/tree/io/TrackingPageIO.java     |   20 +-
 .../persistence/tree/util/PageHandler.java      |    3 +-
 .../wal/AbstractWalRecordsIterator.java         |  291 ++
 .../persistence/wal/ByteBufferExpander.java     |   64 +
 .../cache/persistence/wal/FileInput.java        |   51 +-
 .../cache/persistence/wal/FileWALPointer.java   |    4 +-
 .../wal/FileWriteAheadLogManager.java           |  851 +++---
 .../cache/persistence/wal/RecordSerializer.java |    5 +
 .../persistence/wal/SegmentArchiveResult.java   |   61 +
 .../persistence/wal/SegmentEofException.java    |    3 +-
 .../wal/reader/IgniteWalIteratorFactory.java    |  107 +
 .../wal/reader/StandaloneGridKernalContext.java |  512 ++++
 ...ndaloneIgniteCacheDatabaseSharedManager.java |   30 +
 .../reader/StandaloneIgnitePluginProcessor.java |   38 +
 .../reader/StandaloneWalRecordsIterator.java    |  265 ++
 .../wal/serializer/RecordV1Serializer.java      |  121 +-
 .../query/GridCacheDistributedQueryManager.java |    4 +-
 .../cache/query/GridCacheQueryManager.java      |   44 +-
 .../cache/query/GridCacheQueryResponse.java     |    4 +-
 .../cache/query/QueryEntityClassProperty.java   |  116 +
 .../cache/query/QueryEntityIndexDescriptor.java |  121 +
 .../cache/query/QueryEntityTypeDescriptor.java  |  219 ++
 .../cache/store/CacheStoreManager.java          |    4 +-
 .../store/GridCacheStoreManagerAdapter.java     |   44 +-
 .../cache/transactions/IgniteTxAdapter.java     |  105 +-
 .../cache/transactions/IgniteTxHandler.java     |  178 +-
 .../IgniteTxImplicitSingleStateImpl.java        |    2 +-
 .../transactions/IgniteTxLocalAdapter.java      |   27 +-
 .../cache/transactions/IgniteTxManager.java     |   91 +-
 .../cache/transactions/IgniteTxMap.java         |    2 +-
 .../IgniteTxRemoteStateAdapter.java             |    2 +-
 .../cache/transactions/IgniteTxState.java       |    2 +-
 .../cache/transactions/IgniteTxStateImpl.java   |    2 +-
 .../transactions/TransactionProxyImpl.java      |   46 +-
 .../cache/version/GridCacheVersionManager.java  |    6 -
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 -
 .../IgniteCacheObjectProcessorImpl.java         |    5 -
 .../cluster/ChangeGlobalStateFinishMessage.java |   86 +
 .../cluster/ChangeGlobalStateMessage.java       |  140 +
 .../processors/cluster/ClusterProcessor.java    |    3 +-
 .../cluster/DiscoveryDataClusterState.java      |  157 +
 .../cluster/GridClusterStateProcessor.java      | 1134 +++----
 .../cluster/IgniteChangeGlobalStateSupport.java |    3 +-
 .../continuous/GridContinuousProcessor.java     |   91 +-
 .../datastreamer/DataStreamProcessor.java       |    3 +-
 .../datastreamer/DataStreamerImpl.java          |  295 +-
 .../datastreamer/DataStreamerRequest.java       |   59 +-
 .../datastructures/DataStructuresProcessor.java |   18 +-
 .../datastructures/GridCacheAtomicLongImpl.java |    2 +-
 .../GridCacheAtomicReferenceImpl.java           |    2 +-
 .../GridCacheAtomicSequenceImpl.java            |    2 +-
 .../GridCacheAtomicStampedImpl.java             |    2 +-
 .../GridCacheCountDownLatchImpl.java            |    2 +-
 .../datastructures/GridCacheLockImpl.java       |   17 +-
 .../datastructures/GridCacheQueueAdapter.java   |    1 -
 .../datastructures/GridCacheSemaphoreImpl.java  |    2 +-
 .../datastructures/GridCacheSetImpl.java        |    1 -
 .../dr/IgniteDrDataStreamerCacheUpdater.java    |    2 +-
 .../processors/igfs/IgfsDataManager.java        |    2 +-
 .../internal/processors/igfs/IgfsImpl.java      |    4 +-
 .../processors/igfs/IgfsNoopProcessor.java      |   11 +
 .../internal/processors/igfs/IgfsProcessor.java |   14 +-
 .../processors/igfs/IgfsProcessorAdapter.java   |    3 +-
 .../processors/job/GridJobProcessor.java        |   15 +-
 .../GridMarshallerMappingProcessor.java         |   16 +-
 .../processors/odbc/SqlListenerProcessor.java   |    3 +-
 .../odbc/jdbc/JdbcBatchExecuteRequest.java      |  109 +
 .../odbc/jdbc/JdbcBatchExecuteResult.java       |   96 +
 .../processors/odbc/jdbc/JdbcQuery.java         |   95 +
 .../processors/odbc/jdbc/JdbcRequest.java       |    8 +
 .../odbc/jdbc/JdbcRequestHandler.java           |   66 +-
 .../processors/odbc/jdbc/JdbcResult.java        |   11 +
 .../platform/PlatformAbstractTarget.java        |    6 +-
 .../processors/platform/PlatformIgnition.java   |    5 +-
 .../platform/PlatformNoopProcessor.java         |  132 -
 .../processors/platform/PlatformProcessor.java  |  216 --
 .../platform/PlatformProcessorImpl.java         |  554 ++--
 .../platform/PlatformTargetProxy.java           |   11 +
 .../platform/PlatformTargetProxyImpl.java       |   79 +-
 .../platform/cache/PlatformCache.java           |   33 +-
 .../platform/cluster/PlatformClusterGroup.java  |   68 +-
 .../datastructures/PlatformAtomicReference.java |   13 +-
 .../dotnet/PlatformDotNetCacheStore.java        |   31 +
 .../transactions/PlatformTransactions.java      |    2 +-
 .../utils/PlatformConfigurationUtils.java       |   24 +-
 .../platform/utils/PlatformUtils.java           |    3 +-
 .../processors/query/GridQueryProcessor.java    |    8 +-
 .../internal/processors/query/QueryUtils.java   |    8 +-
 .../processors/rest/GridRestCommand.java        |    6 +
 .../processors/rest/GridRestProcessor.java      |    4 +-
 .../handlers/cache/GridCacheCommandHandler.java |  202 +-
 .../cluster/GridChangeStateCommandHandler.java  |    2 +-
 .../key/GridRedisExpireCommandHandler.java      |  101 +
 .../rest/handlers/redis/package-info.java       |   22 +
 .../server/GridRedisFlushCommandHandler.java    |  117 +
 .../string/GridRedisIncrDecrCommandHandler.java |    2 +-
 .../protocols/tcp/redis/GridRedisCommand.java   |   10 +-
 .../tcp/redis/GridRedisNioListener.java         |    4 +
 .../service/GridServiceProcessor.java           |   61 +-
 .../processors/service/GridServiceProxy.java    |    9 +-
 .../processors/service/ServiceContextImpl.java  |    2 +
 .../processors/task/GridTaskProcessor.java      |   20 +-
 .../timeout/GridTimeoutProcessor.java           |   18 +-
 .../ignite/internal/util/IgniteUtils.java       |  119 +-
 .../internal/util/MutableSingletonList.java     |   53 +
 .../util/StripedCompositeReadWriteLock.java     |    6 +-
 .../ignite/internal/util/StripedExecutor.java   |   93 +-
 .../util/future/GridCompoundFuture.java         |    6 +-
 .../ignite/internal/util/nio/GridNioServer.java |   21 +-
 .../util/tostring/GridToStringBuilder.java      |    2 +-
 .../visor/VisorCoordinatorNodeTask.java         |   39 +
 .../visor/cache/VisorCacheConfiguration.java    |   11 +-
 .../visor/cache/VisorCacheJdbcType.java         |    2 +-
 .../internal/visor/cache/VisorCacheMetrics.java |   12 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |    2 +-
 .../internal/visor/debug/VisorThreadInfo.java   |   10 +-
 .../visor/node/VisorBasicConfiguration.java     |    2 +-
 .../node/VisorMemoryPolicyConfiguration.java    |   13 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |   32 +-
 .../node/VisorNodeDataCollectorJobResult.java   |   73 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |   14 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |   28 +-
 .../visor/node/VisorPersistenceMetrics.java     |  214 ++
 .../node/VisorPersistentStoreConfiguration.java |   36 +-
 .../internal/visor/query/VisorQueryEntity.java  |   32 +-
 .../internal/visor/query/VisorQueryTask.java    |    2 +-
 .../visor/query/VisorScanQueryTask.java         |    2 +-
 .../visor/service/VisorServiceTask.java         |   10 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   10 +-
 .../spi/IgniteSpiOperationTimeoutHelper.java    |    8 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  253 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  123 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   16 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |    6 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |    2 +-
 .../org/apache/ignite/thread/IgniteThread.java  |   45 +-
 .../ignite/thread/IgniteThreadFactory.java      |   15 +-
 .../ignite/thread/IgniteThreadPoolExecutor.java |  141 +-
 .../apache/ignite/transactions/Transaction.java |   14 +
 .../ignite/transactions/TransactionState.java   |    7 +-
 .../resources/META-INF/classnames.properties    |   27 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../org.apache.ignite.plugin.PluginProvider     |    3 +-
 .../affinity/AffinityHistoryCleanupTest.java    |  182 --
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |   55 +-
 .../cache/store/jdbc/model/BinaryTest.java      |  121 +
 .../cache/store/jdbc/model/BinaryTestKey.java   |   87 +
 .../ignite/internal/GridVersionSelfTest.java    |    2 +
 .../IgniteClientReconnectAbstractTest.java      |    5 +
 .../internal/IgniteComputeJobOneThreadTest.java |   75 +
 ...UpdateNotifierPerClusterSettingSelfTest.java |    2 +
 .../internal/TestRecordingCommunicationSpi.java |   10 +
 .../internal/binary/BinaryEnumsSelfTest.java    |   71 +
 .../binary/BinaryMarshallerSelfTest.java        |   61 +
 .../BinaryObjectBuilderAdditionalSelfTest.java  |    2 +-
 ...GridManagerLocalMessageListenerSelfTest.java |    4 +-
 ...unicationBalanceMultipleConnectionsTest.java |    5 +
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |   52 +-
 .../cache/CacheConnectionLeakStoreTxTest.java   |  291 ++
 .../cache/CacheEntryProcessorCopySelfTest.java  |   11 +-
 .../CacheGroupLocalConfigurationSelfTest.java   |  155 +
 .../cache/CacheStopAndDestroySelfTest.java      |    1 +
 ...idAbstractCacheInterceptorRebalanceTest.java |    2 -
 .../GridCacheAbstractLocalStoreSelfTest.java    |    7 +
 .../cache/GridCacheAbstractMetricsSelfTest.java |   24 -
 .../cache/GridCacheAbstractSelfTest.java        |    1 +
 .../cache/GridCacheDeploymentSelfTest.java      |    7 +-
 .../cache/GridCacheLuceneQueryIndexTest.java    |  466 ---
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |    6 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |   15 +
 .../GridCacheQueryIndexingDisabledSelfTest.java |   16 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   10 +-
 ...idCacheValueConsistencyAbstractSelfTest.java |    4 +-
 .../cache/IgniteActiveClusterTest.java          |  182 --
 .../cache/IgniteCacheAbstractTest.java          |   17 +
 .../IgniteCacheBinaryObjectsScanSelfTest.java   |    9 +-
 ...acheBinaryObjectsScanWithEventsSelfTest.java |   30 +
 .../IgniteCacheConfigVariationsFullApiTest.java |    6 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |    1 -
 .../processors/cache/IgniteCacheGroupsTest.java |  149 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |   22 +-
 .../cache/IgniteCachePutAllRestartTest.java     |   15 +
 .../cache/IgniteCacheStartStopLoadTest.java     |    1 -
 .../IgniteClusterActivateDeactivateTest.java    | 1289 ++++++++
 ...erActivateDeactivateTestWithPersistence.java |  197 ++
 .../IgniteDaemonNodeMarshallerCacheTest.java    |   10 -
 .../cache/IgniteDynamicCacheMultinodeTest.java  |  171 ++
 ...iteMarshallerCacheClassNameConflictTest.java |   11 +-
 .../IgniteMarshallerCacheFSRestoreTest.java     |  219 ++
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   28 +
 .../IgniteSemaphoreAbstractSelfTest.java        |   17 +-
 .../CacheGetInsideLockChangingTopologyTest.java |    5 +
 ...ffinityAssignmentNodeJoinValidationTest.java |   46 +-
 .../CacheLateAffinityAssignmentTest.java        |   40 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java |    2 -
 .../distributed/CachePartitionStateTest.java    |  410 +++
 .../GridCacheAbstractNodeRestartSelfTest.java   |   12 +-
 ...titionEvictionDuringReadThroughSelfTest.java |  185 ++
 ...niteBinaryMetadataUpdateNodeRestartTest.java |   10 +
 .../distributed/IgniteCacheGetRestartTest.java  |    5 +
 .../IgniteCacheNearRestartRollbackSelfTest.java |   15 +
 ...ptimisticTxSuspendResumeMultiServerTest.java |   30 +
 .../IgniteOptimisticTxSuspendResumeTest.java    |  751 +++++
 .../IgnitePessimisticTxSuspendResumeTest.java   |   91 +
 .../IgniteRejectConnectOnNodeStopTest.java      |    7 +-
 ...teSynchronizationModesMultithreadedTest.java |    5 +
 ...cOriginatingNodeFailureAbstractSelfTest.java |    2 +-
 .../IgniteTxRemoveTimeoutObjectsTest.java       |  194 ++
 .../TestCacheNodeExcludingFilter.java           |   53 +
 ...idCachePartitionedPreloadEventsSelfTest.java |  143 -
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    2 -
 .../IgniteCachePutRetryAbstractSelfTest.java    |    5 +
 .../dht/NotMappedPartitionInTxTest.java         |  264 ++
 ...LateAffDisabledMultiNodeFullApiSelfTest.java |   35 -
 ...LateAffDisabledMultiNodeFullApiSelfTest.java |   34 -
 .../GridCachePartitionedTxSalvageSelfTest.java  |    8 +-
 ...cingDelayedPartitionMapExchangeSelfTest.java |   58 +-
 .../GridCacheReplicatedNodeRestartSelfTest.java |    2 +-
 ...acheStoreSessionWriteBehindAbstractTest.java |   62 +-
 ...TxStoreSessionWriteBehindCoalescingTest.java |   88 +
 .../IgnitePdsCacheRebalancingAbstractTest.java  |   76 +
 ...gnitePdsRecoveryAfterFileCorruptionTest.java |   35 +-
 ...IgnitePersistentStoreDataStructuresTest.java |    2 +
 .../MemoryPolicyInitializationTest.java         |    3 +-
 .../db/IgnitePdsCacheRestoreTest.java           |  208 ++
 .../db/IgnitePdsWholeClusterRestartTest.java    |    3 +-
 ...ckpointSimulationWithRealCpDisabledTest.java |   74 +-
 .../db/file/IgnitePdsEvictionTest.java          |   16 +-
 .../persistence/db/wal/IgnitePdsWalTlbTest.java |    2 +-
 .../db/wal/IgniteWalFlushFailoverTest.java      |  195 ++
 .../wal/IgniteWalHistoryReservationsTest.java   |   31 +-
 .../db/wal/IgniteWalRecoveryTest.java           |  110 +-
 .../db/wal/crc/IgniteDataIntegrityTests.java    |   10 +-
 .../db/wal/reader/IgniteWalReaderTest.java      |  388 +++
 .../db/wal/reader/MockWalIteratorFactory.java   |  120 +
 .../pagemem/NoOpPageStoreManager.java           |   34 +-
 .../persistence/pagemem/NoOpWALManager.java     |   23 +-
 .../AbstractNodeJoinTemplate.java               |  149 +-
 .../IgniteChangeGlobalStateAbstractTest.java    |   65 +-
 .../IgniteChangeGlobalStateCacheTest.java       |    2 +-
 ...IgniteChangeGlobalStateDataStreamerTest.java |    5 +-
 ...gniteChangeGlobalStateDataStructureTest.java |    6 +-
 .../IgniteChangeGlobalStateFailOverTest.java    |   68 +-
 .../IgniteChangeGlobalStateTest.java            |  158 +-
 .../IgniteStandByClusterTest.java               |  181 +-
 .../GridActivationPartitionedCacheSuit.java     |    2 -
 .../join/JoinActiveNodeToActiveCluster.java     |   62 +-
 ...ctiveNodeToActiveClusterWithPersistence.java |   17 +
 .../IgniteStandByClientReconnectTest.java       |   13 +-
 ...eStandByClientReconnectToNewClusterTest.java |   13 +-
 .../ScanQueryOffheapExpiryPolicySelfTest.java   |  114 +
 ...ContinuousQueryFailoverAbstractSelfTest.java |    2 -
 ...ClientWriteBehindStoreNonCoalescingTest.java |   30 +-
 .../TxOptimisticDeadlockDetectionTest.java      |    2 +
 .../processors/database/BPlusTreeSelfTest.java  |    4 +-
 .../database/FreeListImplSelfTest.java          |    4 +-
 .../database/SwapPathConstructionSelfTest.java  |  157 +
 .../DataStreamProcessorSelfTest.java            |   14 +-
 .../datastreamer/DataStreamerImplSelfTest.java  |  124 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |    1 -
 .../GridServiceProcessorProxySelfTest.java      |   65 +
 .../timeout/GridTimeoutProcessorSelfTest.java   |   68 +-
 .../internal/util/IgniteUtilsSelfTest.java      |   26 +
 .../loadtests/colocation/GridTestMain.java      |   45 -
 .../loadtests/hashmap/GridCacheTestContext.java |    6 +-
 .../marshaller/GridMarshallerAbstractTest.java  |   10 +-
 .../marshaller/GridMarshallerResourceBean.java  |    5 +-
 .../plugin/PlatformTestPluginTarget.java        |    7 +-
 .../org/apache/ignite/spi/GridTcpForwarder.java |   26 +
 ...cpCommunicationSpiMultithreadedSelfTest.java |    2 +-
 .../tcp/TcpCommunicationSpiDropNodesTest.java   |   15 +
 .../TcpCommunicationSpiFaultyClientTest.java    |   20 +-
 .../ignite/testframework/GridTestUtils.java     |   26 +
 .../testframework/junits/GridAbstractTest.java  |   11 +-
 .../junits/GridTestKernalContext.java           |   10 +
 .../cache/GridAbstractCacheStoreSelfTest.java   |   10 +
 .../junits/common/GridCommonAbstractTest.java   |   15 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    5 +
 .../IgniteCacheFullApiSelfTestSuite.java        |    6 -
 .../IgniteCacheRestartTestSuite2.java           |    2 +
 .../testsuites/IgniteCacheTestSuite2.java       |    4 +-
 .../testsuites/IgniteCacheTestSuite3.java       |    4 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    6 +
 .../testsuites/IgniteCacheTestSuite5.java       |    4 +
 .../testsuites/IgniteCacheTestSuite6.java       |   46 +
 .../testsuites/IgniteComputeGridTestSuite.java  |    3 +
 .../ignite/testsuites/IgnitePdsTestSuite.java   |    8 +-
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   13 +-
 .../testsuites/IgniteStandByClusterSuite.java   |    5 +-
 .../testsuites/IgniteUtilSelfTestSuite.java     |    2 +
 .../GridThreadPoolExecutorServiceSelfTest.java  |    9 +-
 .../util/mbeans/GridMBeanDisableSelfTest.java   |  121 +
 modules/docker/2.1.0/Dockerfile                 |   46 +
 modules/docker/2.1.0/run.sh                     |   51 +
 modules/docker/Dockerfile                       |    2 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 .../ignite/tests/p2p/JobStealingTask.java       |   12 +-
 .../org/apache/ignite/tests/p2p/NodeFilter.java |   30 +
 .../extdata/uri/modules/uri-dependency/pom.xml  |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/flink/pom.xml                           |    2 +-
 modules/flume/pom.xml                           |    2 +-
 .../apache/ignite/stream/flume/IgniteSink.java  |    7 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |   26 +-
 .../query/h2/opt/GridH2SpatialIndex.java        |    7 -
 modules/hadoop/pom.xml                          |    2 +-
 .../processors/hadoop/HadoopProcessor.java      |    4 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |   24 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |    5 +-
 ...ileSystemShmemExternalDualAsyncSelfTest.java |    5 +
 modules/hibernate-4.2/pom.xml                   |   26 +-
 modules/hibernate-5.1/pom.xml                   |   26 +-
 modules/hibernate-core/pom.xml                  |    2 +-
 modules/indexing/pom.xml                        |   14 +-
 .../cache/query/CacheQueryPartitionInfo.java    |   46 +-
 .../query/h2/DmlStatementsProcessor.java        |   29 +-
 .../processors/query/h2/H2DatabaseType.java     |   10 +
 .../processors/query/h2/H2RowDescriptor.java    |   24 +-
 .../processors/query/h2/H2TableEngine.java      |    4 +-
 .../internal/processors/query/h2/H2Utils.java   |   27 +
 .../processors/query/h2/IgniteH2Indexing.java   |   51 +-
 .../query/h2/database/H2PkHashIndex.java        |    7 -
 .../query/h2/database/H2TreeIndex.java          |   32 +-
 .../query/h2/database/InlineIndexHelper.java    |  416 ++-
 .../query/h2/opt/GridH2IndexBase.java           |   93 +-
 .../query/h2/opt/GridH2QueryContext.java        |   59 -
 .../query/h2/opt/GridH2RowDescriptor.java       |    5 -
 .../processors/query/h2/opt/GridH2Table.java    |  221 +-
 .../query/h2/opt/GridH2TreeIndex.java           |  602 ----
 .../query/h2/opt/GridLuceneDirectory.java       |  107 +-
 .../processors/query/h2/opt/GridLuceneFile.java |   91 +-
 .../query/h2/opt/GridLuceneIndex.java           |   78 +-
 .../query/h2/opt/GridLuceneInputStream.java     |  120 +-
 .../query/h2/opt/GridLuceneLockFactory.java     |   45 +-
 .../query/h2/opt/GridLuceneOutputStream.java    |   82 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |    5 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  526 +---
 .../query/h2/twostep/MapNodeResults.java        |  108 +
 .../query/h2/twostep/MapQueryResult.java        |  258 ++
 .../query/h2/twostep/MapQueryResults.java       |  155 +
 .../h2/twostep/MapReplicatedReservation.java    |   38 +
 .../query/h2/twostep/MapRequestKey.java         |   65 +
 .../query/h2/twostep/MapReservationKey.java     |   73 +
 .../cache/GridCacheFullTextQuerySelfTest.java   |  367 +++
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |   30 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    3 +-
 ...teCacheFullTextQueryNodeJoiningSelfTest.java |    7 +-
 ...niteCachePartitionedFieldsQuerySelfTest.java |   20 +-
 .../DynamicIndexAbstractConcurrentSelfTest.java |   81 +-
 .../cache/index/H2DynamicTableSelfTest.java     |   85 +
 ...eDbSingleNodeWithIndexingWalRestoreTest.java |  197 ++
 ...oreQueryWithMultipleClassesPerCacheTest.java |  185 ++
 .../processors/query/IgniteSqlRoutingTest.java  |  211 +-
 .../h2/database/InlineIndexHelperTest.java      |  252 +-
 .../query/h2/opt/GridH2TableSelfTest.java       |  172 --
 .../h2/sql/AbstractH2CompareQueryTest.java      |    4 +-
 .../IgniteBinaryCacheQueryTestSuite.java        |    2 +
 .../IgniteCacheQuerySelfTestSuite.java          |    9 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |    3 +
 .../IgniteCacheWithIndexingTestSuite.java       |    3 +
 .../IgnitePdsWithIndexingTestSuite.java         |    4 +
 modules/jcl/pom.xml                             |    2 +-
 modules/jms11/pom.xml                           |    2 +-
 modules/jta/pom.xml                             |    2 +-
 .../processors/cache/jta/CacheJtaManager.java   |    5 +-
 .../processors/cache/jta/CacheJtaResource.java  |   48 +-
 .../cache/AbstractCacheJtaSelfTest.java         |  183 --
 .../GridJtaTransactionManagerSelfTest.java      |  208 ++
 .../GridPartitionedCacheJtaFactorySelfTest.java |   41 -
 ...rtitionedCacheJtaFactoryUseSyncSelfTest.java |   32 -
 ...titionedCacheJtaLookupClassNameSelfTest.java |   83 -
 .../GridReplicatedCacheJtaFactorySelfTest.java  |   32 -
 ...eplicatedCacheJtaFactoryUseSyncSelfTest.java |   32 -
 ...plicatedCacheJtaLookupClassNameSelfTest.java |   32 -
 .../cache/jta/AbstractCacheJtaSelfTest.java     |  248 ++
 .../GridPartitionedCacheJtaFactorySelfTest.java |   41 +
 ...rtitionedCacheJtaFactoryUseSyncSelfTest.java |   32 +
 ...titionedCacheJtaLookupClassNameSelfTest.java |   83 +
 .../GridReplicatedCacheJtaFactorySelfTest.java  |   32 +
 ...eplicatedCacheJtaFactoryUseSyncSelfTest.java |   32 +
 ...plicatedCacheJtaLookupClassNameSelfTest.java |   32 +
 .../processors/cache/jta/package-info.java      |   22 +
 .../ignite/testsuites/IgniteJtaTestSuite.java   |   15 +-
 modules/kafka/pom.xml                           |    2 +-
 modules/kubernetes/pom.xml                      |    2 +-
 modules/log4j/pom.xml                           |    2 +-
 modules/log4j2/pom.xml                          |    2 +-
 modules/mesos/pom.xml                           |    2 +-
 .../apache/ignite/mesos/ClusterProperties.java  |    4 +-
 modules/ml/README.txt                           |    6 +
 modules/ml/licenses/bsd3.txt                    |   51 +
 modules/ml/pom.xml                              |   25 +-
 .../main/java/org/apache/ignite/ml/Model.java   |    1 -
 .../ml/clustering/BaseKMeansClusterer.java      |    2 -
 .../apache/ignite/ml/clustering/Clusterer.java  |    3 +-
 .../clustering/KMeansDistributedClusterer.java  |   52 +-
 .../ml/clustering/KMeansLocalClusterer.java     |   13 +-
 .../java/org/apache/ignite/ml/math/Blas.java    |  355 +++
 .../apache/ignite/ml/math/DistanceMeasure.java  |    3 +-
 .../ignite/ml/math/EuclideanDistance.java       |    5 +-
 .../org/apache/ignite/ml/math/MathUtils.java    |    1 +
 .../java/org/apache/ignite/ml/math/Matrix.java  |   33 +-
 .../apache/ignite/ml/math/MatrixStorage.java    |   12 +-
 .../apache/ignite/ml/math/OrderedMatrix.java    |   24 +
 .../java/org/apache/ignite/ml/math/Tracer.java  |    1 -
 .../java/org/apache/ignite/ml/math/Vector.java  |    8 +
 .../org/apache/ignite/ml/math/VectorUtils.java  |   11 +-
 .../decompositions/CholeskyDecomposition.java   |   11 +-
 .../math/decompositions/EigenDecomposition.java |    2 +-
 .../ml/math/decompositions/LUDecomposition.java |    2 +-
 .../ml/math/decompositions/QRDecomposition.java |    2 +-
 .../math/exceptions/ConvergenceException.java   |    5 +-
 .../exceptions/MathIllegalNumberException.java  |    8 +-
 .../exceptions/MathIllegalStateException.java   |    4 +-
 .../ignite/ml/math/functions/Functions.java     |    8 +-
 .../IgniteIntDoubleToDoubleBiFunction.java      |   27 +
 .../functions/IgniteIntIntToIntBiFunction.java  |   27 +
 .../ml/math/functions/IgniteTriFunction.java    |   35 +
 .../apache/ignite/ml/math/impls/CacheUtils.java |  219 +-
 .../ml/math/impls/matrix/AbstractMatrix.java    |  153 +-
 .../ignite/ml/math/impls/matrix/BlockEntry.java |   50 +
 .../ml/math/impls/matrix/CacheMatrix.java       |    9 +-
 .../impls/matrix/DenseLocalOffHeapMatrix.java   |   29 +
 .../impls/matrix/DenseLocalOnHeapMatrix.java    |   61 +-
 .../matrix/SparseBlockDistributedMatrix.java    |  208 ++
 .../impls/matrix/SparseDistributedMatrix.java   |   59 +-
 .../impls/matrix/SparseLocalOnHeapMatrix.java   |   27 +
 .../storage/matrix/ArrayMatrixStorage.java      |   86 +-
 .../storage/matrix/BaseBlockMatrixKey.java      |   41 +
 .../impls/storage/matrix/BlockMatrixKey.java    |  144 +
 .../storage/matrix/BlockMatrixStorage.java      |  440 +++
 .../storage/matrix/CacheMatrixStorage.java      |    6 +
 .../matrix/DenseOffHeapMatrixStorage.java       |   10 +-
 .../storage/matrix/DiagonalMatrixStorage.java   |   17 +
 .../storage/matrix/FunctionMatrixStorage.java   |    6 +
 .../impls/storage/matrix/MapWrapperStorage.java |    8 +-
 .../storage/matrix/MatrixDelegateStorage.java   |   43 +-
 .../storage/matrix/PivotedMatrixStorage.java    |    5 +
 .../storage/matrix/RandomMatrixStorage.java     |    6 +
 .../matrix/SparseDistributedMatrixStorage.java  |   24 +-
 .../matrix/SparseLocalOnHeapMatrixStorage.java  |   41 +-
 .../vector/DenseLocalOffHeapVectorStorage.java  |    2 +-
 .../storage/vector/MatrixVectorStorage.java     |   11 +
 .../vector/SparseLocalOffHeapVectorStorage.java |    2 +-
 .../vector/SparseLocalOnHeapVectorStorage.java  |   31 +-
 .../impls/vector/AbstractReadOnlyVector.java    |    6 +
 .../ml/math/impls/vector/AbstractVector.java    |    8 +
 .../ml/math/impls/vector/CacheVector.java       |    4 +-
 .../ml/math/impls/vector/DelegatingVector.java  |    6 +
 .../ml/math/impls/vector/MapWrapperVector.java  |    4 +-
 .../ml/math/impls/vector/SparseLocalVector.java |   45 +-
 .../ignite/ml/math/impls/vector/VectorView.java |    2 +-
 .../ignite/ml/math/statistics/Variance.java     |    1 +
 .../ignite/ml/math/statistics/package-info.java |   22 +
 .../org/apache/ignite/ml/math/util/MapUtil.java |    2 +-
 .../apache/ignite/ml/math/util/MatrixUtil.java  |   73 +-
 .../ignite/ml/math/util/package-info.java       |   22 +
 .../java/org/apache/ignite/ml/package-info.java |   22 +
 .../AbstractMultipleLinearRegression.java       |    4 +-
 .../OLSMultipleLinearRegression.java            |    2 +-
 .../ignite/ml/math/d3-matrix-template.html      |    2 +
 .../ignite/ml/math/d3-vector-template.html      |    2 +
 .../org/apache/ignite/ml/IgniteMLTestSuite.java |    4 +-
 .../java/org/apache/ignite/ml/TestUtils.java    |    2 +-
 .../ml/clustering/ClusteringTesetSuite.java     |   32 -
 .../ml/clustering/ClusteringTestSuite.java      |   32 +
 .../KMeansDistributedClustererTest.java         |   43 +-
 .../org/apache/ignite/ml/math/BlasTest.java     |  346 +++
 .../ml/math/MathImplDistributedTestSuite.java   |    2 +
 .../ignite/ml/math/MathImplLocalTestSuite.java  |    1 -
 .../ignite/ml/math/MathImplMainTestSuite.java   |    3 +-
 .../org/apache/ignite/ml/math/TracerTest.java   |    3 +
 .../ignite/ml/math/benchmark/MathBenchmark.java |    2 +-
 .../ml/math/benchmark/VectorBenchmarkTest.java  |    2 +-
 .../decompositions/EigenDecompositionTest.java  |    2 +-
 .../math/impls/matrix/MatrixAttributeTest.java  |    2 -
 .../impls/matrix/MatrixImplementationsTest.java |   12 +-
 .../impls/matrix/MatrixViewConstructorTest.java |    2 +-
 .../SparseDistributedBlockMatrixTest.java       |  379 +++
 .../matrix/SparseDistributedMatrixTest.java     |   56 +-
 .../storage/matrix/MatrixArrayStorageTest.java  |    6 +-
 .../matrix/MatrixStorageImplementationTest.java |    2 +-
 .../RandomAccessSparseVectorStorageTest.java    |    4 +-
 .../math/impls/vector/AbstractVectorTest.java   |    4 +-
 .../PivotedVectorViewConstructorTest.java       |    2 +-
 .../math/impls/vector/VectorAttributesTest.java |    2 +-
 .../ml/math/impls/vector/VectorFoldMapTest.java |    2 +-
 .../impls/vector/VectorImplementationsTest.java |   12 +-
 .../math/impls/vector/VectorIterableTest.java   |    2 +-
 .../math/impls/vector/VectorToMatrixTest.java   |    7 +-
 .../OLSMultipleLinearRegressionTest.java        |    8 +-
 modules/mqtt/pom.xml                            |    2 +-
 .../apache/ignite/stream/mqtt/MqttStreamer.java |    6 +-
 modules/osgi-karaf/pom.xml                      |    2 +-
 modules/osgi-paxlogging/pom.xml                 |    2 +-
 modules/osgi/pom.xml                            |    2 +-
 .../include/ignite/binary/binary_writer.h       |    4 +-
 .../ignite/impl/binary/binary_reader_impl.h     |   50 +-
 .../ignite/impl/binary/binary_type_impl.h       |   67 +
 .../ignite/impl/binary/binary_writer_impl.h     |   43 +-
 .../src/impl/binary/binary_reader_impl.cpp      |   63 +-
 .../src/impl/binary/binary_writer_impl.cpp      |   41 +-
 modules/platforms/cpp/common/configure.ac       |    2 +-
 modules/platforms/cpp/configure.ac              |    2 +-
 modules/platforms/cpp/configure.acrel           |    2 +-
 .../core-test/config/cache-query-default.xml    |   29 +
 modules/platforms/cpp/core-test/configure.ac    |    2 +-
 .../src/binary_reader_writer_raw_test.cpp       |   36 +
 .../core-test/src/binary_reader_writer_test.cpp |  135 +-
 .../cpp/core-test/src/cache_query_test.cpp      |  215 +-
 .../cpp/core-test/src/cluster_test.cpp          |   13 +
 .../cpp/core-test/src/compute_test.cpp          |   91 +-
 modules/platforms/cpp/core/configure.ac         |    2 +-
 modules/platforms/cpp/core/include/Makefile.am  |    2 +
 .../ignite/cache/query/query_fields_row.h       |   28 +
 .../core/include/ignite/cache/query/query_sql.h |    4 +
 .../ignite/cache/query/query_sql_fields.h       |   17 +-
 .../cpp/core/include/ignite/compute/compute.h   |   66 +
 .../platforms/cpp/core/include/ignite/ignite.h  |   21 +-
 .../ignite/impl/cache/query/query_argument.h    |   63 +
 .../impl/cache/query/query_fields_row_impl.h    |   29 +
 .../ignite/impl/cluster/cluster_group_impl.h    |   15 +
 .../include/ignite/impl/compute/compute_impl.h  |  164 +-
 .../ignite/impl/compute/compute_job_result.h    |   54 +-
 .../ignite/impl/compute/compute_task_holder.h   |  204 +-
 .../compute/multiple_job_compute_task_holder.h  |  265 ++
 .../compute/single_job_compute_task_holder.h    |  212 ++
 .../cpp/core/include/ignite/impl/ignite_impl.h  |  106 +-
 .../ignite/impl/interop/interop_target.h        |   16 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |    2 +
 .../cpp/core/project/vs/core.vcxproj.filters    |    6 +
 modules/platforms/cpp/core/src/ignite.cpp       |   10 +
 modules/platforms/cpp/core/src/ignition.cpp     |   23 +-
 .../src/impl/cluster/cluster_group_impl.cpp     |   26 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |   37 +-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |   53 +-
 .../core/src/impl/interop/interop_target.cpp    |   18 +-
 modules/platforms/cpp/examples/configure.ac     |    2 +-
 modules/platforms/cpp/ignite/configure.ac       |    2 +-
 .../cpp/jni/include/ignite/jni/exports.h        |   32 +-
 .../platforms/cpp/jni/include/ignite/jni/java.h |   75 +-
 modules/platforms/cpp/jni/project/vs/module.def |   31 +-
 modules/platforms/cpp/jni/src/exports.cpp       |  120 +-
 modules/platforms/cpp/jni/src/java.cpp          |  537 +---
 modules/platforms/cpp/odbc-test/Makefile.am     |    1 +
 .../cpp/odbc-test/include/complex_type.h        |   53 +-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |    1 +
 .../project/vs/odbc-test.vcxproj.filters        |    3 +
 .../cpp/odbc-test/src/meta_queries_test.cpp     |  240 ++
 .../cpp/odbc/install/ignite-odbc-amd64.wxs      |    2 +-
 .../cpp/odbc/install/ignite-odbc-x86.wxs        |    2 +-
 .../platforms/cpp/odbc/src/meta/column_meta.cpp |    3 +
 modules/platforms/cpp/odbc/src/odbc.cpp         |    2 +-
 .../cpp/odbc/src/query/type_info_query.cpp      |    2 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |    2 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite.Core.Tests.csproj             |    8 +
 .../Binary/BinaryCompactFooterInteropTest.cs    |   29 +-
 .../Binary/BinaryFooterTest.cs                  |  178 ++
 .../Binary/BinarySelfTest.cs                    |   32 -
 .../Serializable/AdvancedSerializationTest.cs   |   31 +
 .../BasicSerializableObjectsTest.cs             |    7 +-
 .../Cache/CacheAbstractTest.cs                  |   26 +-
 .../Cache/CacheAbstractTransactionalTest.cs     |   53 +-
 .../Cache/CacheConfigurationTest.cs             |   21 +
 .../Cache/PersistentStoreTest.cs                |  122 +-
 .../Cache/Query/CacheQueriesTest.cs             |   37 +-
 .../Cache/Store/CacheStoreSessionTest.cs        |   90 +-
 .../Store/CacheStoreSessionTestCodeConfig.cs    |   68 +
 .../Store/CacheStoreSessionTestSharedFactory.cs |   48 +
 .../Cache/Store/CacheStoreTest.cs               |   31 +-
 .../cache-store-session-shared-factory.xml      |   76 +
 .../Config/Cache/Store/cache-store-session.xml  |   20 +-
 .../Config/cache-query.xml                      |    1 -
 .../Config/spring-test.xml                      |    4 +
 .../Deployment/CacheGetFunc.cs                  |   50 +
 .../Deployment/PeerAssemblyLoadingTest.cs       |   24 +
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |   19 +-
 .../IgniteConfigurationSerializerTest.cs        |   13 +-
 .../IgniteConfigurationTest.cs                  |   26 +-
 .../Plugin/PluginTest.cs                        |   13 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |    7 +-
 .../Apache.Ignite.Core.csproj                   |    7 +-
 .../Cache/Configuration/CacheConfiguration.cs   |   78 +-
 .../Apache.Ignite.Core/Common/JavaException.cs  |   15 +-
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |    9 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |   20 +-
 .../IgniteConfigurationSection.xsd              |   15 +
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |   21 +-
 .../Impl/Binary/BinaryObject.cs                 |    2 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |    2 +-
 .../Impl/Binary/BinaryObjectSchemaField.cs      |    3 +
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |   93 +-
 .../Impl/Binary/BinaryProcessor.cs              |    6 +-
 .../Impl/Binary/BinaryReader.cs                 |   49 +-
 .../Impl/Binary/BinaryWriterExtensions.cs       |  107 +
 .../Impl/Binary/SerializableSerializer.cs       |   18 +-
 .../Cache/Affinity/PlatformAffinityFunction.cs  |    7 +-
 .../Impl/Cache/CacheAffinityImpl.cs             |   18 +-
 .../Impl/Cache/CacheEnumerator.cs               |    8 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  115 +-
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |   12 +-
 .../Continuous/ContinuousQueryHandleImpl.cs     |   12 +-
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |    6 +-
 .../Impl/Cache/Query/QueryCursor.cs             |    5 +-
 .../Impl/Cache/Store/CacheStoreInternal.cs      |   14 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |  152 +-
 .../Impl/Common/DelegateTypeDescriptor.cs       |    9 +-
 .../Impl/Common/Listenable.cs                   |    8 +-
 .../Impl/Compute/ComputeImpl.cs                 |   18 +-
 .../Impl/DataStructures/AtomicLong.cs           |    9 +-
 .../Impl/DataStructures/AtomicReference.cs      |    8 +-
 .../Impl/DataStructures/AtomicSequence.cs       |    9 +-
 .../Impl/Datastream/DataStreamerImpl.cs         |   11 +-
 .../Impl/Datastream/StreamReceiverHolder.cs     |   13 +-
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |   11 +-
 .../Impl/Events/RemoteListenEventFilter.cs      |    3 +
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |    5 +-
 .../Impl/IPlatformTargetInternal.cs             |  102 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  308 +-
 .../Apache.Ignite.Core/Impl/Log/JavaLogger.cs   |   23 +-
 .../Impl/Messaging/Messaging.cs                 |   10 +-
 .../PersistentStore/PersistentStoreMetrics.cs   |   88 +
 .../Impl/PlatformDisposableTargetAdapter.cs     |   75 +
 .../Impl/PlatformJniTarget.cs                   |  536 ++++
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   | 1070 -------
 .../Impl/PlatformTargetAdapter.cs               |  534 ++++
 .../Impl/Plugin/PluginContext.cs                |    7 +-
 .../Impl/Services/Services.cs                   |   19 +-
 .../Impl/Transactions/TransactionsImpl.cs       |   29 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |   94 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |   10 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  315 +-
 .../Interop/IPlatformTarget.cs                  |   15 +
 .../NuGet/LINQPad/ComputeExample.linq           |    1 +
 .../PersistentStore/IPersistentStoreMetrics.cs  |   87 +
 .../PersistentStoreConfiguration.cs             |  175 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |    6 +-
 modules/rest-http/pom.xml                       |    2 +-
 modules/rocketmq/pom.xml                        |    5 +-
 modules/scalar-2.10/pom.xml                     |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |   26 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spark-2.10/pom.xml                      |    2 +-
 modules/spark/pom.xml                           |    2 +-
 .../org/apache/ignite/spark/IgniteRDD.scala     |    6 +-
 .../apache/ignite/spark/impl/IgniteSqlRDD.scala |    6 +-
 modules/spring-data/pom.xml                     |    2 +-
 modules/spring/pom.xml                          |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/storm/pom.xml                           |    2 +-
 modules/tools/pom.xml                           |    2 +-
 modules/twitter/pom.xml                         |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |    3 +-
 modules/visor-console-2.10/pom.xml              |    2 +-
 modules/visor-console/pom.xml                   |    2 +-
 .../ignite/visor/commands/VisorConsole.scala    |   12 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   20 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web-console/backend/.eslintrc           |   39 +-
 modules/web-console/backend/app/agentSocket.js  |    4 +-
 .../web-console/backend/app/browsersHandler.js  |    2 +-
 modules/web-console/backend/app/mongo.js        |    6 +-
 modules/web-console/backend/app/settings.js     |    3 +-
 modules/web-console/backend/index.js            |   16 +-
 modules/web-console/backend/middlewares/api.js  |   39 +-
 modules/web-console/backend/package.json        |   27 +-
 .../web-console/backend/services/activities.js  |   12 +-
 modules/web-console/frontend/.eslintrc          |   40 +-
 modules/web-console/frontend/app/app.config.js  |    3 +-
 modules/web-console/frontend/app/app.js         |   39 +-
 .../components/activities-user-dialog/index.js  |   28 +-
 .../app/components/bs-select-menu/style.scss    |   16 +
 .../app/components/bs-select-menu/template.pug  |    2 +
 .../cluster-select/cluster-select.controller.js |   72 +-
 .../cluster-select/cluster-select.pug           |   10 +-
 .../components/grid-item-selected/component.js  |   28 +
 .../components/grid-item-selected/controller.js |   38 +
 .../app/components/grid-item-selected/index.js  |   24 +
 .../components/grid-item-selected/template.pug  |   17 +
 .../list-of-registered-users.tpl.pug            |    4 +-
 .../page-configure-advanced/controller.js       |    9 +-
 .../components/page-configure-basic/service.js  |    8 +-
 .../app/components/page-configure/controller.js |   12 +-
 .../page-configure/controller.spec.js           |   48 -
 .../page-configure/services/PageConfigure.js    |   18 +-
 .../app/components/ui-grid-hovering/cell.js     |   48 +
 .../app/components/ui-grid-hovering/hovering.js |   31 +
 .../app/components/ui-grid-hovering/index.js    |   30 +
 .../app/components/ui-grid-hovering/style.scss  |   22 +
 .../app/components/ui-grid-hovering/viewport.js |   42 +
 .../components/web-console-header/component.js  |   12 +-
 .../app/components/web-console-header/index.js  |    4 +-
 .../components/web-console-header/style.scss    |    6 +-
 .../frontend/app/controllers/auth.controller.js |    9 +-
 .../frontend/app/data/jdbc-types.json           |    6 +-
 .../frontend/app/data/pom-dependencies.json     |   10 +-
 .../frontend/app/decorator/select.js            |   77 -
 .../frontend/app/decorator/tooltip.js           |   73 -
 .../hide-on-state-change.directive.js           |    8 +-
 .../frontend/app/filters/duration.filter.js     |    2 +-
 .../frontend/app/filters/id8.filter.js          |   20 +
 .../helpers/jade/form/form-field-checkbox.pug   |    2 +-
 .../helpers/jade/form/form-field-dropdown.pug   |    1 -
 .../frontend/app/helpers/jade/mixins.pug        |    1 +
 .../frontend/app/modules/ace.module.js          |    6 +-
 .../app/modules/agent/AgentManager.service.js   |   28 +-
 .../app/modules/agent/AgentModal.service.js     |    8 +-
 .../app/modules/branding/branding.module.js     |   16 +-
 .../generator/AbstractTransformer.js            |    4 +-
 .../generator/ConfigurationGenerator.js         |   43 +-
 .../configuration/generator/Maven.service.js    |   39 +-
 .../generator/PlatformGenerator.js              |    4 +-
 .../generator/defaults/Cluster.service.js       |    2 +-
 .../frontend/app/modules/demo/Demo.module.js    |   34 +-
 .../app/modules/dialog/dialog.controller.js     |    4 -
 .../app/modules/form/field/label.directive.js   |    4 +-
 .../app/modules/form/field/tooltip.directive.js |    4 +-
 .../frontend/app/modules/sql/sql.controller.js  |   13 +-
 .../frontend/app/modules/sql/sql.module.js      |   66 +-
 .../frontend/app/modules/states/admin.state.js  |    6 +-
 .../app/modules/states/configuration.state.js   |   35 +-
 .../states/configuration/caches/affinity.pug    |   18 +-
 .../states/configuration/caches/general.pug     |   50 +-
 .../states/configuration/clusters/hadoop.pug    |    2 +-
 .../states/configuration/clusters/memory.pug    |    8 +-
 .../configuration/clusters/persistence.pug      |    2 +-
 .../states/configuration/domains/query.pug      |    9 +
 .../summary/summary-zipper.service.js           |    2 +-
 .../configuration/summary/summary.worker.js     |    4 +-
 .../frontend/app/modules/states/errors.state.js |   12 +-
 .../frontend/app/modules/states/logout.state.js |   12 +-
 .../app/modules/states/password.state.js        |    6 +-
 .../app/modules/states/profile.state.js         |    9 +-
 .../frontend/app/modules/states/signin.state.js |   28 +-
 .../app/modules/user/AclRoute.provider.js       |   52 -
 .../frontend/app/modules/user/user.module.js    |   29 +-
 .../frontend/app/primitives/dropdown/index.scss |   64 -
 .../app/primitives/form-field/index.scss        |   48 +-
 .../frontend/app/primitives/index.js            |    2 +
 .../frontend/app/primitives/modal/index.scss    |    9 +
 .../frontend/app/primitives/panel/index.scss    |    2 +-
 .../frontend/app/primitives/radio/index.pug     |   41 +
 .../frontend/app/primitives/radio/index.scss    |   78 +
 .../frontend/app/primitives/switch/index.pug    |   34 -
 .../frontend/app/primitives/switch/index.scss   |   87 -
 .../frontend/app/primitives/switcher/index.scss |    2 +-
 .../frontend/app/primitives/tooltip/index.scss  |   25 +
 .../app/primitives/typography/index.scss        |    2 +-
 .../app/primitives/ui-grid-settings/index.scss  |   47 +-
 .../frontend/app/primitives/ui-grid/index.scss  |   13 +-
 .../services/AngularStrapSelect.decorator.js    |   77 +
 .../services/AngularStrapTooltip.decorator.js   |  103 +
 .../frontend/app/services/Confirm.service.js    |    2 +-
 .../app/services/CopyToClipboard.service.js     |    4 +-
 .../app/services/LegacyTable.service.js         |    2 +-
 .../frontend/app/services/Messages.service.js   |   10 +
 modules/web-console/frontend/app/vendor.js      |    8 +-
 .../frontend/controllers/domains-controller.js  |    7 +-
 modules/web-console/frontend/package.json       |   59 +-
 .../views/configuration/domains-import.tpl.pug  |    4 +-
 .../frontend/views/includes/header-left.pug     |   41 +-
 modules/web-console/frontend/views/index.pug    |    6 +-
 .../web-console/frontend/views/signin.tpl.pug   |    2 +-
 .../web-console/frontend/views/sql/sql.tpl.pug  |    4 +-
 .../views/templates/agent-download.tpl.pug      |    6 +-
 .../frontend/webpack/webpack.common.js          |    4 +-
 modules/web-console/pom.xml                     |    2 +-
 modules/web-console/web-agent/pom.xml           |    2 +-
 .../console/agent/handlers/RestListener.java    |    7 +-
 .../ignite/console/agent/rest/RestExecutor.java |   49 +-
 .../ignite/console/demo/AgentClusterDemo.java   |   12 +
 .../demo/service/DemoCachesLoadService.java     |    1 +
 modules/web/ignite-appserver-test/pom.xml       |    2 +-
 modules/web/ignite-websphere-test/pom.xml       |    2 +-
 modules/web/pom.xml                             |    2 +-
 .../cache/websession/WebSessionFilter.java      |    2 +-
 modules/yardstick/pom.xml                       |    2 +-
 .../yardstick/IgniteBenchmarkArguments.java     |   79 +
 .../cache/IgniteCacheLockBenchmark.java         |   57 +
 .../yardstick/cache/IgniteLockBenchmark.java    |   47 +
 .../cache/IgnitePutTxLoadBenchmark.java         |  118 +
 .../cache/IgniteStreamerBenchmark.java          |  234 ++
 modules/yarn/pom.xml                            |    2 +-
 modules/zeromq/pom.xml                          |    2 +-
 modules/zookeeper/pom.xml                       |    2 +-
 parent/pom.xml                                  |    4 +-
 pom.xml                                         |    2 +-
 1063 files changed, 40439 insertions(+), 19954 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e66b6642/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 09b4a27,4f7c288..a7c44c9
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@@ -80,8 -81,13 +82,12 @@@ import org.h2.command.dml.Insert
  import org.h2.command.dml.Merge;
  import org.h2.command.dml.Update;
  import org.h2.table.Column;
- import org.h2.value.DataType;
+ import org.h2.util.DateTimeUtils;
+ import org.h2.util.LocalDateTimeUtils;
 -import org.h2.value.DataType;
  import org.h2.value.Value;
+ import org.h2.value.ValueDate;
+ import org.h2.value.ValueTime;
+ import org.h2.value.ValueTimestamp;
  import org.jetbrains.annotations.NotNull;
  import org.jetbrains.annotations.Nullable;
  
@@@ -757,21 -748,7 +775,14 @@@ public class DmlStatementsProcessor 
              return newArr;
          }
  
-         int objType = DataType.getTypeFromClass(val.getClass());
- 
-         if (objType == type)
-             return val;
- 
-         Value h2Val = desc.wrap(val, objType);
- 
-         Object res = h2Val.convertTo(type).getObject();
 -        return H2Utils.convert(val, desc, type);
++        Object res = H2Utils.convert(val, desc, type);
 +
 +        if (res instanceof Date && res.getClass() != Date.class && expCls == Date.class) {
 +            // We can get a Timestamp instead of Date when converting a String to Date without query - let's handle this
 +            return new Date(((Date) res).getTime());
 +        }
 +
 +        return res;
      }
  
      /**


[19/31] ignite git commit: Old approach to new classes.

Posted by vo...@apache.org.
Old approach to new classes.


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

Branch: refs/heads/ignite-6022-proto
Commit: 7a9cd6acee77aad378c41ad52f096489afc5c543
Parents: 94aabda
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 11:31:18 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 11:31:18 2017 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        |  6 +-
 .../query/h2/dml/FastUpdateArgument.java        | 27 ------
 .../query/h2/dml/FastUpdateArguments.java       | 93 --------------------
 .../processors/query/h2/dml/UpdatePlan.java     |  6 +-
 .../query/h2/dml/UpdatePlanBuilder.java         | 16 ++--
 5 files changed, 11 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7a9cd6ac/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 8de756c..4ca034e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -48,11 +48,11 @@ import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.h2.dml.DmlArgument;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlBatchSender;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlDistributedPlanInfo;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils;
 import org.apache.ignite.internal.processors.query.h2.dml.FastUpdate;
-import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
@@ -421,11 +421,11 @@ public class DmlStatementsProcessor {
 
         GridH2RowDescriptor desc = plan.table().rowDescriptor();
 
-        for (List<FastUpdateArgument> argRow : plan.rows()) {
+        for (List<DmlArgument> argRow : plan.rows()) {
             List<Object> row = new ArrayList<>();
 
             for (int j = 0; j < plan.columnNames().length; j++) {
-                Object colVal = argRow.get(j).apply(args);
+                Object colVal = argRow.get(j).get(args);
 
                 if (j == plan.keyColumnIndex() || j == plan.valueColumnIndex())
                     colVal = DmlUtils.convert(colVal, desc, j == plan.keyColumnIndex() ? desc.type().keyClass() :

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a9cd6ac/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java
deleted file mode 100644
index dc90fe9..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.h2.dml;
-
-import org.apache.ignite.internal.util.lang.GridPlainClosure;
-
-/**
- * Operand for fast UPDATE or DELETE (single item operation that does not involve any SELECT).
- */
-public interface FastUpdateArgument extends GridPlainClosure<Object[], Object> {
-    // No-op.
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a9cd6ac/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
deleted file mode 100644
index c7a45a3..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.h2.dml;
-
-import org.apache.ignite.IgniteCheckedException;
-
-/**
- * Arguments for fast, query-less UPDATE or DELETE - key and, optionally, value and new value.
- */
-public final class FastUpdateArguments {
-    /** Operand to compute key. */
-    public final FastUpdateArgument key;
-
-    /** Operand to compute value. */
-    public final FastUpdateArgument val;
-
-    /** Operand to compute new value. */
-    public final FastUpdateArgument newVal;
-
-    /** */
-    public FastUpdateArguments(FastUpdateArgument key, FastUpdateArgument val, FastUpdateArgument newVal) {
-        assert key != null;
-        assert val != null;
-        assert newVal != null;
-
-        this.key = key;
-        this.val = val;
-        this.newVal = newVal;
-    }
-
-    /**
-     * Simple constant value based operand.
-     */
-    public final static class ValueArgument implements FastUpdateArgument {
-        /** Value to return. */
-        private final Object val;
-
-        /**
-         * Constructor.
-         *
-         * @param val Value.
-         */
-        public ValueArgument(Object val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
-            return val;
-        }
-    }
-
-    /**
-     * User given param value operand.
-     */
-    public final static class ParamArgument implements FastUpdateArgument {
-        /** Index of param to take. */
-        private final int paramIdx;
-
-        /**
-         * Constructor.
-         *
-         * @param paramIdx Parameter index.
-         */
-        public ParamArgument(int paramIdx) {
-            assert paramIdx >= 0;
-
-            this.paramIdx = paramIdx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
-            assert arg.length > paramIdx;
-
-            return arg[paramIdx];
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a9cd6ac/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 b1b5ada..f33331a 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
@@ -74,7 +74,7 @@ public final class UpdatePlan {
     private final boolean isLocSubqry;
 
     /** Rows for query-less MERGE or INSERT. */
-    private final List<List<FastUpdateArgument>> rows;
+    private final List<List<DmlArgument>> rows;
 
     /** Number of rows in rows based MERGE or INSERT. */
     private final int rowsNum;
@@ -114,7 +114,7 @@ public final class UpdatePlan {
         int valColIdx,
         String selectQry,
         boolean isLocSubqry,
-        List<List<FastUpdateArgument>> rows,
+        List<List<DmlArgument>> rows,
         int rowsNum,
         @Nullable FastUpdate fastUpdate,
         @Nullable DmlDistributedPlanInfo distributed
@@ -410,7 +410,7 @@ public final class UpdatePlan {
     /**
      * @return Rows for query-less MERGE or INSERT.
      */
-    public List<List<FastUpdateArgument>> rows() {
+    public List<List<DmlArgument>> rows() {
         return rows;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a9cd6ac/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index d04cea9..bbbff17 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -42,12 +42,10 @@ import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlColumn;
-import org.apache.ignite.internal.processors.query.h2.sql.GridSqlConst;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDelete;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlInsert;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlMerge;
-import org.apache.ignite.internal.processors.query.h2.sql.GridSqlParameter;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuery;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
@@ -132,7 +130,7 @@ public final class UpdatePlanBuilder {
 
         List<GridSqlElement[]> elRows = null;
 
-        List<List<FastUpdateArgument>> rows = null;
+        List<List<DmlArgument>> rows = null;
 
         if (stmt instanceof GridSqlInsert) {
             GridSqlInsert ins = (GridSqlInsert) stmt;
@@ -178,16 +176,12 @@ public final class UpdatePlanBuilder {
             rows = new ArrayList<>(elRows.size());
 
             for (GridSqlElement[] elRow : elRows) {
-                List<FastUpdateArgument> row = new ArrayList<>(cols.length);
+                List<DmlArgument> row = new ArrayList<>(cols.length);
 
                 for (GridSqlElement e : elRow) {
-                    if (e instanceof GridSqlConst)
-                        row.add(new FastUpdateArguments.ValueArgument(((GridSqlConst) e).value().getObject()));
-                    else if (e instanceof GridSqlParameter)
-                        row.add(new FastUpdateArguments.ParamArgument(((GridSqlParameter) e).index()));
-                    else
-                        throw new IgniteSQLException("Unexpected element type: " + e.getClass().getSimpleName(),
-                            IgniteQueryErrorCode.UNEXPECTED_ELEMENT_TYPE);
+                    DmlArgument arg = DmlArguments.create(e);
+
+                    row.add(arg);
                 }
 
                 rows.add(row);


[29/31] ignite git commit: Minors.

Posted by vo...@apache.org.
Minors.


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

Branch: refs/heads/ignite-6022-proto
Commit: b87d6c331c5948d50d16cdd3ed3a23dfe23fbfc8
Parents: 2df2c1f
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 16:50:34 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 16:50:34 2017 +0300

----------------------------------------------------------------------
 .../odbc/jdbc/JdbcRequestHandler.java           |  2 +-
 .../query/h2/opt/JdbcBatchLoader.java           | 29 +++++++++++---------
 2 files changed, 17 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b87d6c33/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index 7e508a8..1bda70c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -465,7 +465,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
         }
     }
 
-    public static volatile boolean STREAMER = true;
+    public static volatile boolean STREAMER = false;//Boolean.getBoolean("IGNITE_JDBC_STREAMER");
 
     /**
      * @param req Request.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b87d6c33/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBatchLoader.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBatchLoader.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBatchLoader.java
index f145f58..31d67f0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBatchLoader.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBatchLoader.java
@@ -44,7 +44,7 @@ public class JdbcBatchLoader {
             try {
                 JdbcBatchLoader ldr = new JdbcBatchLoader();
 
-                ldr.load(10_000_000, 10_000, 8, "127.0.0.1");
+                ldr.load(10_000_000, 10_000, 1, "127.0.0.1");
             }
             catch (Exception e) {
                 log("Failed to load data into cloud");
@@ -64,6 +64,8 @@ public class JdbcBatchLoader {
      * @throws Exception If failed to load data to cloud.
      */
     public void load(int total, int batch, int threads, String addr) throws Exception {
+        ExecutorService exec = Executors.newFixedThreadPool(threads);
+
         log("Connecting to IGNITE...");
 
         ComboPooledDataSource dataSrc = new ComboPooledDataSource();
@@ -76,28 +78,29 @@ public class JdbcBatchLoader {
 
             stmt.execute(SQL_CREATE);
 
-            U.closeQuiet(stmt);
-        }
+            int cnt = total / batch;
 
-        int cnt = total / batch;
+            CountDownLatch latch = new CountDownLatch(cnt);
 
-        CountDownLatch latch = new CountDownLatch(cnt);
+            log("Start loading of " + total + " records...");
 
-        ExecutorService exec = Executors.newFixedThreadPool(threads);
+            long start = System.currentTimeMillis();
 
-        log("Start loading of " + total + " records...");
+            for (int i = 0; i < cnt; i++)
+                exec.execute(new Worker(dataSrc, i, batch, latch));
 
-        long start = System.currentTimeMillis();
+            latch.await();
 
-        for (int i = 0; i < cnt; i++)
-            exec.execute(new Worker(dataSrc, i, batch, latch));
+            stmt.execute("FLUSH");
 
-        latch.await();
+            U.closeQuiet(stmt);
 
-        log("Loading time: " + (System.currentTimeMillis() - start) / 1000 + "seconds");
-        log("Loading finished!");
+            log("Loading time: " + (System.currentTimeMillis() - start) / 1000 + "seconds");
+            log("Loading finished!");
+        }
 
         U.shutdownNow(JdbcBatchLoader.class, exec, null);
+
         dataSrc.close();
     }
 


[10/31] ignite git commit: Merge remote-tracking branch 'apache/master' into ignite-4490

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/799098c6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index a7c44c9,243d1dc..9e41bfe
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@@ -56,12 -48,10 +48,13 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.query.GridQueryCancel;
  import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
  import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
- import org.apache.ignite.internal.processors.query.GridQueryProperty;
- import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
  import org.apache.ignite.internal.processors.query.IgniteSQLException;
 +import org.apache.ignite.internal.processors.query.QueryUtils;
 +import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
 +import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
+ import org.apache.ignite.internal.processors.query.h2.dml.DmlBatchSender;
+ import org.apache.ignite.internal.processors.query.h2.dml.DmlDistributedPlanInfo;
+ import org.apache.ignite.internal.processors.query.h2.dml.FastUpdate;
  import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode;
  import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
  import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
@@@ -352,24 -341,32 +352,34 @@@ public class DmlStatementsProcessor 
  
          Integer errKeysPos = null;
  
-         UpdatePlan plan = getPlanForStatement(schemaName, prepStmt, errKeysPos);
+         UpdatePlan plan = getPlanForStatement(schemaName, c, prepared, fieldsQry, loc, errKeysPos);
+ 
+         FastUpdate fastUpdate = plan.fastUpdate();
  
-         if (plan.fastUpdateArgs != null) {
+         if (fastUpdate != null) {
              assert F.isEmpty(failedKeys) && errKeysPos == null;
  
-             return doFastUpdate(plan, fieldsQry.getArgs());
+             return fastUpdate.execute(plan.cacheContext().cache(), fieldsQry.getArgs());
          }
  
-         assert !F.isEmpty(plan.rows) ^ !F.isEmpty(plan.selectQry);
+         if (plan.distributedPlan() != null) {
+             UpdateResult result = doDistributedUpdate(schemaName, fieldsQry, plan, cancel);
+ 
+             // null is returned in case not all nodes support distributed DML.
+             if (result != null)
+                 return result;
+         }
+ 
+         assert !F.isEmpty(plan.selectQuery());
  
 -        QueryCursorImpl<List<?>> cur;
 +        Iterable<List<?>> cur;
  
          // Do a two-step query only if locality flag is not set AND if plan's SELECT corresponds to an actual
-         // subquery and not some dummy stuff like "select 1, 2, 3;"
-         if (!loc && !plan.isLocSubqry) {
-             assert !F.isEmpty(plan.selectQry);
+         // sub-query and not some dummy stuff like "select 1, 2, 3;"
+         if (!loc && !plan.isLocalSubquery()) {
++            assert !F.isEmpty(plan.selectQuery());
 +
-             SqlFieldsQuery newFieldsQry = new SqlFieldsQuery(plan.selectQry, fieldsQry.isCollocated())
+             SqlFieldsQuery newFieldsQry = new SqlFieldsQuery(plan.selectQuery(), fieldsQry.isCollocated())
                  .setArgs(fieldsQry.getArgs())
                  .setDistributedJoins(fieldsQry.isDistributedJoins())
                  .setEnforceJoinOrder(fieldsQry.isEnforceJoinOrder())
@@@ -377,11 -374,11 +387,11 @@@
                  .setPageSize(fieldsQry.getPageSize())
                  .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
  
-             cur = (QueryCursorImpl<List<?>>) idx.queryDistributedSqlFields(schemaName, newFieldsQry, true, cancel,
-                 mainCacheId);
+             cur = (QueryCursorImpl<List<?>>)idx.queryDistributedSqlFields(schemaName, newFieldsQry, true,
+                 cancel, mainCacheId, true).get(0);
          }
 -        else {
 -            final GridQueryFieldsResult res = idx.queryLocalSqlFields(schemaName, plan.selectQuery(),
 +        else if (F.isEmpty(plan.rows)) {
-             final GridQueryFieldsResult res = idx.queryLocalSqlFields(schemaName, plan.selectQry,
++            final GridQueryFieldsResult res = idx.queryLocalSqlFields(schemaName, plan.selectQry(),
                  F.asList(fieldsQry.getArgs()), filters, fieldsQry.isEnforceJoinOrder(), fieldsQry.getTimeout(), cancel);
  
              cur = new QueryCursorImpl<>(new Iterable<List<?>>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/799098c6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
index 0000000,054e708..a2cd553
mode 000000,100644..100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
@@@ -1,0 -1,609 +1,639 @@@
+ /*
+  * 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.dml;
+ 
+ import java.util.ArrayList;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Set;
+ import org.apache.ignite.IgniteException;
+ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+ import org.apache.ignite.internal.processors.query.IgniteSQLException;
+ import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap;
+ import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+ import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlAlias;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlArray;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlAst;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlColumn;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlConst;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDelete;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunction;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlJoin;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlKeyword;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlOperation;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlOperationType;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlParameter;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuery;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlSelect;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlSubquery;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlTable;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlType;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlUnion;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlUpdate;
+ import org.apache.ignite.internal.util.lang.IgnitePair;
+ import org.apache.ignite.internal.util.typedef.F;
+ import org.apache.ignite.internal.util.typedef.internal.U;
+ import org.apache.ignite.lang.IgnitePredicate;
+ import org.h2.command.Parser;
+ import org.h2.expression.Expression;
+ import org.h2.table.Column;
+ import org.h2.table.Table;
+ import org.h2.util.IntArray;
+ import org.h2.value.DataType;
+ import org.h2.value.Value;
+ import org.h2.value.ValueDate;
+ import org.h2.value.ValueInt;
+ import org.h2.value.ValueString;
+ import org.h2.value.ValueTime;
+ import org.h2.value.ValueTimestamp;
+ import org.jetbrains.annotations.Nullable;
+ 
+ /**
+  * AST utils for DML
+  */
+ public final class DmlAstUtils {
+     /**
+      * Empty ctor to prevent initialization.
+      */
+     private DmlAstUtils() {
+         // No-op.
+     }
+ 
+     /**
+      * Create SELECT on which subsequent INSERT or MERGE will be based.
+      *
+      * @param cols Columns to insert values into.
+      * @param rows Rows to create pseudo-SELECT upon.
+      * @param subQry Subquery to use rather than rows.
 -     * @return Subquery or pseudo-SELECT to evaluate inserted expressions.
++     * @param desc Row descriptor.
++     * @return Subquery or pseudo-SELECT to evaluate inserted expressions, or {@code null} no query needs to be run.
+      */
+     public static GridSqlQuery selectForInsertOrMerge(GridSqlColumn[] cols, List<GridSqlElement[]> rows,
+         GridSqlQuery subQry) {
+         if (!F.isEmpty(rows)) {
+             assert !F.isEmpty(cols);
+ 
+             GridSqlSelect sel = new GridSqlSelect();
+ 
+             GridSqlFunction from = new GridSqlFunction(GridSqlFunctionType.TABLE);
+ 
+             sel.from(from);
+ 
+             GridSqlArray[] args = new GridSqlArray[cols.length];
+ 
++            boolean noQry = true;
++
+             for (int i = 0; i < cols.length; i++) {
+                 GridSqlArray arr = new GridSqlArray(rows.size());
+ 
+                 String colName = cols[i].columnName();
+ 
+                 GridSqlAlias alias = new GridSqlAlias(colName, arr);
+ 
+                 alias.resultType(cols[i].resultType());
+ 
+                 from.addChild(alias);
+ 
+                 args[i] = arr;
+ 
+                 GridSqlColumn newCol = new GridSqlColumn(null, from, null,"TABLE", colName);
+ 
+                 newCol.resultType(cols[i].resultType());
+ 
+                 sel.addColumn(newCol, true);
+             }
+ 
+             for (GridSqlElement[] row : rows) {
+                 assert cols.length == row.length;
+ 
 -                for (int i = 0; i < row.length; i++)
++                for (int i = 0; i < row.length; i++) {
++                    GridSqlElement el = row[i];
++
++                    noQry &= (el instanceof GridSqlConst || el instanceof GridSqlParameter);
++
+                     args[i].addChild(row[i]);
++                }
+             }
+ 
++            if (noQry)
++                return null;
++
+             return sel;
+         }
+         else {
+             assert subQry != null;
+ 
+             return subQry;
+         }
+     }
+ 
+     /**
+      * Generate SQL SELECT based on DELETE's WHERE, LIMIT, etc.
+      *
+      * @param del Delete statement.
+      * @param keysParamIdx Index for .
+      * @return SELECT statement.
+      */
+     public static GridSqlSelect selectForDelete(GridSqlDelete del, @Nullable Integer keysParamIdx) {
+         GridSqlSelect mapQry = new GridSqlSelect();
+ 
+         mapQry.from(del.from());
+ 
+         Set<GridSqlTable> tbls = new HashSet<>();
+ 
+         collectAllGridTablesInTarget(del.from(), tbls);
+ 
+         assert tbls.size() == 1 : "Failed to determine target table for DELETE";
+ 
+         GridSqlTable tbl = tbls.iterator().next();
+ 
+         GridH2Table gridTbl = tbl.dataTable();
+ 
+         assert gridTbl != null : "Failed to determine target grid table for DELETE";
+ 
+         Column h2KeyCol = gridTbl.getColumn(GridH2KeyValueRowOnheap.KEY_COL);
+ 
+         Column h2ValCol = gridTbl.getColumn(GridH2KeyValueRowOnheap.VAL_COL);
+ 
+         GridSqlColumn keyCol = new GridSqlColumn(h2KeyCol, tbl, h2KeyCol.getName());
+         keyCol.resultType(GridSqlType.fromColumn(h2KeyCol));
+ 
+         GridSqlColumn valCol = new GridSqlColumn(h2ValCol, tbl, h2ValCol.getName());
+         valCol.resultType(GridSqlType.fromColumn(h2ValCol));
+ 
+         mapQry.addColumn(keyCol, true);
+         mapQry.addColumn(valCol, true);
+ 
+         GridSqlElement where = del.where();
+         if (keysParamIdx != null)
+             where = injectKeysFilterParam(where, keyCol, keysParamIdx);
+ 
+         mapQry.where(where);
+         mapQry.limit(del.limit());
+ 
+         return mapQry;
+     }
+ 
+     /**
+      * @param update UPDATE statement.
+      * @return {@code null} if given statement directly updates {@code _val} column with a literal or param value
+      * and filters by single non expression key (and, optionally,  by single non expression value).
+      */
+     public static FastUpdate getFastUpdateArgs(GridSqlUpdate update) {
+         IgnitePair<GridSqlElement> filter = findKeyValueEqualityCondition(update.where());
+ 
+         if (filter == null)
+             return null;
+ 
+         if (update.cols().size() != 1)
+             return null;
+ 
+         Table tbl = update.cols().get(0).column().getTable();
+         if (!(tbl instanceof GridH2Table))
+             return null;
+ 
+         GridH2RowDescriptor desc = ((GridH2Table)tbl).rowDescriptor();
+         if (!desc.isValueColumn(update.cols().get(0).column().getColumnId()))
+             return null;
+ 
+         GridSqlElement set = update.set().get(update.cols().get(0).columnName());
+ 
+         if (!(set instanceof GridSqlConst || set instanceof GridSqlParameter))
+             return null;
+ 
 -        return FastUpdate.create(filter.getKey(), filter.getValue(), set);
++        return new FastUpdateArguments(operandForElement(filter.getKey()), operandForElement(filter.getValue()),
++            operandForElement(set));
++    }
++
++    /**
++     * Create operand based on exact type of SQL element.
++     *
++     * @param el element.
++     * @return Operand.
++     */
++    private static FastUpdateArgument operandForElement(GridSqlElement el) {
++        assert el == null ^ (el instanceof GridSqlConst || el instanceof GridSqlParameter);
++
++        if (el == null)
++            return FastUpdateArguments.NULL_ARGUMENT;
++
++        if (el instanceof GridSqlConst)
++            return new FastUpdateArguments.ValueArgument(((GridSqlConst)el).value().getObject());
++        else
++            return new FastUpdateArguments.ParamArgument(((GridSqlParameter)el).index());
+     }
+ 
+     /**
+      * @param del DELETE statement.
+      * @return {@code true} if given statement filters by single non expression key.
+      */
+     public static FastUpdate getFastDeleteArgs(GridSqlDelete del) {
+         IgnitePair<GridSqlElement> filter = findKeyValueEqualityCondition(del.where());
+ 
+         if (filter == null)
+             return null;
+ 
+         return FastUpdate.create(filter.getKey(), filter.getValue(), null);
+     }
+ 
+     /**
+      * @param where Element to test.
+      * @return Whether given element corresponds to {@code WHERE _key = ?}, and key is a literal expressed
+      * in query or a query param.
+      */
+     @SuppressWarnings("RedundantCast")
+     private static IgnitePair<GridSqlElement> findKeyValueEqualityCondition(GridSqlElement where) {
+         if (where == null || !(where instanceof GridSqlOperation))
+             return null;
+ 
+         GridSqlOperation whereOp = (GridSqlOperation) where;
+ 
+         // Does this WHERE limit only by _key?
+         if (isKeyEqualityCondition(whereOp))
+             return new IgnitePair<>((GridSqlElement)whereOp.child(1), null);
+ 
+         // Or maybe it limits both by _key and _val?
+         if (whereOp.operationType() != GridSqlOperationType.AND)
+             return null;
+ 
+         GridSqlElement left = whereOp.child(0);
+ 
+         GridSqlElement right = whereOp.child(1);
+ 
+         if (!(left instanceof GridSqlOperation && right instanceof GridSqlOperation))
+             return null;
+ 
+         GridSqlOperation leftOp = (GridSqlOperation) left;
+ 
+         GridSqlOperation rightOp = (GridSqlOperation) right;
+ 
+         if (isKeyEqualityCondition(leftOp)) { // _key = ? and _val = ?
+             if (!isValueEqualityCondition(rightOp))
+                 return null;
+ 
+             return new IgnitePair<>((GridSqlElement)leftOp.child(1), (GridSqlElement)rightOp.child(1));
+         }
+         else if (isKeyEqualityCondition(rightOp)) { // _val = ? and _key = ?
+             if (!isValueEqualityCondition(leftOp))
+                 return null;
+ 
+             return new IgnitePair<>((GridSqlElement)rightOp.child(1), (GridSqlElement)leftOp.child(1));
+         }
+         else // Neither
+             return null;
+     }
+ 
+     /**
+      * @param op Operation.
+      * @param key true - check for key equality condition,
+      *            otherwise check for value equality condition
+      * @return Whether this condition is of form {@code colName} = ?
+      */
+     private static boolean isEqualityCondition(GridSqlOperation op, boolean key) {
+         if (op.operationType() != GridSqlOperationType.EQUAL)
+             return false;
+ 
+         GridSqlElement left = op.child(0);
+         GridSqlElement right = op.child(1);
+ 
+         if (!(left instanceof GridSqlColumn))
+             return false;
+ 
+         GridSqlColumn column = (GridSqlColumn)left;
+         if (!(column.column().getTable() instanceof GridH2Table))
+             return false;
+ 
+         GridH2RowDescriptor desc =((GridH2Table) column.column().getTable()).rowDescriptor();
+ 
+         return  (key ? desc.isKeyColumn(column.column().getColumnId()) :
+                        desc.isValueColumn(column.column().getColumnId())) &&
+                 (right instanceof GridSqlConst || right instanceof GridSqlParameter);
+     }
+ 
+     /**
+      * @param op Operation.
+      * @return Whether this condition is of form _key = ?
+      */
+     private static boolean isKeyEqualityCondition(GridSqlOperation op) {
+         return isEqualityCondition(op, true);
+     }
+ 
+     /**
+      * @param op Operation.
+      * @return Whether this condition is of form _val = ?
+      */
+     private static boolean isValueEqualityCondition(GridSqlOperation op) {
+         return isEqualityCondition(op, false);
+     }
+ 
+ 
+     /**
+      * Generate SQL SELECT based on UPDATE's WHERE, LIMIT, etc.
+      *
+      * @param update Update statement.
+      * @param keysParamIdx Index of new param for the array of keys.
+      * @return SELECT statement.
+      */
+     public static GridSqlSelect selectForUpdate(GridSqlUpdate update, @Nullable Integer keysParamIdx) {
+         GridSqlSelect mapQry = new GridSqlSelect();
+ 
+         mapQry.from(update.target());
+ 
+         Set<GridSqlTable> tbls = new HashSet<>();
+ 
+         collectAllGridTablesInTarget(update.target(), tbls);
+ 
+         assert tbls.size() == 1 : "Failed to determine target table for UPDATE";
+ 
+         GridSqlTable tbl = tbls.iterator().next();
+ 
+         GridH2Table gridTbl = tbl.dataTable();
+ 
+         assert gridTbl != null : "Failed to determine target grid table for UPDATE";
+ 
+         Column h2KeyCol = gridTbl.getColumn(GridH2KeyValueRowOnheap.KEY_COL);
+ 
+         Column h2ValCol = gridTbl.getColumn(GridH2KeyValueRowOnheap.VAL_COL);
+ 
+         GridSqlColumn keyCol = new GridSqlColumn(h2KeyCol, tbl, h2KeyCol.getName());
+         keyCol.resultType(GridSqlType.fromColumn(h2KeyCol));
+ 
+         GridSqlColumn valCol = new GridSqlColumn(h2ValCol, tbl, h2ValCol.getName());
+         valCol.resultType(GridSqlType.fromColumn(h2ValCol));
+ 
+         mapQry.addColumn(keyCol, true);
+         mapQry.addColumn(valCol, true);
+ 
+         for (GridSqlColumn c : update.cols()) {
+             String newColName = Parser.quoteIdentifier("_upd_" + c.columnName());
+             // We have to use aliases to cover cases when the user
+             // wants to update _val field directly (if it's a literal)
+             GridSqlAlias alias = new GridSqlAlias(newColName, elementOrDefault(update.set().get(c.columnName()), c), true);
+             alias.resultType(c.resultType());
+             mapQry.addColumn(alias, true);
+         }
+ 
+         GridSqlElement where = update.where();
+         if (keysParamIdx != null)
+             where = injectKeysFilterParam(where, keyCol, keysParamIdx);
+ 
+         mapQry.where(where);
+         mapQry.limit(update.limit());
+ 
+         return mapQry;
+     }
+ 
+     /**
+      * Do what we can to compute default value for this column (mimics H2 behavior).
+      * @see Table#getDefaultValue
+      * @see Column#validateConvertUpdateSequence
+      * @param el SQL element.
+      * @param col Column.
+      * @return {@link GridSqlConst#NULL}, if {@code el} is null, or {@code el} if
+      * it's not {@link GridSqlKeyword#DEFAULT}, or computed default value.
+      */
+     private static GridSqlElement elementOrDefault(GridSqlElement el, GridSqlColumn col) {
+         if (el == null)
+             return GridSqlConst.NULL;
+ 
+         if (el != GridSqlKeyword.DEFAULT)
+             return el;
+ 
+         Column h2Col = col.column();
+ 
+         Expression dfltExpr = h2Col.getDefaultExpression();
+ 
+         Value dfltVal;
+ 
+         try {
+             dfltVal = dfltExpr != null ? dfltExpr.getValue(null) : null;
+         }
+         catch (Exception ignored) {
+             throw new IgniteSQLException("Failed to evaluate default value for a column " + col.columnName());
+         }
+ 
+         if (dfltVal != null)
+             return new GridSqlConst(dfltVal);
+ 
+         int type = h2Col.getType();
+ 
+         DataType dt = DataType.getDataType(type);
+ 
+         if (dt.decimal)
+             dfltVal = ValueInt.get(0).convertTo(type);
+         else if (dt.type == Value.TIMESTAMP)
+             dfltVal = ValueTimestamp.fromMillis(U.currentTimeMillis());
+         else if (dt.type == Value.TIME)
+             dfltVal = ValueTime.fromNanos(0);
+         else if (dt.type == Value.DATE)
+             dfltVal = ValueDate.fromMillis(U.currentTimeMillis());
+         else
+             dfltVal = ValueString.get("").convertTo(type);
+ 
+         return new GridSqlConst(dfltVal);
+     }
+ 
+     /**
+      * Append additional condition to WHERE for it to select only specific keys.
+      *
+      * @param where Initial condition.
+      * @param keyCol Column to base the new condition on.
+      * @return New condition.
+      */
+     private static GridSqlElement injectKeysFilterParam(GridSqlElement where, GridSqlColumn keyCol, int paramIdx) {
+         // Yes, we need a subquery for "WHERE _key IN ?" to work with param being an array without dirty query rewriting.
+         GridSqlSelect sel = new GridSqlSelect();
+ 
+         GridSqlFunction from = new GridSqlFunction(GridSqlFunctionType.TABLE);
+ 
+         sel.from(from);
+ 
+         GridSqlColumn col = new GridSqlColumn(null, from, null, "TABLE", "_IGNITE_ERR_KEYS");
+ 
+         sel.addColumn(col, true);
+ 
+         GridSqlAlias alias = new GridSqlAlias("_IGNITE_ERR_KEYS", new GridSqlParameter(paramIdx));
+ 
+         alias.resultType(keyCol.resultType());
+ 
+         from.addChild(alias);
+ 
+         GridSqlElement e = new GridSqlOperation(GridSqlOperationType.IN, keyCol, new GridSqlSubquery(sel));
+ 
+         if (where == null)
+             return e;
+         else
+             return new GridSqlOperation(GridSqlOperationType.AND, where, e);
+     }
+ 
+     /**
+      * @param qry Select.
+      * @param params Parameters.
+      * @param target Extracted parameters.
+      * @param paramIdxs Parameter indexes.
+      * @return Extracted parameters list.
+      */
+     @SuppressWarnings("unused")
+     private static List<Object> findParams(GridSqlQuery qry, Object[] params, ArrayList<Object> target,
+         IntArray paramIdxs) {
+         if (qry instanceof GridSqlSelect)
+             return findParams((GridSqlSelect)qry, params, target, paramIdxs);
+ 
+         GridSqlUnion union = (GridSqlUnion)qry;
+ 
+         findParams(union.left(), params, target, paramIdxs);
+         findParams(union.right(), params, target, paramIdxs);
+ 
+         findParams((GridSqlElement)qry.limit(), params, target, paramIdxs);
+         findParams((GridSqlElement)qry.offset(), params, target, paramIdxs);
+ 
+         return target;
+     }
+ 
+     /**
+      * @param qry Select.
+      * @param params Parameters.
+      * @param target Extracted parameters.
+      * @param paramIdxs Parameter indexes.
+      * @return Extracted parameters list.
+      */
+     private static List<Object> findParams(GridSqlSelect qry, Object[] params, ArrayList<Object> target,
+         IntArray paramIdxs) {
+         if (params.length == 0)
+             return target;
+ 
+         for (GridSqlAst el : qry.columns(false))
+             findParams((GridSqlElement)el, params, target, paramIdxs);
+ 
+         findParams((GridSqlElement)qry.from(), params, target, paramIdxs);
+         findParams((GridSqlElement)qry.where(), params, target, paramIdxs);
+ 
+         // Don't search in GROUP BY and HAVING since they expected to be in select list.
+ 
+         findParams((GridSqlElement)qry.limit(), params, target, paramIdxs);
+         findParams((GridSqlElement)qry.offset(), params, target, paramIdxs);
+ 
+         return target;
+     }
+ 
+     /**
+      * @param el Element.
+      * @param params Parameters.
+      * @param target Extracted parameters.
+      * @param paramIdxs Parameter indexes.
+      */
+     private static void findParams(@Nullable GridSqlElement el, Object[] params, ArrayList<Object> target,
+         IntArray paramIdxs) {
+         if (el == null)
+             return;
+ 
+         if (el instanceof GridSqlParameter) {
+             // H2 Supports queries like "select ?5" but first 4 non-existing parameters are need to be set to any value.
+             // Here we will set them to NULL.
+             final int idx = ((GridSqlParameter)el).index();
+ 
+             while (target.size() < idx)
+                 target.add(null);
+ 
+             if (params.length <= idx)
+                 throw new IgniteException("Invalid number of query parameters. " +
+                     "Cannot find " + idx + " parameter.");
+ 
+             Object param = params[idx];
+ 
+             if (idx == target.size())
+                 target.add(param);
+             else
+                 target.set(idx, param);
+ 
+             paramIdxs.add(idx);
+         }
+         else if (el instanceof GridSqlSubquery)
+             findParams(((GridSqlSubquery)el).subquery(), params, target, paramIdxs);
+         else
+             for (int i = 0; i < el.size(); i++)
+                 findParams((GridSqlElement)el.child(i), params, target, paramIdxs);
+     }
+ 
+     /**
+      * Processes all the tables and subqueries using the given closure.
+      *
+      * @param from FROM element.
+      * @param c Closure each found table and subquery will be passed to. If returns {@code true} the we need to stop.
+      * @return {@code true} If we have found.
+      */
+     @SuppressWarnings("RedundantCast")
+     private static boolean findTablesInFrom(GridSqlElement from, IgnitePredicate<GridSqlElement> c) {
+         if (from == null)
+             return false;
+ 
+         if (from instanceof GridSqlTable || from instanceof GridSqlSubquery)
+             return c.apply(from);
+ 
+         if (from instanceof GridSqlJoin) {
+             // Left and right.
+             if (findTablesInFrom((GridSqlElement)from.child(0), c))
+                 return true;
+ 
+             if (findTablesInFrom((GridSqlElement)from.child(1), c))
+                 return true;
+ 
+             // We don't process ON condition because it is not a joining part of from here.
+             return false;
+         }
+         else if (from instanceof GridSqlAlias)
+             return findTablesInFrom((GridSqlElement)from.child(), c);
+         else if (from instanceof GridSqlFunction)
+             return false;
+ 
+         throw new IllegalStateException(from.getClass().getName() + " : " + from.getSQL());
+     }
+ 
+     /**
+      * @param from From element.
+      * @param tbls Tables.
+      */
+     public static void collectAllGridTablesInTarget(GridSqlElement from, final Set<GridSqlTable> tbls) {
+         findTablesInFrom(from, new IgnitePredicate<GridSqlElement>() {
+             @Override public boolean apply(GridSqlElement el) {
+                 if (el instanceof GridSqlTable)
+                     tbls.add((GridSqlTable)el);
+ 
+                 return false;
+             }
+         });
+     }
+ 
+     /**
+      * @param target Expression to extract the table from.
+      * @return Back end table for this element.
+      */
+     public static GridSqlTable gridTableForElement(GridSqlElement target) {
+         Set<GridSqlTable> tbls = new HashSet<>();
+ 
+         collectAllGridTablesInTarget(target, tbls);
+ 
+         if (tbls.size() != 1)
+             throw new IgniteSQLException("Failed to determine target table", IgniteQueryErrorCode.TABLE_NOT_FOUND);
+ 
+         return tbls.iterator().next();
+     }
+ }

http://git-wip-us.apache.org/repos/asf/ignite/blob/799098c6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
index 534a164,31dc52d..fa86836
--- 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
@@@ -17,65 -17,108 +17,119 @@@
  
  package org.apache.ignite.internal.processors.query.h2.dml;
  
 +import java.util.List;
+ import org.apache.ignite.IgniteCheckedException;
+ import org.apache.ignite.binary.BinaryObject;
+ import org.apache.ignite.binary.BinaryObjectBuilder;
+ import org.apache.ignite.internal.processors.cache.GridCacheContext;
+ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+ import org.apache.ignite.internal.processors.query.GridQueryProperty;
+ import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+ 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.GridH2RowDescriptor;
  import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
  import org.apache.ignite.internal.util.typedef.F;
+ import org.apache.ignite.internal.util.typedef.T3;
+ import org.apache.ignite.lang.IgniteBiTuple;
+ import org.h2.table.Column;
+ import org.jetbrains.annotations.Nullable;
+ 
+ import java.util.HashMap;
+ import java.util.List;
+ import java.util.Map;
+ 
+ import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT;
  
  /**
   * Update plan - where to take data to update cache from and how to construct new keys and values, if needed.
   */
  public final class UpdatePlan {
      /** Initial statement to drive the rest of the logic. */
-     public final UpdateMode mode;
+     private final UpdateMode mode;
  
      /** Target table to be affected by initial DML statement. */
-     public final GridH2Table tbl;
+     private final GridH2Table tbl;
  
      /** Column names to set or update. */
-     public final String[] colNames;
+     private final String[] colNames;
  
-     /**
-      * Expected column types to set or insert/merge.
-      * @see org.h2.value.Value
-      */
-     public final int[] colTypes;
+     /** Column types to set for insert/merge. */
+     private final int[] colTypes;
  
      /** Method to create key for INSERT or MERGE, ignored for UPDATE and DELETE. */
-     public final KeyValueSupplier keySupplier;
+     private final KeyValueSupplier keySupplier;
  
      /** Method to create value to put to cache, ignored for DELETE. */
-     public final KeyValueSupplier valSupplier;
+     private final KeyValueSupplier valSupplier;
  
-     /** Index of key column, if it's explicitly mentioned in column list of MERGE or INSERT,
-      * ignored for UPDATE and DELETE. */
-     public final int keyColIdx;
+     /** Key column index. */
+     private final int keyColIdx;
  
-     /** Index of value column, if it's explicitly mentioned in column list. Ignored for UPDATE and DELETE. */
-     public final int valColIdx;
+     /** Value column index. */
+     private final int valColIdx;
  
      /** SELECT statement built upon initial DML statement. */
-     public final String selectQry;
+     private final String selectQry;
  
      /** Subquery flag - {@code true} if {@link #selectQry} is an actual subquery that retrieves data from some cache. */
-     public final boolean isLocSubqry;
+     private final boolean isLocSubqry;
  
 +    /** */
 +    public final List<List<FastUpdateArgument>> rows;
 +
      /** Number of rows in rows based MERGE or INSERT. */
-     public final int rowsNum;
+     private final int rowsNum;
  
      /** Arguments for fast UPDATE or DELETE. */
-     public final FastUpdateArguments fastUpdateArgs;
+     private final FastUpdate fastUpdate;
  
-     /** */
+     /** Additional info for distributed update. */
+     private final DmlDistributedPlanInfo distributed;
+ 
++    /*
 +    private UpdatePlan(UpdateMode mode, GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
-         KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-         List<List<FastUpdateArgument>> rows, int rowsNum, FastUpdateArguments fastUpdateArgs) {
++                       KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
++                       List<List<FastUpdateArgument>> rows, int rowsNum, FastUpdateArguments fastUpdateArgs) {
++     */
++
+     /**
+      * Constructor.
+      *
+      * @param mode Mode.
+      * @param tbl Table.
+      * @param colNames Column names.
+      * @param colTypes Column types.
+      * @param keySupplier Key supplier.
+      * @param valSupplier Value supplier.
+      * @param keyColIdx Key column index.
+      * @param valColIdx value column index.
+      * @param selectQry Select query.
+      * @param isLocSubqry Local subquery flag.
+      * @param rowsNum Rows number.
+      * @param fastUpdate Fast update (if any).
+      * @param distributed Distributed plan (if any)
+      */
+     public UpdatePlan(
+         UpdateMode mode,
+         GridH2Table tbl,
+         String[] colNames,
+         int[] colTypes,
+         KeyValueSupplier keySupplier,
+         KeyValueSupplier valSupplier,
+         int keyColIdx,
+         int valColIdx,
+         String selectQry,
+         boolean isLocSubqry,
+         int rowsNum,
+         @Nullable FastUpdate fastUpdate,
+         @Nullable DmlDistributedPlanInfo distributed
+     ) {
          this.colNames = colNames;
          this.colTypes = colTypes;
 +        this.rows = rows;
          this.rowsNum = rowsNum;
+ 
          assert mode != null;
          assert tbl != null;
  
@@@ -87,48 -130,260 +141,298 @@@
          this.valColIdx = valColIdx;
          this.selectQry = selectQry;
          this.isLocSubqry = isLocSubqry;
-         this.fastUpdateArgs = fastUpdateArgs;
+         this.fastUpdate = fastUpdate;
+         this.distributed = distributed;
      }
  
-     /** */
+     /**
+      * Constructor for delete operation or fast update.
+      *
+      * @param mode Mode.
+      * @param tbl Table.
+      * @param selectQry Select query.
+      * @param fastUpdate Fast update arguments (if any).
+      * @param distributed Distributed plan (if any)
+      */
+     public UpdatePlan(
+         UpdateMode mode,
+         GridH2Table tbl,
+         String selectQry,
+         @Nullable FastUpdate fastUpdate,
+         @Nullable DmlDistributedPlanInfo distributed
+     ) {
+         this(
+             mode,
+             tbl,
+             null,
+             null,
+             null,
+             null,
+             -1,
+             -1,
+             selectQry,
+             false,
+             0,
+             fastUpdate,
+             distributed
+         );
+     }
+ 
+     /**
+      * Convert a row into key-value pair.
+      *
+      * @param row Row to process.
+      * @throws IgniteCheckedException if failed.
+      */
+     public IgniteBiTuple<?, ?> processRow(List<?> row) throws IgniteCheckedException {
+         GridH2RowDescriptor rowDesc = tbl.rowDescriptor();
+         GridQueryTypeDescriptor desc = rowDesc.type();
+ 
+         GridCacheContext cctx = rowDesc.context();
+ 
+         Object key = keySupplier.apply(row);
+ 
+         if (QueryUtils.isSqlType(desc.keyClass())) {
+             assert keyColIdx != -1;
+ 
+             key = DmlUtils.convert(key, rowDesc, desc.keyClass(), colTypes[keyColIdx]);
+         }
+ 
+         Object val = valSupplier.apply(row);
+ 
+         if (QueryUtils.isSqlType(desc.valueClass())) {
+             assert valColIdx != -1;
+ 
+             val = DmlUtils.convert(val, rowDesc, desc.valueClass(), colTypes[valColIdx]);
+         }
+ 
+         if (key == null) {
+             if (F.isEmpty(desc.keyFieldName()))
+                 throw new IgniteSQLException("Key for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_KEY);
+             else
+                 throw new IgniteSQLException("Null value is not allowed for column '" + desc.keyFieldName() + "'",
+                     IgniteQueryErrorCode.NULL_KEY);
+         }
+ 
+         if (val == null) {
+             if (F.isEmpty(desc.valueFieldName()))
+                 throw new IgniteSQLException("Value for INSERT, MERGE, or UPDATE must not be null",
+                     IgniteQueryErrorCode.NULL_VALUE);
+             else
+                 throw new IgniteSQLException("Null value is not allowed for column '" + desc.valueFieldName() + "'",
+                     IgniteQueryErrorCode.NULL_VALUE);
+         }
+ 
+         Map<String, Object> newColVals = new HashMap<>();
+ 
+         for (int i = 0; i < colNames.length; i++) {
+             if (i == keyColIdx || i == valColIdx)
+                 continue;
+ 
+             String colName = colNames[i];
+ 
+             GridQueryProperty prop = desc.property(colName);
+ 
+             assert prop != null;
+ 
+             Class<?> expCls = prop.type();
+ 
+             newColVals.put(colName, DmlUtils.convert(row.get(i), rowDesc, expCls, colTypes[i]));
+         }
+ 
+         // We update columns in the order specified by the table for a reason - table's
+         // column order preserves their precedence for correct update of nested properties.
+         Column[] cols = tbl.getColumns();
+ 
+         // First 3 columns are _key, _val and _ver. Skip 'em.
+         for (int i = DEFAULT_COLUMNS_COUNT; i < cols.length; i++) {
+             if (tbl.rowDescriptor().isKeyValueOrVersionColumn(i))
+                 continue;
+ 
+             String colName = cols[i].getName();
+ 
+             if (!newColVals.containsKey(colName))
+                 continue;
+ 
+             Object colVal = newColVals.get(colName);
+ 
+             desc.setValue(colName, key, val, colVal);
+         }
+ 
+         if (cctx.binaryMarshaller()) {
+             if (key instanceof BinaryObjectBuilder)
+                 key = ((BinaryObjectBuilder) key).build();
+ 
+             if (val instanceof BinaryObjectBuilder)
+                 val = ((BinaryObjectBuilder) val).build();
+         }
+ 
+         desc.validateKeyAndValue(key, val);
+ 
+         return new IgniteBiTuple<>(key, val);
+     }
+ 
+     /**
+      * Convert a row into value.
+      *
+      * @param row Row to process.
+      * @throws IgniteCheckedException if failed.
+      */
+     public T3<Object, Object, Object> processRowForUpdate(List<?> row) throws IgniteCheckedException {
+         GridH2RowDescriptor rowDesc = tbl.rowDescriptor();
+         GridQueryTypeDescriptor desc = rowDesc.type();
+ 
+         GridCacheContext cctx = rowDesc.context();
+ 
+         boolean hasNewVal = (valColIdx != -1);
+ 
+         boolean hasProps = !hasNewVal || colNames.length > 1;
+ 
+         Object key = row.get(0);
+ 
+         Object oldVal = row.get(1);
+ 
+         if (cctx.binaryMarshaller() && !(oldVal instanceof BinaryObject))
+             oldVal = cctx.grid().binary().toBinary(oldVal);
+ 
+         Object newVal;
+ 
+         Map<String, Object> newColVals = new HashMap<>();
+ 
+         for (int i = 0; i < colNames.length; i++) {
+             if (hasNewVal && i == valColIdx - 2)
+                 continue;
+ 
+             GridQueryProperty prop = tbl.rowDescriptor().type().property(colNames[i]);
+ 
+             assert prop != null : "Unknown property: " + colNames[i];
+ 
+             newColVals.put(colNames[i], DmlUtils.convert(row.get(i + 2), rowDesc, prop.type(), colTypes[i]));
+         }
+ 
+         newVal = valSupplier.apply(row);
+ 
+         if (newVal == null)
+             throw new IgniteSQLException("New value for UPDATE must not be null", IgniteQueryErrorCode.NULL_VALUE);
+ 
+         // Skip key and value - that's why we start off with 3rd column
+         for (int i = 0; i < tbl.getColumns().length - DEFAULT_COLUMNS_COUNT; i++) {
+             Column c = tbl.getColumn(i + DEFAULT_COLUMNS_COUNT);
+ 
+             if (rowDesc.isKeyValueOrVersionColumn(c.getColumnId()))
+                 continue;
+ 
+             GridQueryProperty prop = desc.property(c.getName());
+ 
+             if (prop.key())
+                 continue; // Don't get values of key's columns - we won't use them anyway
+ 
+             boolean hasNewColVal = newColVals.containsKey(c.getName());
+ 
+             if (!hasNewColVal)
+                 continue;
+ 
+             Object colVal = newColVals.get(c.getName());
+ 
+             // UPDATE currently does not allow to modify key or its fields, so we must be safe to pass null as key.
+             rowDesc.setColumnValue(null, newVal, colVal, i);
+         }
+ 
+         if (cctx.binaryMarshaller() && hasProps) {
+             assert newVal instanceof BinaryObjectBuilder;
+ 
+             newVal = ((BinaryObjectBuilder) newVal).build();
+         }
+ 
+         desc.validateKeyAndValue(key, newVal);
+ 
+         return new T3<>(key, oldVal, newVal);
+     }
+ 
+     /**
+      * @return Update mode.
+      */
+     public UpdateMode mode() {
+         return mode;
+     }
+ 
+     /**
+      * @return Cache context.
+      */
+     public GridCacheContext cacheContext() {
+         return tbl.cache();
+     }
+ 
+     /**
+      * @return Distributed plan info (for skip-reducer mode).
+      */
+     @Nullable public DmlDistributedPlanInfo distributedPlan() {
+         return distributed;
+     }
+ 
+     /**
+      * @return Row count.
+      */
+     public int rowCount() {
+         return rowsNum;
+     }
+ 
+     /**
+      * @return Select query.
+      */
+     public String selectQuery() {
+         return selectQry;
+     }
+ 
+     /**
+      * @return Local subquery flag.
+      */
+     @Nullable public boolean isLocalSubquery() {
+         return isLocSubqry;
+     }
+ 
+     /**
+      * @return Fast update.
+      */
+     @Nullable public FastUpdate fastUpdate() {
+         return fastUpdate;
+     }
++
++    /*
 +    public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
-         KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-         List<List<FastUpdateArgument>> rows, int rowsNum) {
++                                      KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
++                                      List<List<FastUpdateArgument>> rows, int rowsNum) {
 +        assert !F.isEmpty(colNames);
 +
 +        return new UpdatePlan(UpdateMode.MERGE, tbl, colNames, colTypes, keySupplier, valSupplier, keyColIdx, valColIdx,
 +            selectQry, isLocSubqry, rows, rowsNum, null);
 +    }
 +
-     /** */
 +    public static UpdatePlan forInsert(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
-         KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-         List<List<FastUpdateArgument>> rows, int rowsNum) {
++                                       KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
++                                       List<List<FastUpdateArgument>> rows, int rowsNum) {
 +        assert !F.isEmpty(colNames);
 +
 +        return new UpdatePlan(UpdateMode.INSERT, tbl, colNames, colTypes, keySupplier, valSupplier, keyColIdx, valColIdx,
 +            selectQry, isLocSubqry, rows, rowsNum, null);
 +    }
 +
-     /** */
 +    public static UpdatePlan forUpdate(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier valSupplier,
-         int valColIdx, String selectQry) {
++                                       int valColIdx, String selectQry) {
 +        assert !F.isEmpty(colNames);
 +
 +        return new UpdatePlan(UpdateMode.UPDATE, tbl, colNames, colTypes, null, valSupplier, -1, valColIdx, selectQry,
 +            false, null, 0, null);
 +    }
 +
-     /** */
 +    public static UpdatePlan forDelete(GridH2Table tbl, String selectQry) {
 +        return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, null, -1, -1, selectQry, false, null, 0, null);
 +    }
 +
-     /** */
 +    public static UpdatePlan forFastUpdate(UpdateMode mode, GridH2Table tbl, FastUpdateArguments fastUpdateArgs) {
 +        assert mode == UpdateMode.UPDATE || mode == UpdateMode.DELETE;
 +
 +        return new UpdatePlan(mode, tbl, null, null, null, null, -1, -1, null, false, null, 0, fastUpdateArgs);
 +    }
- 
++     */
  }

http://git-wip-us.apache.org/repos/asf/ignite/blob/799098c6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index cf6cd88,a551639..52efd6d
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@@ -18,7 -18,9 +18,10 @@@
  package org.apache.ignite.internal.processors.query.h2.dml;
  
  import java.lang.reflect.Constructor;
+ import java.sql.Connection;
+ import java.sql.PreparedStatement;
+ import java.sql.SQLException;
 +import java.util.ArrayList;
  import java.util.HashSet;
  import java.util.List;
  import java.util.Set;
@@@ -33,18 -37,17 +38,19 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.query.IgniteSQLException;
  import org.apache.ignite.internal.processors.query.QueryUtils;
  import org.apache.ignite.internal.processors.query.h2.DmlStatementsProcessor;
+ import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
  import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
  import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
- import org.apache.ignite.internal.processors.query.h2.sql.DmlAstUtils;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlColumn;
 +import org.apache.ignite.internal.processors.query.h2.sql.GridSqlConst;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDelete;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlInsert;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlMerge;
 +import org.apache.ignite.internal.processors.query.h2.sql.GridSqlParameter;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuery;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlSelect;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlTable;
@@@ -122,11 -135,7 +142,11 @@@ public final class UpdatePlanBuilder 
              desc = tbl.dataTable().rowDescriptor();
  
              cols = ins.columns();
-             sel = DmlAstUtils.selectForInsertOrMerge(cols, ins.rows(), ins.query(), desc);
+             sel = DmlAstUtils.selectForInsertOrMerge(cols, ins.rows(), ins.query());
 +
 +            if (sel == null)
 +                elRows = ins.rows();
 +
              isTwoStepSubqry = (ins.query() != null);
              rowsNum = isTwoStepSubqry ? 0 : ins.rows().size();
          }
@@@ -135,53 -144,21 +155,47 @@@
  
              target = merge.into();
  
-             tbl = gridTableForElement(target);
+             tbl = DmlAstUtils.gridTableForElement(target);
              desc = tbl.dataTable().rowDescriptor();
  
-             // This check also protects us from attempts to update key or its fields directly -
-             // when no key except cache key can be used, it will serve only for uniqueness checks,
-             // not for updates, and hence will allow putting new pairs only.
-             // We don't quote _key and _val column names on CREATE TABLE, so they are always uppercase here.
-             GridSqlColumn[] keys = merge.keys();
-             if (keys.length != 1 || !desc.isKeyColumn(tbl.dataTable().getColumn(keys[0].columnName()).getColumnId()))
-                 throw new CacheException("SQL MERGE does not support arbitrary keys");
- 
              cols = merge.columns();
-             sel = DmlAstUtils.selectForInsertOrMerge(cols, merge.rows(), merge.query(), desc);
+             sel = DmlAstUtils.selectForInsertOrMerge(cols, merge.rows(), merge.query());
 +
 +            if (sel == null)
 +                elRows = merge.rows();
 +
              isTwoStepSubqry = (merge.query() != null);
              rowsNum = isTwoStepSubqry ? 0 : merge.rows().size();
          }
-         else throw new IgniteSQLException("Unexpected DML operation [cls=" + stmt.getClass().getName() + ']',
+         else {
+             throw new IgniteSQLException("Unexpected DML operation [cls=" + stmt.getClass().getName() + ']',
                  IgniteQueryErrorCode.UNEXPECTED_OPERATION);
+         }
  
 +        if (elRows != null) {
 +            assert sel == null;
 +
 +            rows = new ArrayList<>(elRows.size());
 +
 +            for (GridSqlElement[] elRow : elRows) {
 +                List<FastUpdateArgument> row = new ArrayList<>(cols.length);
 +
 +                for (GridSqlElement e : elRow) {
 +                    if (e instanceof GridSqlConst)
 +                        row.add(new FastUpdateArguments.ValueArgument(((GridSqlConst) e).value().getObject()));
 +                    else if (e instanceof GridSqlParameter)
 +                        row.add(new FastUpdateArguments.ParamArgument(((GridSqlParameter) e).index()));
 +                    else
 +                        throw new IgniteSQLException("Unexpected element type: " + e.getClass().getSimpleName(),
 +                            IgniteQueryErrorCode.UNEXPECTED_ELEMENT_TYPE);
 +                }
 +
 +                rows.add(row);
 +            }
 +        }
 +
          // Let's set the flag only for subqueries that have their FROM specified.
 -        isTwoStepSubqry = (isTwoStepSubqry && (sel instanceof GridSqlUnion ||
 +        isTwoStepSubqry &= (sel != null && (sel instanceof GridSqlUnion ||
              (sel instanceof GridSqlSelect && ((GridSqlSelect) sel).from() != null)));
  
          int keyColIdx = -1;
@@@ -233,12 -210,28 +247,33 @@@
          KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, true, false);
          KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false, false);
  
-         if (stmt instanceof GridSqlMerge)
-             return UpdatePlan.forMerge(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
-                 valColIdx, sel != null ? sel.getSQL() : null, !isTwoStepSubqry, rows, rowsNum);
-         else
-             return UpdatePlan.forInsert(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
-                 valColIdx, sel != null ? sel.getSQL() : null, !isTwoStepSubqry, rows, rowsNum);
+         String selectSql = sel.getSQL();
+ 
+         DmlDistributedPlanInfo distributed = (rowsNum == 0 && !F.isEmpty(selectSql)) ?
+             checkPlanCanBeDistributed(idx, conn, fieldsQuery, loc, selectSql, tbl.dataTable().cacheName()) : null;
+ 
+         UpdateMode mode = stmt instanceof GridSqlMerge ? UpdateMode.MERGE : UpdateMode.INSERT;
+ 
+         return new UpdatePlan(
+             mode,
+             tbl.dataTable(),
+             colNames,
+             colTypes,
+             keySupplier,
+             valSupplier,
+             keyColIdx,
+             valColIdx,
+             selectSql,
+             !isTwoStepSubqry,
+             rowsNum,
+             null,
+             distributed
+         );
++
++        /*
++        if (sel == null)
++                elRows = merge.rows();
++         */
      }
  
      /**


[17/31] ignite git commit: Merge branch 'master' into ignite-4490

Posted by vo...@apache.org.
Merge branch 'master' into ignite-4490


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

Branch: refs/heads/ignite-6022-proto
Commit: 58ffec9f71349f4bab27ed81aad43091483d74e2
Parents: b810acb 91be7af
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 11:10:14 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 11:10:14 2017 +0300

----------------------------------------------------------------------
 .../ignite/spi/communication/tcp/TcpCommunicationSpi.java      | 2 +-
 .../java/org/apache/ignite/ml/knn/models/KNNModelFormat.java   | 6 +++++-
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[20/31] ignite git commit: Minors.

Posted by vo...@apache.org.
Minors.


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

Branch: refs/heads/ignite-6022-proto
Commit: 850cd7e2ba077132a8127f373b4c97e51d092685
Parents: 7a9cd6a
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 11:53:00 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 11:53:00 2017 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        | 17 +++------
 .../processors/query/h2/dml/UpdatePlan.java     | 22 +++++++----
 .../query/h2/dml/UpdatePlanBuilder.java         | 40 ++++++++++----------
 3 files changed, 41 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/850cd7e2/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 4ca034e..7f1f5d5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -52,7 +52,6 @@ import org.apache.ignite.internal.processors.query.h2.dml.DmlArgument;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlBatchSender;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlDistributedPlanInfo;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils;
-import org.apache.ignite.internal.processors.query.h2.dml.FastUpdate;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
@@ -169,7 +168,7 @@ public class DmlStatementsProcessor {
             UpdateResult r;
 
             try {
-                r = executeUpdateStatement(schemaName, cctx, conn, prepared, fieldsQry, loc, filters, cancel, errKeys);
+                r = executeUpdateStatement(schemaName, cctx, conn, prepared, fieldsQry, loc, filters, cancel);
             }
             finally {
                 cctx.operationContextPerCall(opCtx);
@@ -337,27 +336,23 @@ public class DmlStatementsProcessor {
      * @param loc Local query flag.
      * @param filters Cache name and key filter.
      * @param cancel Query cancel state holder.
-     * @param failedKeys Keys to restrict UPDATE and DELETE operations with. Null or empty array means no restriction.
      * @return Pair [number of successfully processed items; keys that have failed to be processed]
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"ConstantConditions", "unchecked"})
     private UpdateResult executeUpdateStatement(String schemaName, final GridCacheContext cctx, Connection c,
-        Prepared prepared, SqlFieldsQuery fieldsQry, boolean loc, IndexingQueryFilter filters,
-        GridQueryCancel cancel, Object[] failedKeys) throws IgniteCheckedException {
+        Prepared prepared, SqlFieldsQuery fieldsQry, boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel)
+        throws IgniteCheckedException {
         int mainCacheId = cctx.cacheId();
 
         Integer errKeysPos = null;
 
         UpdatePlan plan = getPlanForStatement(schemaName, c, prepared, fieldsQry, loc, errKeysPos);
 
-        FastUpdate fastUpdate = plan.fastUpdate();
-
-        if (fastUpdate != null) {
-            assert F.isEmpty(failedKeys) && errKeysPos == null;
+        UpdateResult fastUpdateRes = plan.processFast(fieldsQry.getArgs());
 
-            return fastUpdate.execute(plan.cacheContext().cache(), fieldsQry.getArgs());
-        }
+        if (fastUpdateRes != null)
+            return fastUpdateRes;
 
         if (plan.distributedPlan() != null) {
             UpdateResult result = doDistributedUpdate(schemaName, fieldsQry, plan, cancel);

http://git-wip-us.apache.org/repos/asf/ignite/blob/850cd7e2/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 f33331a..10e7519 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
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.h2.UpdateResult;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.util.typedef.F;
@@ -345,6 +346,20 @@ public final class UpdatePlan {
     }
 
     /**
+     * Process fast DML operation if possible.
+     *
+     * @param args QUery arguments.
+     * @return Update result or {@code null} if fast update is not applicable for plan.
+     * @throws IgniteCheckedException If failed.
+     */
+    public UpdateResult processFast(Object[] args) throws IgniteCheckedException {
+        if (fastUpdate != null)
+            return fastUpdate.execute(cacheContext().cache(), args);
+
+        return null;
+    }
+
+    /**
      * @return Update mode.
      */
     public UpdateMode mode() {
@@ -387,13 +402,6 @@ public final class UpdatePlan {
     }
 
     /**
-     * @return Fast update.
-     */
-    @Nullable public FastUpdate fastUpdate() {
-        return fastUpdate;
-    }
-
-    /**
      * @return Names of affected columns.
      */
     public String[] columnNames() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/850cd7e2/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index bbbff17..563456b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -130,8 +130,6 @@ public final class UpdatePlanBuilder {
 
         List<GridSqlElement[]> elRows = null;
 
-        List<List<DmlArgument>> rows = null;
-
         if (stmt instanceof GridSqlInsert) {
             GridSqlInsert ins = (GridSqlInsert) stmt;
             target = ins.into();
@@ -170,24 +168,6 @@ public final class UpdatePlanBuilder {
                 IgniteQueryErrorCode.UNEXPECTED_OPERATION);
         }
 
-        if (elRows != null) {
-            assert sel == null;
-
-            rows = new ArrayList<>(elRows.size());
-
-            for (GridSqlElement[] elRow : elRows) {
-                List<DmlArgument> row = new ArrayList<>(cols.length);
-
-                for (GridSqlElement e : elRow) {
-                    DmlArgument arg = DmlArguments.create(e);
-
-                    row.add(arg);
-                }
-
-                rows.add(row);
-            }
-        }
-
         // Let's set the flag only for subqueries that have their FROM specified.
         isTwoStepSubqry &= (sel != null && (sel instanceof GridSqlUnion ||
             (sel instanceof GridSqlSelect && ((GridSqlSelect) sel).from() != null)));
@@ -248,6 +228,26 @@ public final class UpdatePlanBuilder {
 
         UpdateMode mode = stmt instanceof GridSqlMerge ? UpdateMode.MERGE : UpdateMode.INSERT;
 
+        List<List<DmlArgument>> rows = null;
+
+        if (elRows != null) {
+            assert sel == null;
+
+            rows = new ArrayList<>(elRows.size());
+
+            for (GridSqlElement[] elRow : elRows) {
+                List<DmlArgument> row = new ArrayList<>(cols.length);
+
+                for (GridSqlElement el : elRow) {
+                    DmlArgument arg = DmlArguments.create(el);
+
+                    row.add(arg);
+                }
+
+                rows.add(row);
+            }
+        }
+
         return new UpdatePlan(
             mode,
             tbl.dataTable(),


[31/31] ignite git commit: Merge.

Posted by vo...@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/771422cb
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/771422cb
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/771422cb

Branch: refs/heads/ignite-6022-proto
Commit: 771422cb1e1f2503c93fd0317c65f5ef315a2a52
Parents: eeb116b
Author: devozerov <vo...@gridgain.com>
Authored: Tue Dec 19 15:15:57 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Dec 19 15:15:57 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/query/h2/DmlStatementsProcessor.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/771422cb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 60c67d8..a52881d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -235,7 +235,7 @@ public class DmlStatementsProcessor {
                 List<List<?>> cur = plan.createRows(argss);
 
                 UpdateResult res =
-                    processDmlSelectResultBatched(cctx, plan, cur, fieldsQry.getPageSize(), fieldsQry.isStreaming());
+                    processDmlSelectResultBatched(plan, cur, fieldsQry.getPageSize(), fieldsQry.isStreaming());
 
                 Collection<UpdateResult> ress = new ArrayList<>(1);
 
@@ -504,8 +504,8 @@ public class DmlStatementsProcessor {
         return processDmlSelectResult(cctx, plan, cur, pageSize);
     }
 
-    private UpdateResult processDmlSelectResultBatched(GridCacheContext cctx, UpdatePlan plan, Collection<List<?>> rows,
-        int pageSize, boolean streaming) throws IgniteCheckedException {
+    private UpdateResult processDmlSelectResultBatched(UpdatePlan plan, Collection<List<?>> rows, int pageSize,
+        boolean streaming) throws IgniteCheckedException {
         switch (plan.mode()) {
             case MERGE:
                 // TODO


[16/31] ignite git commit: Cleanup

Posted by vo...@apache.org.
Cleanup


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

Branch: refs/heads/ignite-6022-proto
Commit: b810acbc911753bd2791373faa86d69186fb0c3e
Parents: 478fb43
Author: Alexander Paschenko <al...@gmail.com>
Authored: Sun Dec 17 21:38:43 2017 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Sun Dec 17 21:38:43 2017 +0300

----------------------------------------------------------------------
 .../processors/query/h2/dml/DmlAstUtils.java    | 18 --------
 .../processors/query/h2/dml/UpdatePlan.java     | 44 --------------------
 2 files changed, 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b810acbc/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
index b6c4a2a..161ff4a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
@@ -222,24 +222,6 @@ public final class DmlAstUtils {
     }
 
     /**
-     * Create operand based on exact type of SQL element.
-     *
-     * @param el element.
-     * @return Operand.
-     */
-    private static FastUpdateArgument operandForElement(GridSqlElement el) {
-        assert el == null ^ (el instanceof GridSqlConst || el instanceof GridSqlParameter);
-
-        if (el == null)
-            return FastUpdateArguments.NULL_ARGUMENT;
-
-        if (el instanceof GridSqlConst)
-            return new FastUpdateArguments.ValueArgument(((GridSqlConst)el).value().getObject());
-        else
-            return new FastUpdateArguments.ParamArgument(((GridSqlParameter)el).index());
-    }
-
-    /**
      * @param del DELETE statement.
      * @return {@code true} if given statement filters by single non expression key.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b810acbc/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 96298d8..b1b5ada 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
@@ -85,12 +85,6 @@ public final class UpdatePlan {
     /** Additional info for distributed update. */
     private final DmlDistributedPlanInfo distributed;
 
-    /*
-    private UpdatePlan(UpdateMode mode, GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
-                       KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-                       List<List<FastUpdateArgument>> rows, int rowsNum, FastUpdateArguments fastUpdateArgs) {
-     */
-
     /**
      * Constructor.
      *
@@ -440,42 +434,4 @@ public final class UpdatePlan {
     public GridH2Table table() {
         return tbl;
     }
-
-    /*
-    public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
-                                      KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-                                      List<List<FastUpdateArgument>> rows, int rowsNum) {
-        assert !F.isEmpty(colNames);
-
-        return new UpdatePlan(UpdateMode.MERGE, tbl, colNames, colTypes, keySupplier, valSupplier, keyColIdx, valColIdx,
-            selectQry, isLocSubqry, rows, rowsNum, null);
-    }
-
-    public static UpdatePlan forInsert(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
-                                       KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-                                       List<List<FastUpdateArgument>> rows, int rowsNum) {
-        assert !F.isEmpty(colNames);
-
-        return new UpdatePlan(UpdateMode.INSERT, tbl, colNames, colTypes, keySupplier, valSupplier, keyColIdx, valColIdx,
-            selectQry, isLocSubqry, rows, rowsNum, null);
-    }
-
-    public static UpdatePlan forUpdate(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier valSupplier,
-                                       int valColIdx, String selectQry) {
-        assert !F.isEmpty(colNames);
-
-        return new UpdatePlan(UpdateMode.UPDATE, tbl, colNames, colTypes, null, valSupplier, -1, valColIdx, selectQry,
-            false, null, 0, null);
-    }
-
-    public static UpdatePlan forDelete(GridH2Table tbl, String selectQry) {
-        return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, null, -1, -1, selectQry, false, null, 0, null);
-    }
-
-    public static UpdatePlan forFastUpdate(UpdateMode mode, GridH2Table tbl, FastUpdateArguments fastUpdateArgs) {
-        assert mode == UpdateMode.UPDATE || mode == UpdateMode.DELETE;
-
-        return new UpdatePlan(mode, tbl, null, null, null, null, -1, -1, null, false, null, 0, fastUpdateArgs);
-    }
-     */
 }


[26/31] ignite git commit: WIP on benchmark.

Posted by vo...@apache.org.
WIP on benchmark.


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

Branch: refs/heads/ignite-6022-proto
Commit: d23bdd9d8672273214ba407cdc5e67378984182f
Parents: 5f888e3
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 14:37:36 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 14:37:36 2017 +0300

----------------------------------------------------------------------
 .../query/h2/opt/JdbcBenchmarkRunner.java       | 104 ++++++++++---------
 1 file changed, 55 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d23bdd9d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
index a09cee4..18dc575 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/JdbcBenchmarkRunner.java
@@ -20,7 +20,7 @@ public class JdbcBenchmarkRunner {
 
     private static final int THREAD_CNT = 1;
 
-    private static final int BATCH_SIZE = 10000;
+    private static final int BATCH_SIZE = 1000;
 
     private static final LongAdder OPS = new LongAdder();
 
@@ -31,6 +31,8 @@ public class JdbcBenchmarkRunner {
 
         IgniteConfiguration cfg = new IgniteConfiguration().setLocalHost("127.0.0.1");
 
+        cfg.setClientConnectorConfiguration(null);
+
 //        DataStorageConfiguration dsCfg = new DataStorageConfiguration().setWalMode(WALMode.LOG_ONLY);
 //
 //        dsCfg.getDefaultDataRegionConfiguration().setPersistenceEnabled(true);
@@ -43,73 +45,77 @@ public class JdbcBenchmarkRunner {
         try (Ignite node = Ignition.start(cfg)) {
             node.active(true);
 
-            try (Connection conn = connect()) {
-                execute(conn, "CREATE TABLE tbl (id BIGINT PRIMARY KEY, v1 BIGINT, v2 BIGINT, v3 BIGINT, v4 BIGINT)");
-            }
+            IgniteConfiguration cliCfg = new IgniteConfiguration().setLocalHost("127.0.0.1").setIgniteInstanceName("cli").setClientMode(true);
 
-            new Thread(new Runnable() {
-                @Override public void run() {
-                    while (!done) {
-                        long startTime = System.currentTimeMillis();
-                        long startOps = OPS.longValue();
+            try (Ignite cli = Ignition.start(cliCfg)) {
+                try (Connection conn = connect()) {
+                    execute(conn, "CREATE TABLE tbl (id BIGINT PRIMARY KEY, v1 BIGINT, v2 BIGINT, v3 BIGINT, v4 BIGINT)");
+                }
 
-                        try {
-                            Thread.sleep(3000L);
-                        }
-                        catch (InterruptedException e) {
-                            break;
-                        }
+                new Thread(new Runnable() {
+                    @Override public void run() {
+                        while (!done) {
+                            long startTime = System.currentTimeMillis();
+                            long startOps = OPS.longValue();
+
+                            try {
+                                Thread.sleep(3000L);
+                            }
+                            catch (InterruptedException e) {
+                                break;
+                            }
 
-                        long endTime = System.currentTimeMillis();
-                        long endOps = OPS.longValue();
+                            long endTime = System.currentTimeMillis();
+                            long endOps = OPS.longValue();
 
-                        double t = 1000 * (double)(endOps - startOps) / (double)(endTime - startTime);
+                            double t = 1000 * (double)(endOps - startOps) / (double)(endTime - startTime);
 
-                        if (!done)
-                            System.out.println("Throughput: " + String.format("%1$,.2f", t) + " ops/sec");
+                            if (!done)
+                                System.out.println("Throughput: " + String.format("%1$,.2f", t) + " ops/sec");
+                        }
                     }
-                }
-            }).start();
+                }).start();
 
-            JdbcRequestHandler.STREAMER = true;
+                JdbcRequestHandler.STREAMER = true;
 
-            long start = System.currentTimeMillis();
+                long start = System.currentTimeMillis();
 
-            CyclicBarrier startBarrier = new CyclicBarrier(THREAD_CNT);
-            CountDownLatch stopLatch = new CountDownLatch(THREAD_CNT);
+                CyclicBarrier startBarrier = new CyclicBarrier(THREAD_CNT);
+                CountDownLatch stopLatch = new CountDownLatch(THREAD_CNT);
 
-            for (int i = 0; i < THREAD_CNT; i++) {
-                final int i0 = i;
+                for (int i = 0; i < THREAD_CNT; i++) {
+                    final int i0 = i;
 
-                new Thread(new Runnable() {
-                    @SuppressWarnings("InfiniteLoopStatement")
-                    @Override public void run() {
-                        try (Connection conn = connect()) {
-                            startBarrier.await();
+                    new Thread(new Runnable() {
+                        @SuppressWarnings("InfiniteLoopStatement")
+                        @Override public void run() {
+                            try (Connection conn = connect()) {
+                                startBarrier.await();
 
-                            doUpdate(conn, i0);
+                                doUpdate(conn, i0);
 
-                            execute(conn, "FLUSH");
-                        }
-                        catch (Exception e) {
-                            System.out.println("ERROR: " + e);
-                        }
-                        finally {
-                            stopLatch.countDown();
+                                execute(conn, "FLUSH");
+                            }
+                            catch (Exception e) {
+                                System.out.println("ERROR: " + e);
+                            }
+                            finally {
+                                stopLatch.countDown();
+                            }
                         }
-                    }
-                }).start();
-            }
+                    }).start();
+                }
 
-            stopLatch.await();
+                stopLatch.await();
 
-            done = true;
+                done = true;
 
-            long end = System.currentTimeMillis();
+                long end = System.currentTimeMillis();
 
-            float dur = (float)((double)(end - start) / 1000);
+                float dur = (float)((double)(end - start) / 1000);
 
-            System.out.println("TOTAL DURATION: " + dur);
+                System.out.println("TOTAL DURATION: " + dur);
+            }
         }
     }
 


[24/31] ignite git commit: Streaming.

Posted by vo...@apache.org.
Streaming.


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

Branch: refs/heads/ignite-6022-proto
Commit: d57d406f13db2457dd0e315dcf3d992a6af1f9b0
Parents: 03f9fe7
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 14:34:14 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 14:34:14 2017 +0300

----------------------------------------------------------------------
 .../cache/query/SqlFieldsQueryEx.java           | 13 +++
 .../odbc/jdbc/JdbcRequestHandler.java           |  5 ++
 .../query/h2/DmlStatementsProcessor.java        | 85 ++++++++++++++------
 .../processors/query/h2/IgniteH2Indexing.java   | 15 ++++
 4 files changed, 93 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d57d406f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java
index fb098a7..2d46d28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java
@@ -38,6 +38,8 @@ public final class SqlFieldsQueryEx extends SqlFieldsQuery {
     /** Batched arguments. */
     private List<Object[]> batchedArgs;
 
+    private boolean streaming;
+
     public SqlFieldsQueryEx(String sql, Boolean isQry) {
         super(sql);
 
@@ -63,6 +65,7 @@ public final class SqlFieldsQueryEx extends SqlFieldsQuery {
         this.isQry = qry.isQry;
         this.skipReducerOnUpdate = qry.skipReducerOnUpdate;
         this.batchedArgs = qry.batchedArgs;
+        this.streaming = qry.streaming;
     }
 
     /**
@@ -164,6 +167,16 @@ public final class SqlFieldsQueryEx extends SqlFieldsQuery {
         return skipReducerOnUpdate;
     }
 
+    public boolean isStreaming() {
+        return streaming;
+    }
+
+    public SqlFieldsQuery setStreaming(boolean streaming) {
+        this.streaming = streaming;
+
+        return this;
+    }
+
     /**
      * Add batched arguments.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/d57d406f/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index 2530360..c28c831 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -465,6 +465,8 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
         }
     }
 
+    public static volatile boolean STREAMER = false;
+
     /**
      * @param req Request.
      * @return Response.
@@ -506,6 +508,9 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
                     qry.setLazy(lazy);
 
                     qry.setSchema(schemaName);
+
+                    if (STREAMER)
+                        qry.setStreaming(true);
                 }
 
                 assert qry != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d57d406f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index df14c85..60c67d8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -44,6 +44,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx;
+import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
 import org.apache.ignite.internal.processors.odbc.SqlStateCode;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
@@ -233,7 +234,8 @@ public class DmlStatementsProcessor {
             try {
                 List<List<?>> cur = plan.createRows(argss);
 
-                UpdateResult res = processDmlSelectResultBatched(cctx, plan, cur, fieldsQry.getPageSize());
+                UpdateResult res =
+                    processDmlSelectResultBatched(cctx, plan, cur, fieldsQry.getPageSize(), fieldsQry.isStreaming());
 
                 Collection<UpdateResult> ress = new ArrayList<>(1);
 
@@ -503,14 +505,14 @@ public class DmlStatementsProcessor {
     }
 
     private UpdateResult processDmlSelectResultBatched(GridCacheContext cctx, UpdatePlan plan, Collection<List<?>> rows,
-        int pageSize) throws IgniteCheckedException {
+        int pageSize, boolean streaming) throws IgniteCheckedException {
         switch (plan.mode()) {
             case MERGE:
                 // TODO
                 throw new IgniteCheckedException("Unsupported, fix");
 
             case INSERT:
-                return new UpdateResult(doInsertBatched(plan, rows, pageSize), X.EMPTY_OBJECT_ARRAY);
+                return new UpdateResult(doInsertBatched(plan, rows, pageSize, streaming), X.EMPTY_OBJECT_ARRAY);
 
             default:
                 throw new IgniteSQLException("Unexpected DML operation [mode=" + plan.mode() + ']',
@@ -791,48 +793,81 @@ public class DmlStatementsProcessor {
         }
     }
 
+    public static volatile DataStreamerImpl streamer;
+
+    public void flushStreamer() {
+        DataStreamerImpl streamer0 = streamer;
+
+        if (streamer0 != null)
+            streamer0.flush();
+    }
+
     /**
      * Execute INSERT statement plan.
-     * @param cursor Cursor to take inserted data from.
+     * @param rows Cursor to take inserted data from.
      * @param pageSize Batch size for streaming, anything <= 0 for single page operations.
      * @return Number of items affected.
      * @throws IgniteCheckedException if failed, particularly in case of duplicate keys.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    private long doInsertBatched(UpdatePlan plan, Collection<List<?>> cursor, int pageSize)
+    private long doInsertBatched(UpdatePlan plan, Collection<List<?>> rows, int pageSize, boolean streaming)
         throws IgniteCheckedException {
         GridCacheContext cctx = plan.cacheContext();
 
-        // Keys that failed to INSERT due to duplication.
-        DmlBatchSender sender = new DmlBatchSender(cctx, pageSize);
+        if (streaming) {
+            DataStreamerImpl streamer0 = streamer;
 
-        for (List<?> row : cursor) {
-            final IgniteBiTuple keyValPair = plan.processRow(row);
+            if (streamer0 == null) {
+                streamer0 = cctx.kernalContext().dataStream().dataStreamer(cctx.name());
 
-            sender.add(keyValPair.getKey(), new InsertEntryProcessor(keyValPair.getValue()));
-        }
+                streamer = streamer0;
+            }
 
-        // TODO: Tale page size in count?
-        sender.flush();
+            List<IgniteBiTuple> keyValPairs = new ArrayList<>(rows.size());
 
-        SQLException resEx = sender.error();
+            for (List<?> row : rows) {
+                IgniteBiTuple keyValPair = plan.processRow(row);
 
-        if (!F.isEmpty(sender.failedKeys())) {
-            String msg = "Failed to INSERT some keys because they are already in cache " +
-                "[keys=" + sender.failedKeys() + ']';
+                keyValPairs.add(keyValPair);
+            }
 
-            SQLException dupEx = new SQLException(msg, SqlStateCode.CONSTRAINT_VIOLATION);
+            streamer.addData(keyValPairs);
 
-            if (resEx == null)
-                resEx = dupEx;
-            else
-                resEx.setNextException(dupEx);
+            return rows.size();
         }
+        else {
 
-        if (resEx != null)
-            throw new IgniteSQLException(resEx);
+            // Keys that failed to INSERT due to duplication.
+            DmlBatchSender sender = new DmlBatchSender(cctx, pageSize);
+
+            for (List<?> row : rows) {
+                final IgniteBiTuple keyValPair = plan.processRow(row);
+
+                sender.add(keyValPair.getKey(), new InsertEntryProcessor(keyValPair.getValue()));
+            }
+
+            // TODO: Tale page size in count?
+            sender.flush();
+
+            SQLException resEx = sender.error();
+
+            if (!F.isEmpty(sender.failedKeys())) {
+                String msg = "Failed to INSERT some keys because they are already in cache " +
+                    "[keys=" + sender.failedKeys() + ']';
+
+                SQLException dupEx = new SQLException(msg, SqlStateCode.CONSTRAINT_VIOLATION);
 
-        return sender.updateCount();
+                if (resEx == null)
+                    resEx = dupEx;
+                else
+                    resEx.setNextException(dupEx);
+            }
+
+            if (resEx != null)
+                throw new IgniteSQLException(resEx);
+
+            return sender.updateCount();
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d57d406f/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 31a6645..f5124a4 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
@@ -1405,6 +1405,21 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> queryDistributedSqlFields(String schemaName, SqlFieldsQuery qry,
         boolean keepBinary, GridQueryCancel cancel, @Nullable Integer mainCacheId, boolean failOnMultipleStmts) {
+        if ("FLUSH".equalsIgnoreCase(qry.getSql())) {
+            dmlProc.flushStreamer();
+
+            QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
+                (Collections.singletonList(0L)), null, false);
+
+            resCur.fieldsMeta(UPDATE_RESULT_META);
+
+            List<FieldsQueryCursor<List<?>>> res = new ArrayList<>(1);
+
+            res.add(resCur);
+
+            return res;
+        }
+
         List<FieldsQueryCursor<List<?>>> res = tryQueryDistributedSqlFieldsNative(schemaName, qry);
 
         if (res != null)


[08/31] ignite git commit: Merge branch 'master' into ignite-4490

Posted by vo...@apache.org.
Merge branch 'master' into ignite-4490


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

Branch: refs/heads/ignite-6022-proto
Commit: c4ee47d2cf98382611e6723cf46680f581e4b7e6
Parents: e66b664 6983e87
Author: devozerov <vo...@gridgain.com>
Authored: Fri Sep 8 13:51:09 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Sep 8 13:51:09 2017 +0300

----------------------------------------------------------------------
 .../example-persistent-store.xml                |   23 -
 examples/pom-standalone-lgpl.xml                |   19 +
 examples/pom-standalone.xml                     |   19 +
 examples/pom.xml                                |   21 +-
 .../datastructures/IgniteSemaphoreExample.java  |   21 +-
 .../persistentstore/PersistentStoreExample.java |   29 +-
 .../ml/math/matrix/CacheMatrixExample.java      |    4 +-
 .../ml/math/vector/CacheVectorExample.java      |    4 +-
 modules/aop/pom.xml                             |    2 +-
 modules/apache-license-gen/pom.xml              |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/benchmarks/pom.xml                      |    2 +-
 modules/camel/pom.xml                           |    2 +-
 modules/cassandra/pom.xml                       |    2 +-
 modules/cassandra/serializers/pom.xml           |    4 +-
 modules/cassandra/store/pom.xml                 |    4 +-
 modules/clients/pom.xml                         |    8 +-
 modules/clients/src/test/config/jdbc-config.xml |    1 +
 .../jdbc2/JdbcConnectionReopenTest.java         |   51 +
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |   36 +
 .../jdbc2/JdbcDeleteStatementSelfTest.java      |   22 +
 .../jdbc2/JdbcInsertStatementSelfTest.java      |  159 ++
 .../jdbc2/JdbcMergeStatementSelfTest.java       |   41 +
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |  154 +-
 .../jdbc2/JdbcStatementBatchingSelfTest.java    |  133 ++
 .../jdbc2/JdbcUpdateStatementSelfTest.java      |   24 +
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   16 +-
 .../jdbc/thin/JdbcThinAbstractSelfTest.java     |   71 +-
 .../thin/JdbcThinComplexDmlDdlSelfTest.java     |  486 ++++
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   | 1431 +++++++++++-
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |  337 ++-
 .../JdbcThinMissingLongArrayResultsTest.java    |  341 +++
 .../jdbc/thin/JdbcThinNoDefaultSchemaTest.java  |   82 +-
 .../thin/JdbcThinPreparedStatementSelfTest.java |  261 ++-
 .../jdbc/thin/JdbcThinResultSetSelfTest.java    | 1193 +++++++++-
 .../jdbc/thin/JdbcThinStatementSelfTest.java    |  781 ++++++-
 modules/clients/src/test/keystore/ca/node01.jks |  Bin 0 -> 3719 bytes
 modules/clients/src/test/keystore/ca/node02.jks |  Bin 0 -> 4598 bytes
 modules/clients/src/test/keystore/ca/node03.jks |  Bin 0 -> 3754 bytes
 modules/clients/src/test/keystore/ca/oneca.cnf  |   31 +
 modules/clients/src/test/keystore/ca/oneca.key  |   28 +
 .../clients/src/test/keystore/ca/oneindex.txt   |    1 +
 .../src/test/keystore/ca/oneindex.txt.attr      |    1 +
 modules/clients/src/test/keystore/ca/oneserial  |    1 +
 .../clients/src/test/keystore/ca/trust-both.jks |  Bin 0 -> 1718 bytes
 .../clients/src/test/keystore/ca/trust-one.jks  |  Bin 0 -> 877 bytes
 .../clients/src/test/keystore/ca/trust-two.jks  |  Bin 0 -> 891 bytes
 modules/clients/src/test/keystore/ca/twoca.cnf  |   31 +
 modules/clients/src/test/keystore/ca/twoca.key  |   28 +
 .../clients/src/test/keystore/ca/twoindex.txt   |    2 +
 .../src/test/keystore/ca/twoindex.txt.attr      |    1 +
 modules/clients/src/test/keystore/ca/twoserial  |    1 +
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 modules/core/pom.xml                            |    2 +-
 .../org/apache/ignite/IgniteJdbcDriver.java     |   31 +
 .../org/apache/ignite/IgniteJdbcThinDriver.java |   24 +-
 .../java/org/apache/ignite/IgniteServices.java  |  112 +-
 .../apache/ignite/IgniteSystemProperties.java   |    3 +
 .../org/apache/ignite/cache/CacheMetrics.java   |   10 +
 .../org/apache/ignite/cache/QueryEntity.java    |   15 +-
 .../org/apache/ignite/cache/QueryIndex.java     |   10 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |   38 +-
 .../query/annotations/QueryGroupIndex.java      |   20 +
 .../cache/query/annotations/QuerySqlField.java  |   23 +
 .../configuration/CheckpointWriteOrder.java     |   33 +
 .../configuration/IgniteConfiguration.java      |    5 +
 .../configuration/MemoryConfiguration.java      |    8 +-
 .../PersistentStoreConfiguration.java           |   31 +-
 .../internal/DuplicateTypeIdException.java      |   74 +
 .../internal/IgniteDiagnosticMessage.java       |    2 +-
 .../org/apache/ignite/internal/IgniteEx.java    |   14 +
 .../apache/ignite/internal/IgniteKernal.java    |   46 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    6 +
 .../ignite/internal/IgniteServicesImpl.java     |   64 +-
 .../ignite/internal/MarshallerContextImpl.java  |   27 +-
 .../internal/MarshallerMappingFileStore.java    |   76 +-
 .../ignite/internal/binary/BinaryContext.java   |   66 +-
 .../ignite/internal/binary/BinaryUtils.java     |   26 +
 .../binary/builder/BinaryObjectBuilderImpl.java |    2 +
 .../GridClientConnectionManagerAdapter.java     |    1 -
 .../connection/GridClientNioTcpConnection.java  |    2 +-
 .../internal/jdbc/JdbcDatabaseMetadata.java     |    2 +-
 .../internal/jdbc/thin/JdbcThinConnection.java  |  135 +-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     | 1589 +++++++++++++
 .../jdbc/thin/JdbcThinParameterMetadata.java    |  115 +
 .../jdbc/thin/JdbcThinPreparedStatement.java    |   52 +-
 .../internal/jdbc/thin/JdbcThinResultSet.java   |  640 ++++--
 .../internal/jdbc/thin/JdbcThinStatement.java   |  102 +-
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |  154 +-
 .../internal/jdbc/thin/JdbcThinUtils.java       |   16 +-
 .../internal/jdbc2/JdbcBatchUpdateTask.java     |  215 ++
 .../ignite/internal/jdbc2/JdbcConnection.java   |   62 +-
 .../internal/jdbc2/JdbcDatabaseMetadata.java    |  504 +++--
 .../internal/jdbc2/JdbcPreparedStatement.java   |   25 +-
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |   16 +
 .../ignite/internal/jdbc2/JdbcQueryTaskV2.java  |  107 +
 .../ignite/internal/jdbc2/JdbcResultSet.java    |   11 +-
 .../ignite/internal/jdbc2/JdbcStatement.java    |   75 +-
 .../managers/communication/GridIoManager.java   |    7 -
 .../communication/GridIoMessageFactory.java     |    9 +-
 .../internal/managers/discovery/DiscoCache.java |  141 +-
 .../discovery/GridDiscoveryManager.java         |  159 +-
 .../ignite/internal/pagemem/PageUtils.java      |   26 +
 .../affinity/AffinityTopologyVersion.java       |    7 +
 .../affinity/GridAffinityAssignmentCache.java   |   42 +
 .../affinity/GridAffinityProcessor.java         |    8 +-
 .../cache/CacheAffinitySharedManager.java       |  754 +++++--
 .../cache/CacheClusterMetricsMXBeanImpl.java    |   10 +
 .../processors/cache/CacheGroupContext.java     |   18 +-
 .../cache/CacheLocalMetricsMXBeanImpl.java      |   10 +
 .../processors/cache/CacheMetricsImpl.java      |   36 +-
 .../processors/cache/CacheMetricsSnapshot.java  |   18 +
 .../processors/cache/CacheObjectUtils.java      |   65 +-
 .../cache/CachePartitionExchangeWorkerTask.java |    5 +-
 .../ClientCacheChangeDummyDiscoveryMessage.java |    5 +
 .../cache/ClientCacheUpdateTimeout.java         |    5 +
 .../processors/cache/ClusterCachesInfo.java     |   22 +-
 .../processors/cache/ExchangeContext.java       |  131 ++
 .../cache/ExchangeDiscoveryEvents.java          |  262 +++
 .../processors/cache/GridCacheAdapter.java      |   14 +-
 .../processors/cache/GridCacheContext.java      |    2 +-
 .../processors/cache/GridCacheEntryEx.java      |   10 +
 .../processors/cache/GridCacheIoManager.java    |   57 +-
 .../processors/cache/GridCacheLockCallback.java |   51 +
 .../processors/cache/GridCacheMapEntry.java     |   33 +-
 .../processors/cache/GridCacheMvccCallback.java |   51 -
 .../processors/cache/GridCacheMvccFuture.java   |   37 -
 .../processors/cache/GridCacheMvccManager.java  |   46 +-
 .../GridCachePartitionExchangeManager.java      |  517 ++++-
 .../processors/cache/GridCacheProcessor.java    |  139 +-
 .../cache/GridCacheSharedContext.java           |   20 +
 .../processors/cache/GridCacheUtils.java        |  153 +-
 .../cache/GridCacheVersionedFuture.java         |   37 +
 .../cache/IgniteCacheOffheapManager.java        |    2 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    | 1199 +---------
 .../processors/cache/IgniteCacheProxyImpl.java  |   26 +-
 .../cache/binary/BinaryMetadataTransport.java   |   14 +-
 .../binary/CacheObjectBinaryProcessor.java      |   20 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   20 +-
 .../dht/ClientCacheDhtTopologyFuture.java       |   12 +-
 .../dht/GridClientPartitionTopology.java        |  225 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   18 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   50 +-
 .../distributed/dht/GridDhtLockFuture.java      |    6 +-
 .../dht/GridDhtPartitionTopology.java           |   67 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  696 +++---
 .../dht/GridDhtPartitionsReservation.java       |    2 +-
 .../distributed/dht/GridDhtTopologyFuture.java  |   36 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   97 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   48 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  447 ++--
 .../dht/GridPartitionedGetFuture.java           |    4 +-
 .../dht/GridPartitionedSingleGetFuture.java     |    4 +-
 .../GridDhtAtomicAbstractUpdateFuture.java      |    2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   65 +-
 .../GridNearAtomicSingleUpdateFuture.java       |    1 -
 .../dht/colocated/GridDhtColocatedCache.java    |    6 +-
 .../colocated/GridDhtColocatedLockFuture.java   |    8 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |    4 +-
 .../preloader/CacheGroupAffinityMessage.java    |  339 +++
 .../CachePartitionFullCountersMap.java          |  135 ++
 .../CachePartitionPartialCountersMap.java       |  184 ++
 .../preloader/ForceRebalanceExchangeTask.java   |    5 +
 .../dht/preloader/GridDhtPartitionDemander.java |   31 +-
 .../preloader/GridDhtPartitionExchangeId.java   |   11 +
 .../dht/preloader/GridDhtPartitionMap.java      |   26 +-
 .../dht/preloader/GridDhtPartitionSupplier.java |    2 +-
 .../GridDhtPartitionsAbstractMessage.java       |   39 +-
 .../GridDhtPartitionsExchangeFuture.java        | 2116 ++++++++++++++----
 .../preloader/GridDhtPartitionsFullMessage.java |  226 +-
 .../GridDhtPartitionsSingleMessage.java         |  108 +-
 .../GridDhtPartitionsSingleRequest.java         |   55 +-
 .../dht/preloader/GridDhtPreloader.java         |   40 +-
 .../IgniteDhtPartitionCountersMap.java          |    7 +
 .../IgniteDhtPartitionCountersMap2.java         |   69 +
 .../dht/preloader/InitNewCoordinatorFuture.java |  339 +++
 .../RebalanceReassignExchangeTask.java          |    5 +
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearCacheEntry.java    |    4 +-
 .../distributed/near/GridNearGetFuture.java     |    6 +-
 .../distributed/near/GridNearLockFuture.java    |    8 +-
 ...arOptimisticSerializableTxPrepareFuture.java |    3 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |    3 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    3 +-
 .../near/GridNearTransactionalCache.java        |    6 +-
 .../cache/distributed/near/GridNearTxLocal.java |   24 +-
 .../near/GridNearTxPrepareFutureAdapter.java    |    4 +-
 .../near/GridNearTxPrepareRequest.java          |   14 +
 .../processors/cache/local/GridLocalCache.java  |    5 +-
 .../cache/local/GridLocalLockFuture.java        |    4 +-
 .../local/atomic/GridLocalAtomicCache.java      |    8 +-
 .../GridCacheDatabaseSharedManager.java         |  334 ++-
 .../persistence/GridCacheOffheapManager.java    |   32 +-
 .../IgniteCacheDatabaseSharedManager.java       |   49 +-
 .../cache/persistence/MemoryMetricsImpl.java    |   26 +-
 .../cache/persistence/MemoryPolicy.java         |    3 +-
 .../cache/persistence/MetadataStorage.java      |   21 +-
 .../cache/persistence/file/FilePageStore.java   |    4 +-
 .../persistence/file/FilePageStoreManager.java  |    4 +-
 .../persistence/freelist/FreeListImpl.java      |    7 +-
 .../persistence/pagemem/PageMemoryImpl.java     |   80 +-
 .../cache/persistence/tree/io/PageIO.java       |   25 +-
 .../wal/AbstractWalRecordsIterator.java         |   11 +-
 .../persistence/wal/ByteBufferExpander.java     |   27 +-
 .../wal/FileWriteAheadLogManager.java           |   79 +-
 .../reader/StandaloneWalRecordsIterator.java    |    9 +-
 .../processors/cache/query/CacheQueryEntry.java |   58 +
 .../query/GridCacheDistributedQueryManager.java |   16 +-
 .../cache/query/GridCacheQueryAdapter.java      |   59 +-
 .../cache/query/GridCacheQueryManager.java      |  476 ++--
 .../cache/query/QueryEntityTypeDescriptor.java  |   17 +-
 .../cache/transactions/IgniteTxAdapter.java     |    2 +-
 .../cache/transactions/IgniteTxHandler.java     |  215 +-
 .../IgniteTxImplicitSingleStateImpl.java        |    5 +-
 .../cache/transactions/IgniteTxManager.java     |    6 +-
 .../cache/tree/AbstractDataInnerIO.java         |  109 +
 .../cache/tree/AbstractDataLeafIO.java          |  108 +
 .../cache/tree/AbstractPendingEntryInnerIO.java |   99 +
 .../cache/tree/AbstractPendingEntryLeafIO.java  |   98 +
 .../cache/tree/CacheDataRowStore.java           |   80 +
 .../processors/cache/tree/CacheDataTree.java    |  256 +++
 .../cache/tree/CacheIdAwareDataInnerIO.java     |   48 +
 .../cache/tree/CacheIdAwareDataLeafIO.java      |   48 +
 .../tree/CacheIdAwarePendingEntryInnerIO.java   |   48 +
 .../tree/CacheIdAwarePendingEntryLeafIO.java    |   48 +
 .../processors/cache/tree/DataInnerIO.java      |   48 +
 .../processors/cache/tree/DataLeafIO.java       |   48 +
 .../internal/processors/cache/tree/DataRow.java |  105 +
 .../cache/tree/PendingEntriesTree.java          |  118 +
 .../cache/tree/PendingEntryInnerIO.java         |   48 +
 .../cache/tree/PendingEntryLeafIO.java          |   48 +
 .../processors/cache/tree/PendingRow.java       |   83 +
 .../processors/cache/tree/PendingRowIO.java     |   44 +
 .../processors/cache/tree/RowLinkIO.java        |   44 +
 .../processors/cache/tree/SearchRow.java        |   76 +
 .../IgniteCacheObjectProcessorImpl.java         |  164 --
 .../UserCacheObjectByteArrayImpl.java           |   59 +
 .../cacheobject/UserCacheObjectImpl.java        |   82 +
 .../cacheobject/UserKeyCacheObjectImpl.java     |  101 +
 .../closure/GridClosureProcessor.java           |   36 +-
 .../cluster/GridClusterStateProcessor.java      |   62 +-
 .../continuous/GridContinuousProcessor.java     |    7 +-
 .../datastreamer/DataStreamProcessor.java       |   57 +-
 .../datastreamer/DataStreamerImpl.java          |  130 +-
 .../internal/processors/job/GridJobWorker.java  |   19 -
 .../odbc/SqlListenerConnectionContext.java      |   36 +-
 .../processors/odbc/SqlListenerNioListener.java |   82 +-
 .../odbc/SqlListenerRequestHandler.java         |    9 +
 .../processors/odbc/SqlListenerUtils.java       |    6 +
 .../odbc/jdbc/JdbcBatchExecuteRequest.java      |   20 +-
 .../odbc/jdbc/JdbcBatchExecuteResult.java       |    6 +
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |   75 +-
 .../odbc/jdbc/JdbcConnectionContext.java        |  118 +
 .../processors/odbc/jdbc/JdbcIndexMeta.java     |  192 ++
 .../odbc/jdbc/JdbcMetaColumnsRequest.java       |  102 +
 .../odbc/jdbc/JdbcMetaColumnsResult.java        |   99 +
 .../odbc/jdbc/JdbcMetaIndexesRequest.java       |   88 +
 .../odbc/jdbc/JdbcMetaIndexesResult.java        |   98 +
 .../odbc/jdbc/JdbcMetaParamsRequest.java        |   87 +
 .../odbc/jdbc/JdbcMetaParamsResult.java         |   97 +
 .../odbc/jdbc/JdbcMetaPrimaryKeysRequest.java   |   88 +
 .../odbc/jdbc/JdbcMetaPrimaryKeysResult.java    |   99 +
 .../odbc/jdbc/JdbcMetaSchemasRequest.java       |   73 +
 .../odbc/jdbc/JdbcMetaSchemasResult.java        |   73 +
 .../odbc/jdbc/JdbcMetaTablesRequest.java        |   87 +
 .../odbc/jdbc/JdbcMetaTablesResult.java         |   97 +
 .../processors/odbc/jdbc/JdbcParameterMeta.java |  163 ++
 .../odbc/jdbc/JdbcPrimaryKeyMeta.java           |  131 ++
 .../odbc/jdbc/JdbcQueryCloseRequest.java        |    4 +-
 .../odbc/jdbc/JdbcQueryExecuteRequest.java      |    8 +-
 .../odbc/jdbc/JdbcQueryExecuteResult.java       |   12 +-
 .../odbc/jdbc/JdbcQueryFetchRequest.java        |    4 +-
 .../odbc/jdbc/JdbcQueryFetchResult.java         |   12 +-
 .../odbc/jdbc/JdbcQueryMetadataRequest.java     |   18 +-
 .../odbc/jdbc/JdbcQueryMetadataResult.java      |   14 +-
 .../processors/odbc/jdbc/JdbcRequest.java       |   67 +-
 .../odbc/jdbc/JdbcRequestHandler.java           |  282 ++-
 .../processors/odbc/jdbc/JdbcResult.java        |   58 +-
 .../processors/odbc/jdbc/JdbcTableMeta.java     |   82 +
 .../processors/odbc/jdbc/JdbcUtils.java         |   37 +-
 .../odbc/odbc/OdbcConnectionContext.java        |  116 +
 .../processors/odbc/odbc/OdbcMessageParser.java |    4 +-
 .../odbc/odbc/OdbcQueryExecuteResult.java       |   24 +-
 .../odbc/odbc/OdbcRequestHandler.java           |   89 +-
 .../processors/odbc/odbc/OdbcUtils.java         |   22 +
 .../platform/cache/PlatformCache.java           |    2 +
 .../datastreamer/PlatformDataStreamer.java      |    3 +-
 .../services/PlatformAbstractService.java       |    8 +
 .../platform/services/PlatformServices.java     |   64 +-
 .../platform/utils/PlatformUtils.java           |    2 +-
 .../processors/query/GridQueryIndexing.java     |   18 +-
 .../processors/query/GridQueryProcessor.java    |  170 +-
 .../query/GridQueryTypeDescriptor.java          |    7 +
 .../internal/processors/query/QueryField.java   |   64 +
 .../internal/processors/query/QuerySchema.java  |   37 +-
 .../query/QueryTypeDescriptorImpl.java          |   32 +-
 .../internal/processors/query/QueryUtils.java   |    7 +-
 .../messages/GridQueryNextPageResponse.java     |   36 +-
 .../query/schema/SchemaExchangeWorkerTask.java  |    5 +
 .../schema/SchemaIndexCacheVisitorImpl.java     |    7 +
 .../SchemaNodeLeaveExchangeWorkerTask.java      |    5 +
 .../query/schema/SchemaOperationWorker.java     |    2 +-
 .../SchemaAbstractAlterTableOperation.java      |   39 +
 .../SchemaAlterTableAddColumnOperation.java     |   96 +
 .../GridServiceDeploymentCompoundFuture.java    |  197 ++
 .../service/GridServiceProcessor.java           |  566 +++--
 .../service/PreparedConfigurations.java         |   53 +
 .../processors/task/GridTaskWorker.java         |   30 +-
 .../internal/util/GridSpinReadWriteLock.java    |    2 +-
 .../apache/ignite/internal/util/GridUnsafe.java |   14 +
 .../ignite/internal/util/IgniteUtils.java       |   41 +
 .../util/future/GridCompoundFuture.java         |   55 +-
 .../ignite/internal/util/lang/GridFunc.java     |    3 +-
 .../internal/util/nio/GridNioKeyAttachment.java |   33 +
 .../util/nio/GridNioRecoveryDescriptor.java     |    3 +-
 .../ignite/internal/util/nio/GridNioServer.java |  248 +-
 .../util/nio/GridSelectorNioSessionImpl.java    |   28 +-
 .../internal/util/nio/ssl/GridNioSslFilter.java |   12 +-
 .../apache/ignite/internal/util/typedef/X.java  |    4 +-
 .../visor/cache/VisorCacheStopTask.java         |   22 +-
 .../visor/cache/VisorCacheStopTaskArg.java      |   27 +
 .../node/VisorNodeDataCollectorTaskResult.java  |   29 +
 .../internal/visor/query/VisorQueryTask.java    |    1 +
 .../internal/visor/query/VisorQueryTaskArg.java |   41 +-
 .../org/apache/ignite/mxbean/IgniteMXBean.java  |   21 +
 .../services/ServiceDeploymentException.java    |   77 +
 .../communication/tcp/TcpCommunicationSpi.java  |   89 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   47 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   10 +
 .../tcp/internal/TcpDiscoveryNode.java          |   20 +
 .../org/apache/ignite/thread/IgniteThread.java  |    9 +
 .../resources/META-INF/classnames.properties    |   85 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 modules/core/src/test/config/tests.properties   |    8 +
 .../org.apache.ignite.plugin.PluginProvider     |    3 +-
 ...atformCachePluginConfigurationClosureFactory |    1 -
 ...rm.PlatformPluginConfigurationClosureFactory |    1 -
 .../cache/store/GridStoreLoadCacheTest.java     |  120 +
 .../ignite/internal/GridDiscoverySelfTest.java  |   30 -
 .../IgniteClientReconnectBinaryContexTest.java  |  115 +
 .../internal/TestDelayingCommunicationSpi.java  |   63 +
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   22 +
 ...CacheExchangeMessageDuplicatedStateTest.java |    9 +-
 .../cache/CacheGroupsMetricsRebalanceTest.java  |  123 +
 .../cache/CacheRebalancingSelfTest.java         |   88 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    |   17 +-
 ...ridCacheQuerySqlFieldInlineSizeSelfTest.java |  160 ++
 .../processors/cache/GridCacheTestEntryEx.java  |    6 +
 ...dMemoryConfigurationConsistencySelfTest.java |   79 +
 ...IgniteClientCacheInitializationFailTest.java |   10 +-
 .../IgniteClientCacheStartFailoverTest.java     |    4 +-
 .../IgniteClusterActivateDeactivateTest.java    |    4 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   26 +-
 ...niteTopologyValidatorGridSplitCacheTest.java |    6 +-
 ...AffinityCoordinatorDynamicStartStopTest.java |    2 +-
 .../binary/BinaryMetadataUpdatesFlowTest.java   |   48 +
 ...eAbstractDataStructuresFailoverSelfTest.java |    7 +-
 .../distributed/CacheExchangeMergeTest.java     | 1528 +++++++++++++
 .../CacheLateAffinityAssignmentTest.java        |  600 +++--
 ...CacheLoadingConcurrentGridStartSelfTest.java |   11 +
 .../CacheLockReleaseNodeLeaveTest.java          |   13 +-
 .../distributed/CachePartitionStateTest.java    |   18 +-
 ...ncurrentGridStartSelfTestAllowOverwrite.java |   33 +
 ...titionEvictionDuringReadThroughSelfTest.java |    2 +
 ...niteCacheClientNodeChangingTopologyTest.java |    5 +-
 ...teCacheClientNodePartitionsExchangeTest.java |   52 +-
 .../IgniteCacheMessageWriteTimeoutTest.java     |    4 +-
 .../IgniteOptimisticTxSuspendResumeTest.java    |   11 +-
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |  111 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   36 +-
 .../IgnitePdsCacheRebalancingAbstractTest.java  |  137 +-
 .../persistence/IgnitePdsDynamicCacheTest.java  |   43 +
 ...MarshallerMappingRestoreOnNodeStartTest.java |  116 +
 ...nitePersistenceSequentialCheckpointTest.java |   44 +
 .../IgnitePersistentStoreCacheGroupsTest.java   |   35 +-
 ...faultPageSizeBackwardsCompatibilityTest.java |  147 ++
 .../db/file/IgnitePdsEvictionTest.java          |    3 +-
 .../wal/IgniteWalHistoryReservationsTest.java   |   70 +-
 .../db/wal/IgniteWalRecoveryTest.java           |    2 +
 .../db/wal/crc/IgniteDataIntegrityTests.java    |   66 +-
 .../IgniteChangeGlobalStateTest.java            |   11 +-
 .../IgniteStandByClusterTest.java               |   48 +-
 .../join/JoinInActiveNodeToActiveCluster.java   |    4 +-
 .../GridCacheQueryTransformerSelfTest.java      |   41 +
 ...ContinuousQueryFailoverAbstractSelfTest.java |   11 +-
 .../processors/database/BPlusTreeSelfTest.java  |   39 +-
 .../datastreamer/DataStreamerImplSelfTest.java  |  143 +-
 .../processors/igfs/IgfsIgniteMock.java         |   10 +
 ...ServiceDeploymentCompoundFutureSelfTest.java |  242 ++
 ...GridServiceProcessorBatchDeploySelfTest.java |  742 ++++++
 .../internal/util/nio/GridNioSelfTest.java      |    2 +-
 .../platform/PlatformCacheWriteMetricsTask.java |   10 +
 .../platform/plugin/PlatformTestPlugin.java     |   27 -
 .../plugin/PlatformTestPluginConfiguration.java |   63 -
 ...rmTestPluginConfigurationClosureFactory.java |   61 -
 .../plugin/PlatformTestPluginException.java     |   34 -
 .../plugin/PlatformTestPluginExtension.java     |   51 -
 .../plugin/PlatformTestPluginProvider.java      |  103 -
 .../plugin/PlatformTestPluginTarget.java        |  236 --
 .../cache/PlatformGetCachePluginsTask.java      |   85 -
 .../PlatformTestCachePluginConfiguration.java   |   55 -
 ...formTestCachePluginConfigurationClosure.java |   48 -
 ...tCachePluginConfigurationClosureFactory.java |   37 -
 .../cache/PlatformTestCachePluginProvider.java  |   72 -
 .../spi/GridTcpSpiForwardingSelfTest.java       |    1 +
 .../GridAbstractCommunicationSelfTest.java      |   27 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   28 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |   39 +-
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   47 +-
 ...CommunicationRecoveryAckClosureSelfTest.java |   36 +-
 .../tcp/TcpCommunicationSpiDropNodesTest.java   |    3 +-
 .../discovery/tcp/IgniteClientConnectTest.java  |  163 ++
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   84 +-
 .../tcp/TcpDiscoverySslTrustedSelfTest.java     |   42 +
 .../TcpDiscoverySslTrustedUntrustedTest.java    |  140 ++
 .../ignite/testframework/GridTestUtils.java     |   20 +
 .../junits/common/GridCommonAbstractTest.java   |   22 +-
 .../junits/multijvm/IgniteProcessProxy.java     |   10 +
 .../ignite/testsuites/IgniteBasicTestSuite.java |    3 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |    7 +-
 .../testsuites/IgniteCacheTestSuite2.java       |    7 +-
 .../testsuites/IgniteCacheTestSuite6.java       |    3 +
 .../IgniteClientReconnectTestSuite.java         |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    4 +
 .../ignite/testsuites/IgnitePdsTestSuite.java   |    3 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   10 +
 modules/extdata/p2p/pom.xml                     |    2 +-
 modules/extdata/platform/README.txt             |    4 +
 .../extdata/platform/licenses/apache-2.0.txt    |  202 ++
 modules/extdata/platform/pom.xml                |   46 +
 .../platform/plugin/PlatformTestPlugin.java     |   27 +
 .../plugin/PlatformTestPluginConfiguration.java |   63 +
 ...rmTestPluginConfigurationClosureFactory.java |   61 +
 .../plugin/PlatformTestPluginException.java     |   34 +
 .../plugin/PlatformTestPluginExtension.java     |   51 +
 .../plugin/PlatformTestPluginProvider.java      |  103 +
 .../plugin/PlatformTestPluginTarget.java        |  236 ++
 .../cache/PlatformGetCachePluginsTask.java      |   85 +
 .../PlatformTestCachePluginConfiguration.java   |   55 +
 ...formTestCachePluginConfigurationClosure.java |   48 +
 ...tCachePluginConfigurationClosureFactory.java |   37 +
 .../cache/PlatformTestCachePluginProvider.java  |   72 +
 .../platform/plugin/cache/package-info.java     |   22 +
 .../ignite/platform/plugin/package-info.java    |   22 +
 .../org.apache.ignite.plugin.PluginProvider     |    1 +
 ...atformCachePluginConfigurationClosureFactory |    1 +
 ...rm.PlatformPluginConfigurationClosureFactory |    1 +
 .../extdata/uri/modules/uri-dependency/pom.xml  |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/flink/pom.xml                           |    2 +-
 modules/flume/pom.xml                           |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |    2 +-
 .../query/h2/opt/GridH2SpatialIndex.java        |    4 +-
 modules/hadoop/pom.xml                          |    2 +-
 .../HadoopExternalCommunication.java            |    5 +-
 modules/hibernate-4.2/pom.xml                   |    2 +-
 modules/hibernate-5.1/pom.xml                   |    2 +-
 modules/hibernate-core/pom.xml                  |    2 +-
 modules/indexing/pom.xml                        |    2 +-
 .../query/h2/DmlStatementsProcessor.java        |  273 ++-
 .../processors/query/h2/H2RowDescriptor.java    |  105 +-
 .../internal/processors/query/h2/H2Schema.java  |    2 +
 .../processors/query/h2/H2TableDescriptor.java  |    3 +-
 .../processors/query/h2/H2TableEngine.java      |   13 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   88 +-
 .../query/h2/database/H2TreeIndex.java          |   16 +-
 .../query/h2/ddl/DdlStatementsProcessor.java    |   50 +-
 .../query/h2/opt/GridH2IndexBase.java           |    4 +-
 .../processors/query/h2/opt/GridH2Table.java    |   71 +-
 .../query/h2/opt/GridLuceneDirectory.java       |    9 +-
 .../processors/query/h2/opt/GridLuceneFile.java |    6 +-
 .../query/h2/opt/GridLuceneIndex.java           |   17 +-
 .../h2/sql/GridSqlAlterTableAddColumn.java      |  113 +
 .../query/h2/sql/GridSqlQueryParser.java        |  160 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  243 +-
 .../query/h2/twostep/GridMergeIndex.java        |    7 +-
 .../h2/twostep/GridMergeIndexIterator.java      |  165 ++
 .../h2/twostep/GridReduceQueryExecutor.java     |   68 +-
 .../query/h2/twostep/MapNodeResults.java        |   19 +-
 .../query/h2/twostep/MapQueryLazyWorker.java    |  176 ++
 .../query/h2/twostep/MapQueryLazyWorkerKey.java |   97 +
 .../query/h2/twostep/MapQueryResult.java        |   46 +-
 .../query/h2/twostep/MapQueryResults.java       |   26 +-
 .../query/h2/twostep/MapRequestKey.java         |   23 +-
 .../h2/twostep/msg/GridH2QueryRequest.java      |    9 +
 ...titionOnAffinityRunWithCollisionSpiTest.java |    3 +-
 ...ynamicColumnsAbstractConcurrentSelfTest.java | 1056 +++++++++
 .../cache/index/DynamicColumnsAbstractTest.java |  311 +++
 ...umnsConcurrentAtomicPartitionedSelfTest.java |   33 +
 ...lumnsConcurrentAtomicReplicatedSelfTest.java |   33 +
 ...currentTransactionalPartitionedSelfTest.java |   33 +
 ...ncurrentTransactionalReplicatedSelfTest.java |   33 +
 .../DynamicIndexAbstractConcurrentSelfTest.java |  128 +-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  |  348 +++
 .../H2DynamicColumnsClientBasicSelfTest.java    |   28 +
 .../H2DynamicColumnsServerBasicSelfTest.java    |   28 +
 ...icColumnsServerCoordinatorBasicSelfTest.java |   28 +
 .../cache/index/LongIndexNameTest.java          |  212 ++
 .../processors/query/LazyQuerySelfTest.java     |  389 ++++
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   18 +-
 .../query/h2/IgniteSqlBigIntegerKeyTest.java    |  163 ++
 .../h2/database/InlineIndexHelperTest.java      |    4 +-
 .../query/h2/sql/GridQueryParsingTest.java      |   92 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   15 +
 .../IgniteCacheQuerySelfTestSuite2.java         |    9 +
 .../IgnitePdsWithIndexingCoreTestSuite.java     |    2 +
 modules/jcl/pom.xml                             |    2 +-
 modules/jms11/pom.xml                           |    2 +-
 modules/jta/pom.xml                             |    2 +-
 modules/kafka/pom.xml                           |    2 +-
 modules/kubernetes/pom.xml                      |    2 +-
 modules/log4j/pom.xml                           |    2 +-
 modules/log4j2/pom.xml                          |    2 +-
 modules/mesos/pom.xml                           |    2 +-
 modules/ml/pom.xml                              |    3 +-
 .../clustering/KMeansDistributedClusterer.java  |   55 +-
 .../java/org/apache/ignite/ml/math/Blas.java    |    4 +-
 .../ignite/ml/math/IdentityValueMapper.java     |    2 +
 .../java/org/apache/ignite/ml/math/Matrix.java  |   16 +
 .../apache/ignite/ml/math/MatrixKeyMapper.java  |   30 -
 .../apache/ignite/ml/math/MatrixStorage.java    |    7 +
 .../org/apache/ignite/ml/math/ValueMapper.java  |   37 -
 .../apache/ignite/ml/math/VectorKeyMapper.java  |   29 -
 .../ignite/ml/math/distributed/CacheUtils.java  |  546 +++++
 .../ml/math/distributed/DistributedStorage.java |   35 +
 .../ml/math/distributed/MatrixKeyMapper.java    |   33 +
 .../ignite/ml/math/distributed/ValueMapper.java |   37 +
 .../ml/math/distributed/VectorKeyMapper.java    |   32 +
 .../math/distributed/keys/BlockMatrixKey.java   |   30 +
 .../math/distributed/keys/MatrixCacheKey.java   |   35 +
 .../math/distributed/keys/RowColMatrixKey.java  |   30 +
 .../distributed/keys/impl/BlockMatrixKey.java   |  144 ++
 .../distributed/keys/impl/SparseMatrixKey.java  |  142 ++
 .../distributed/keys/impl/package-info.java     |   22 +
 .../ml/math/distributed/keys/package-info.java  |   22 +
 .../ml/math/distributed/package-info.java       |   22 +
 .../apache/ignite/ml/math/impls/CacheUtils.java |  559 -----
 .../ml/math/impls/matrix/AbstractMatrix.java    |   25 +
 .../ml/math/impls/matrix/CacheMatrix.java       |    6 +-
 .../impls/matrix/DenseLocalOnHeapMatrix.java    |    4 +-
 .../matrix/SparseBlockDistributedMatrix.java    |   16 +-
 .../impls/matrix/SparseDistributedMatrix.java   |   83 +-
 .../storage/matrix/BaseBlockMatrixKey.java      |   41 -
 .../impls/storage/matrix/BlockMatrixKey.java    |  144 --
 .../storage/matrix/BlockMatrixStorage.java      |   38 +-
 .../storage/matrix/CacheMatrixStorage.java      |    9 +-
 .../matrix/DenseOffHeapMatrixStorage.java       |    5 +
 .../storage/matrix/DiagonalMatrixStorage.java   |    5 +
 .../storage/matrix/FunctionMatrixStorage.java   |    5 +
 .../storage/matrix/MatrixDelegateStorage.java   |    5 +
 .../storage/matrix/PivotedMatrixStorage.java    |    5 +
 .../storage/matrix/RandomMatrixStorage.java     |    5 +
 .../matrix/SparseDistributedMatrixStorage.java  |   54 +-
 .../matrix/SparseLocalOnHeapMatrixStorage.java  |    6 +-
 .../storage/vector/CacheVectorStorage.java      |    4 +-
 .../ml/math/impls/vector/CacheVector.java       |    6 +-
 .../ml/math/impls/matrix/CacheMatrixTest.java   |    2 +-
 .../impls/matrix/MatrixImplementationsTest.java |   17 +
 .../impls/matrix/MatrixKeyMapperForTests.java   |    2 +-
 .../SparseDistributedBlockMatrixTest.java       |    7 +-
 .../matrix/SparseDistributedMatrixTest.java     |   40 +-
 .../ml/math/impls/vector/CacheVectorTest.java   |    2 +-
 modules/mqtt/pom.xml                            |    2 +-
 modules/osgi-karaf/pom.xml                      |    2 +-
 modules/osgi-paxlogging/pom.xml                 |    2 +-
 modules/osgi/pom.xml                            |    2 +-
 .../platforms/cpp/common/include/Makefile.am    |    1 +
 .../cpp/common/include/ignite/common/lazy.h     |  172 ++
 .../cpp/common/include/ignite/common/utils.h    |   51 +
 .../cpp/common/project/vs/common.vcxproj        |    1 +
 .../common/project/vs/common.vcxproj.filters    |    3 +
 .../platforms/cpp/core-test/config/invalid.xml  |   39 +
 .../cpp/core-test/config/isolated-32.xml        |   51 +
 .../cpp/core-test/config/isolated-default.xml   |   59 +
 .../platforms/cpp/core-test/config/isolated.xml |   32 +
 .../core-test/config/persistence-store-32.xml   |   51 +
 .../config/persistence-store-default.xml        |   36 +
 .../cpp/core-test/config/persistence-store.xml  |   32 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |   10 +
 .../project/vs/core-test.vcxproj.filters        |   19 +
 .../cpp/core-test/src/cluster_test.cpp          |   33 +
 .../cpp/core-test/src/ignition_test.cpp         |   37 +-
 .../cpp/core-test/src/interop_test.cpp          |   11 -
 .../ignite/cache/query/query_sql_fields.h       |   71 +-
 .../impl/cache/query/query_fields_row_impl.h    |    2 +-
 .../cpp/core/include/ignite/impl/ignite_impl.h  |   25 +-
 modules/platforms/cpp/core/src/ignition.cpp     |    2 +-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |   45 +-
 modules/platforms/cpp/ignite/src/ignite.cpp     |   23 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |    3 +-
 .../cpp/odbc-test/include/test_utils.h          |    9 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |    8 +-
 .../cpp/odbc-test/src/configuration_test.cpp    |   11 +
 .../cpp/odbc-test/src/meta_queries_test.cpp     |  113 +
 .../cpp/odbc-test/src/queries_test.cpp          |  184 +-
 .../platforms/cpp/odbc-test/src/test_utils.cpp  |   13 +
 .../include/ignite/odbc/config/configuration.h  |   26 +
 .../cpp/odbc/include/ignite/odbc/cursor.h       |    7 +
 .../cpp/odbc/include/ignite/odbc/message.h      |   18 +-
 .../odbc/include/ignite/odbc/protocol_version.h |    1 +
 .../ignite/odbc/query/column_metadata_query.h   |    3 +
 .../odbc/include/ignite/odbc/query/data_query.h |    3 +
 .../ignite/odbc/query/table_metadata_query.h    |    3 +
 .../include/ignite/odbc/query/type_info_query.h |    3 +
 .../odbc/system/ui/dsn_configuration_window.h   |    4 +
 .../src/system/ui/dsn_configuration_window.cpp  |   34 +-
 .../cpp/odbc/src/config/configuration.cpp       |    2 +
 .../cpp/odbc/src/config/connection_info.cpp     |   19 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |    6 +-
 modules/platforms/cpp/odbc/src/cursor.cpp       |    5 +
 modules/platforms/cpp/odbc/src/dsn_config.cpp   |    3 +
 modules/platforms/cpp/odbc/src/message.cpp      |   22 +-
 .../platforms/cpp/odbc/src/meta/column_meta.cpp |    3 +
 .../platforms/cpp/odbc/src/protocol_version.cpp |    8 +-
 .../cpp/odbc/src/query/batch_query.cpp          |    7 +-
 .../odbc/src/query/column_metadata_query.cpp    |   16 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |   30 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |   16 +-
 .../cpp/odbc/src/query/type_info_query.cpp      |   16 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |   32 +-
 .../Binary/BinaryDynamicRegistrationTest.cs     |   49 +
 .../Binary/BinarySelfTest.cs                    |   19 +-
 .../Cache/Affinity/AffinityFieldTest.cs         |   10 +-
 .../Cache/CacheAbstractTest.cs                  |   50 +-
 .../Compute/ComputeApiTest.cs                   |    6 +
 .../Dataload/DataStreamerTest.cs                |   71 +-
 .../Services/ServicesAsyncWrapper.cs            |    9 +-
 .../Services/ServicesTest.cs                    |   81 +-
 .../Apache.Ignite.Core.csproj                   |    6 +-
 .../Cache/CachePartialUpdateException.cs        |    9 +-
 .../Configuration/MemoryPolicyConfiguration.cs  |    4 +-
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |   31 +-
 .../Datastream/StreamTransformer.cs             |    6 +-
 .../Datastream/StreamVisitor.cs                 |    6 +-
 .../Apache.Ignite.Core/Events/JobEvent.cs       |    6 +-
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |    4 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |  132 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |   12 +-
 .../Impl/Binary/BinaryProcessor.cs              |    2 +-
 .../Impl/Binary/BinaryReader.cs                 |  110 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |  157 +-
 .../Impl/Binary/BinaryTypeId.cs                 |  228 ++
 .../Impl/Binary/BinaryUtils.cs                  |  155 +-
 .../Impl/Binary/BinaryWriter.cs                 |  161 +-
 .../Impl/Binary/BinaryWriterExtensions.cs       |    4 +-
 .../Impl/Binary/Marshaller.cs                   |   75 +-
 .../Impl/Binary/Metadata/BinaryType.cs          |   62 +-
 .../Impl/Cache/CacheAffinityImpl.cs             |    2 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheEntry.cs |   31 +-
 .../Impl/Cache/CacheEntryFilterHolder.cs        |    2 +-
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |    4 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |    3 +-
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |  264 ---
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |    2 +-
 .../Impl/Cache/Query/PlatformQueryQursorBase.cs |   84 +
 .../Impl/Cache/Query/QueryCursor.cs             |    2 +-
 .../Impl/Cache/Query/QueryCursorBase.cs         |  288 +++
 .../Impl/Cache/Store/CacheStoreInternal.cs      |   14 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |    2 +-
 .../Apache.Ignite.Core/Impl/Common/Classpath.cs |    1 +
 .../Impl/Compute/Closure/ComputeActionJob.cs    |    4 +-
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |    6 +-
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |    4 +-
 .../Closure/ComputeReducingClosureTask.cs       |    2 +-
 .../Compute/Closure/IComputeResourceInjector.cs |    2 +-
 .../Impl/Compute/ComputeFunc.cs                 |    2 +-
 .../Impl/Compute/ComputeJob.cs                  |    2 +-
 .../Impl/Compute/ComputeJobHolder.cs            |   21 +-
 .../Impl/Compute/ComputeOutFunc.cs              |    2 +-
 .../Impl/Deployment/PeerAssemblyResolver.cs     |   13 +-
 .../Impl/Deployment/PeerLoadingExtensions.cs    |    4 +-
 .../Impl/Deployment/PeerLoadingObjectHolder.cs  |    4 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   14 +-
 .../Apache.Ignite.Core/Impl/IIgniteInternal.cs  |   69 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |   13 +-
 .../Impl/Messaging/MessageListenerHolder.cs     |    4 +-
 .../Impl/PlatformTargetAdapter.cs               |   13 +-
 .../Impl/Resource/ResourceProcessor.cs          |    2 +-
 .../Impl/Resource/ResourceTypeDescriptor.cs     |    2 +-
 .../Impl/Services/ServiceProxySerializer.cs     |   36 +
 .../Impl/Services/Services.cs                   |   30 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |   37 +-
 .../PersistentStoreConfiguration.cs             |    2 +-
 .../Resource/InstanceResourceAttribute.cs       |    3 +-
 .../Services/ServiceDeploymentException.cs      |  111 +
 .../Services/ServiceInvocationException.cs      |    9 +-
 modules/platforms/dotnet/README.md              |    2 +-
 modules/platforms/dotnet/docfx/.gitignore       |    9 +
 .../dotnet/docfx/Apache.Ignite.docfx.json       |   88 +
 modules/platforms/dotnet/docfx/README.txt       |    2 +
 modules/platforms/dotnet/docfx/api/index.md     |    5 +
 modules/platforms/dotnet/docfx/filterConfig.yml |   19 +
 .../platforms/dotnet/docfx/generate-docs.cmd    |    1 +
 .../platforms/dotnet/docfx/images/favicon.ico   |  Bin 0 -> 1150 bytes
 .../dotnet/docfx/images/logo_ignite_32_32.png   |  Bin 0 -> 1676 bytes
 modules/platforms/dotnet/docfx/index.md         |   10 +
 modules/platforms/dotnet/docfx/toc.yml          |   19 +
 modules/rest-http/pom.xml                       |    2 +-
 .../http/jetty/GridJettyRestHandler.java        |   31 +-
 modules/rocketmq/pom.xml                        |    2 +-
 modules/scalar-2.10/pom.xml                     |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |    2 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spark-2.10/pom.xml                      |    2 +-
 modules/spark/pom.xml                           |    2 +-
 modules/spring-data/pom.xml                     |    2 +-
 modules/spring/pom.xml                          |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/storm/pom.xml                           |    2 +-
 modules/tools/pom.xml                           |    2 +-
 modules/twitter/pom.xml                         |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 modules/visor-console-2.10/pom.xml              |    2 +-
 modules/visor-console/pom.xml                   |    2 +-
 .../commands/cache/VisorCacheCommand.scala      |    4 +-
 .../commands/top/VisorTopologyCommand.scala     |  111 +-
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |    9 +-
 .../top/VisorActivationCommandSpec.scala        |   67 +
 .../testsuites/VisorConsoleSelfTestSuite.scala  |    3 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web-console/backend/app/agentSocket.js  |   29 +-
 modules/web-console/backend/app/mongo.js        |    6 +-
 modules/web-console/frontend/app/app.js         |    3 +-
 .../web-console-footer-links/component.js       |   23 +
 .../web-console-footer-links/style.scss         |   31 +
 .../web-console-footer-links/template.pug       |   17 +
 .../app/components/web-console-footer/index.js  |    4 +-
 .../components/web-console-footer/style.scss    |    8 +-
 .../components/web-console-footer/template.pug  |    1 +
 .../components/web-console-header/style.scss    |   55 +-
 .../components/web-console-header/template.pug  |   13 +-
 .../frontend/app/data/getting-started.json      |   13 +
 .../ui-ace-java/ui-ace-java.controller.js       |   13 +
 .../ui-ace-spring/ui-ace-spring.controller.js   |   13 +
 .../app/modules/agent/AgentManager.service.js   |   42 +-
 .../modules/configuration/Version.service.js    |    2 +-
 .../generator/AbstractTransformer.js            |    2 +-
 .../generator/ConfigurationGenerator.js         |    3 +-
 .../frontend/app/modules/sql/sql.controller.js  |   21 +-
 .../states/configuration/caches/memory.pug      |    2 +-
 .../states/configuration/caches/store.pug       |    2 +-
 .../states/configuration/clusters/connector.pug |    2 +-
 .../states/configuration/clusters/memory.pug    |    2 +-
 .../states/configuration/clusters/swap.pug      |    2 +-
 .../states/configuration/domains/general.pug    |    2 +-
 .../states/configuration/domains/query.pug      |    7 +
 .../states/configuration/domains/store.pug      |    2 +-
 .../app/services/LegacyTable.service.js         |    6 +-
 .../frontend/controllers/domains-controller.js  |   24 +-
 .../frontend/public/images/multicluster.png     |  Bin 0 -> 21921 bytes
 .../frontend/public/stylesheets/style.scss      |    4 +
 .../views/configuration/domains-import.tpl.pug  |    6 +
 .../frontend/views/includes/header-right.pug    |    4 -
 modules/web-console/frontend/views/index.pug    |    8 +-
 .../web-console/frontend/views/signin.tpl.pug   |    6 +
 .../web-console/frontend/views/sql/sql.tpl.pug  |    7 +
 .../frontend/webpack/webpack.common.js          |    1 -
 modules/web-console/pom.xml                     |    2 +-
 modules/web-console/web-agent/pom.xml           |    2 +-
 .../agent/handlers/AbstractListener.java        |   21 +
 .../console/agent/handlers/ClusterListener.java |   34 +-
 .../ignite/console/agent/rest/RestExecutor.java |  201 +-
 .../ignite/console/agent/rest/RestResult.java   |   25 +-
 modules/web/ignite-appserver-test/pom.xml       |    2 +-
 modules/web/ignite-websphere-test/pom.xml       |    2 +-
 modules/web/pom.xml                             |    2 +-
 .../config/benchmark-bin-identity.properties    |    1 +
 .../config/benchmark-cache-load.properties      |    1 +
 .../config/benchmark-client-mode.properties     |    1 +
 .../config/benchmark-failover.properties        |    1 +
 .../yardstick/config/benchmark-full.properties  |    1 +
 .../yardstick/config/benchmark-h2.properties    |    1 +
 .../config/benchmark-multicast.properties       |    1 +
 .../yardstick/config/benchmark-mysql.properties |    1 +
 .../yardstick/config/benchmark-pgsql.properties |    1 +
 .../config/benchmark-put-indexed-val.properties |    1 +
 .../benchmark-query-put-separated.properties    |    1 +
 .../yardstick/config/benchmark-query.properties |    1 +
 .../config/benchmark-remote.properties          |    1 +
 .../config/benchmark-sql-dml.properties         |    1 +
 .../yardstick/config/benchmark-store.properties |    1 +
 modules/yardstick/config/benchmark.properties   |    1 +
 modules/yardstick/pom.xml                       |    2 +-
 .../cache/WaitMapExchangeFinishCallable.java    |    4 +-
 .../cache/jdbc/JdbcAbstractBenchmark.java       |    4 +-
 .../yardstick/cache/jdbc/RdbmsBenchmark.java    |    6 +-
 modules/yarn/pom.xml                            |    2 +-
 modules/zeromq/pom.xml                          |    2 +-
 modules/zookeeper/pom.xml                       |    2 +-
 parent/pom.xml                                  |    7 +
 pom.xml                                         |  156 +-
 794 files changed, 40244 insertions(+), 9748 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c4ee47d2/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index a7c44c9,0ff9cfe..3f89840
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@@ -360,15 -362,13 +363,15 @@@ public class DmlStatementsProcessor 
              return doFastUpdate(plan, fieldsQry.getArgs());
          }
  
 -        assert !F.isEmpty(plan.selectQry);
 +        assert !F.isEmpty(plan.rows) ^ !F.isEmpty(plan.selectQry);
  
 -        QueryCursorImpl<List<?>> cur;
 +        Iterable<List<?>> cur;
  
          // Do a two-step query only if locality flag is not set AND if plan's SELECT corresponds to an actual
-         // subquery and not some dummy stuff like "select 1, 2, 3;"
+         // sub-query and not some dummy stuff like "select 1, 2, 3;"
          if (!loc && !plan.isLocSubqry) {
 +            assert !F.isEmpty(plan.selectQry);
 +
              SqlFieldsQuery newFieldsQry = new SqlFieldsQuery(plan.selectQry, fieldsQry.isCollocated())
                  .setArgs(fieldsQry.getArgs())
                  .setDistributedJoins(fieldsQry.isDistributedJoins())


[03/31] ignite git commit: Merge remote-tracking branch 'apache/ignite-2.0' into ignite-4490

Posted by vo...@apache.org.
Merge remote-tracking branch 'apache/ignite-2.0' into ignite-4490

# Conflicts:
#	modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java


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

Branch: refs/heads/ignite-6022-proto
Commit: 5982e39a7bbf5c40ffc614f7efe6302338c93710
Parents: 8c785cc aa46bc7
Author: Alexander Paschenko <al...@gmail.com>
Authored: Wed Feb 8 18:40:38 2017 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Wed Feb 8 18:40:38 2017 +0300

----------------------------------------------------------------------
 .gitignore                                      |   2 +-
 DEVNOTES.txt                                    |   3 +
 assembly/dependencies-fabric-lgpl.xml           |   1 +
 assembly/dependencies-fabric.xml                |   1 +
 .../tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java |  37 +-
 .../TcpDiscoveryS3IpFinderAbstractSelfTest.java |  84 ++
 ...3IpFinderAwsCredentialsProviderSelfTest.java |  46 +
 ...scoveryS3IpFinderAwsCredentialsSelfTest.java |  45 +
 .../s3/TcpDiscoveryS3IpFinderSelfTest.java      |  79 --
 .../ignite/testsuites/IgniteS3TestSuite.java    |  26 +-
 .../stream/camel/IgniteCamelStreamerTest.java   |   4 +-
 .../cassandra/persistence/PojoKeyField.java     |   2 +-
 .../ClientAbstractMultiNodeSelfTest.java        |   8 +-
 ...BinaryMarshallerInsertStatementSelfTest.java |   2 -
 ...cBinaryMarshallerMergeStatementSelfTest.java |   2 -
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |   2 +-
 .../ignite/jdbc/JdbcMetadataSelfTest.java       |   4 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   3 +-
 modules/cloud/pom.xml                           |   6 +-
 .../ignite/codegen/MessageCodeGenerator.java    |   2 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   4 +-
 .../apache/ignite/IgniteSystemProperties.java   |  13 +
 .../org/apache/ignite/cache/QueryEntity.java    |   1 -
 .../ignite/cache/affinity/AffinityKey.java      |   4 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |   6 +
 .../store/jdbc/CacheAbstractJdbcStore.java      |  11 +-
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   |   2 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java |  19 +
 .../cache/store/jdbc/JdbcTypesTransformer.java  |  17 +
 .../store/jdbc/dialect/BasicJdbcDialect.java    |  31 +-
 .../configuration/CacheConfiguration.java       |   4 +-
 .../configuration/HadoopConfiguration.java      |   2 +-
 .../configuration/IgniteConfiguration.java      | 110 ++-
 .../org/apache/ignite/events/CacheEvent.java    |   6 +-
 .../ignite/events/CacheQueryReadEvent.java      |   8 +-
 .../apache/ignite/hadoop/HadoopInputSplit.java  |  54 ++
 .../org/apache/ignite/hadoop/HadoopJob.java     |  74 ++
 .../ignite/hadoop/HadoopMapReducePlan.java      |  80 ++
 .../ignite/hadoop/HadoopMapReducePlanner.java   |  40 +
 .../igfs/IgfsGroupDataBlocksKeyMapper.java      |  17 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |  28 +
 .../local/LocalIgfsSecondaryFileSystem.java     | 130 ++-
 .../apache/ignite/internal/GridComponent.java   |  48 +-
 .../ignite/internal/GridKernalContext.java      |  29 +-
 .../ignite/internal/GridKernalContextImpl.java  |  58 +-
 .../ignite/internal/GridPluginComponent.java    |  21 +-
 .../org/apache/ignite/internal/GridTopic.java   |   3 +
 .../ignite/internal/IgniteComputeImpl.java      |   2 +
 .../apache/ignite/internal/IgniteKernal.java    |  34 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  89 +-
 .../ignite/internal/MappingStoreTask.java       |  58 ++
 .../internal/MarshallerContextAdapter.java      | 211 -----
 .../ignite/internal/MarshallerContextImpl.java  | 664 +++++++++-----
 .../internal/MarshallerMappingFileStore.java    | 174 ++++
 .../ignite/internal/MarshallerPlatformIds.java  |  30 +
 .../ignite/internal/binary/BinaryContext.java   |  12 +-
 .../internal/binary/BinaryEnumObjectImpl.java   |  14 +-
 .../internal/binary/BinaryMarshaller.java       |   1 -
 .../ignite/internal/binary/BinaryMetadata.java  |   5 +-
 .../internal/binary/BinaryObjectExImpl.java     |   8 +-
 .../ignite/internal/binary/BinaryTypeProxy.java |  15 +-
 .../ignite/internal/binary/BinaryUtils.java     |   4 +-
 .../GridClientConnectionManagerAdapter.java     |   7 +-
 .../impl/connection/GridClientTopology.java     |  53 +-
 .../GridClientOptimizedMarshaller.java          |  16 +-
 .../impl/GridRouterCommandLineStartup.java      |   4 +-
 .../cluster/ClusterNodeLocalMapImpl.java        |   1 -
 .../igfs/common/IgfsHandshakeRequest.java       |  17 -
 .../internal/igfs/common/IgfsMarshaller.java    |   6 +-
 .../internal/jdbc2/JdbcSqlFieldsQuery.java      |   1 -
 .../ignite/internal/jdbc2/JdbcStatement.java    |   2 -
 .../internal/managers/GridManagerAdapter.java   |  19 +-
 .../managers/communication/GridIoManager.java   |   8 +-
 .../communication/GridIoMessageFactory.java     |  12 +
 .../managers/communication/GridIoPolicy.java    |  13 +-
 .../discovery/GridDiscoveryManager.java         |  85 +-
 .../internal/mxbean/IgniteStandardMXBean.java   |  12 +-
 .../processors/GridProcessorAdapter.java        |  21 +-
 .../cache/CacheEvictableEntryImpl.java          |   4 +-
 .../cache/CacheInvokeDirectResult.java          |   2 +-
 .../processors/cache/CacheInvokeResult.java     |   2 +-
 .../processors/cache/CacheLazyEntry.java        |   4 +-
 .../processors/cache/CacheMetricsImpl.java      |  10 +-
 .../processors/cache/CacheObjectAdapter.java    |   7 +-
 .../internal/processors/cache/CacheType.java    |   8 +-
 .../processors/cache/GridCacheAdapter.java      |  46 +-
 .../cache/GridCacheAffinityManager.java         |  60 +-
 .../processors/cache/GridCacheAtomicFuture.java |   3 +-
 .../processors/cache/GridCacheContext.java      |  17 -
 .../cache/GridCacheDeploymentManager.java       |   4 +-
 .../cache/GridCacheEvictionManager.java         |   6 +-
 .../cache/GridCacheMvccCandidate.java           |   9 +-
 .../GridCachePartitionExchangeManager.java      |   2 +-
 .../processors/cache/GridCacheProcessor.java    | 288 +++---
 .../processors/cache/GridCacheReturn.java       |   2 +-
 .../processors/cache/GridCacheSwapManager.java  |   2 +-
 .../processors/cache/GridCacheTtlManager.java   |   5 +-
 .../processors/cache/GridCacheUtils.java        |  35 +-
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 .../processors/cache/QueryCursorImpl.java       |   6 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |  16 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   2 +-
 .../CacheDataStructuresManager.java             |   2 +-
 .../GridDistributedCacheAdapter.java            |   9 +-
 .../distributed/GridDistributedCacheEntry.java  |   3 -
 .../GridDistributedLockResponse.java            |   1 -
 .../dht/GridClientPartitionTopology.java        |   1 -
 .../dht/GridDhtAssignmentFetchFuture.java       |   3 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  16 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   2 +-
 .../distributed/dht/GridDhtGetSingleFuture.java |   2 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   6 +-
 .../distributed/dht/GridDhtLockFuture.java      |  35 +-
 .../dht/GridDhtPartitionTopology.java           |   1 -
 .../dht/GridDhtPartitionTopologyImpl.java       |   6 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  21 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   8 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   1 -
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  11 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  13 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   2 +-
 .../distributed/dht/GridDhtUnlockRequest.java   |   3 +-
 .../dht/GridPartitionedGetFuture.java           |   3 +-
 .../dht/GridPartitionedSingleGetFuture.java     |   4 +-
 .../GridDhtAtomicAbstractUpdateFuture.java      |   2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  22 +-
 .../GridNearAtomicAbstractUpdateFuture.java     |  34 +-
 .../atomic/GridNearAtomicFullUpdateRequest.java |   1 -
 .../GridNearAtomicSingleUpdateFuture.java       |  46 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  61 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  16 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  14 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   8 +-
 .../dht/preloader/GridDhtPartitionDemander.java |  16 +-
 .../dht/preloader/GridDhtPartitionMap2.java     |   3 +-
 .../dht/preloader/GridDhtPartitionSupplier.java |  12 +-
 .../GridDhtPartitionsExchangeFuture.java        |  13 +-
 .../dht/preloader/GridDhtPreloader.java         |   4 +-
 .../distributed/near/GridNearAtomicCache.java   |   3 +-
 .../distributed/near/GridNearCacheEntry.java    |   6 +-
 .../distributed/near/GridNearGetFuture.java     |  12 +-
 .../distributed/near/GridNearGetRequest.java    |  77 +-
 .../distributed/near/GridNearLockFuture.java    |  11 +-
 .../distributed/near/GridNearLockRequest.java   |  81 +-
 ...arOptimisticSerializableTxPrepareFuture.java |   4 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |  13 +-
 ...ridNearOptimisticTxPrepareFutureAdapter.java |  13 +
 .../GridNearPessimisticTxPrepareFuture.java     |   2 +-
 .../near/GridNearSingleGetRequest.java          |  57 +-
 .../near/GridNearTransactionalCache.java        |   8 +-
 .../near/GridNearTxFinishFuture.java            |  15 +-
 .../cache/distributed/near/GridNearTxLocal.java |  17 +-
 .../processors/cache/local/GridLocalCache.java  |   1 +
 .../local/atomic/GridLocalAtomicCache.java      |   1 +
 .../processors/cache/query/CacheQuery.java      |   1 -
 .../cache/query/GridCacheQueryAdapter.java      |  12 +-
 .../cache/query/GridCacheQueryManager.java      |  26 +-
 .../cache/query/GridCacheQueryRequest.java      |   2 +
 .../cache/query/GridCacheSqlQuery.java          |   6 +-
 .../continuous/CacheContinuousQueryEvent.java   |   8 +-
 .../continuous/CacheContinuousQueryHandler.java |   4 +-
 .../continuous/CacheContinuousQueryManager.java |   6 +-
 .../store/GridCacheStoreManagerAdapter.java     |  30 +-
 .../cache/store/GridCacheWriteBehindStore.java  |   2 +-
 .../cache/transactions/IgniteTxAdapter.java     |   4 +-
 .../cache/transactions/IgniteTxEntry.java       |   2 +-
 .../cache/transactions/IgniteTxHandler.java     |   6 +-
 .../transactions/IgniteTxLocalAdapter.java      |  53 +-
 .../cache/transactions/IgniteTxLocalEx.java     |   3 +
 .../cache/transactions/IgniteTxManager.java     |   3 +-
 .../cache/transactions/IgniteTxStateImpl.java   |   1 -
 .../cache/transactions/TxDeadlockDetection.java |   2 +-
 .../cache/transactions/TxEntryValueHolder.java  |   6 +-
 .../GridCacheVersionConflictContext.java        |   2 +-
 .../IgniteCacheObjectProcessorImpl.java         |   7 +-
 .../processors/clock/GridClockServer.java       |   4 +-
 .../closure/GridClosureProcessor.java           |   6 +-
 .../processors/cluster/ClusterProcessor.java    |  53 +-
 .../continuous/GridContinuousMessage.java       |   2 +-
 .../continuous/GridContinuousProcessor.java     | 144 ++-
 .../datastreamer/DataStreamProcessor.java       |  62 +-
 .../datastreamer/DataStreamerImpl.java          |  43 +-
 .../datastructures/CollocatedSetItemKey.java    |   2 +-
 .../GridCacheAtomicLongValue.java               |   2 +
 .../GridCacheAtomicSequenceImpl.java            |   2 +
 .../GridCacheAtomicSequenceValue.java           |   2 +
 .../GridCacheCountDownLatchValue.java           |   3 +
 .../datastructures/GridCacheSetImpl.java        |   2 +-
 .../datastructures/GridCacheSetItemKey.java     |   2 +-
 .../datastructures/GridSetQueryPredicate.java   |   2 +-
 .../processors/hadoop/HadoopClasspathUtils.java |   4 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   9 +-
 .../processors/hadoop/HadoopFileBlock.java      |   1 +
 .../processors/hadoop/HadoopInputSplit.java     |  54 --
 .../internal/processors/hadoop/HadoopJob.java   | 107 ---
 .../internal/processors/hadoop/HadoopJobEx.java | 140 +++
 .../processors/hadoop/HadoopJobInfo.java        |  54 +-
 .../processors/hadoop/HadoopJobProperty.java    |   9 +-
 .../processors/hadoop/HadoopMapReducePlan.java  |  80 --
 .../hadoop/HadoopMapReducePlanner.java          |  40 -
 .../processors/hadoop/HadoopTaskContext.java    |  16 +-
 .../processors/hadoop/HadoopTaskInfo.java       |   1 +
 .../hadoop/counter/HadoopCounterWriter.java     |   4 +-
 .../shuffle/HadoopDirectShuffleMessage.java     |  34 +-
 .../processors/igfs/IgfsBaseBlockKey.java       |  42 +
 .../internal/processors/igfs/IgfsBlockKey.java  |  26 +-
 .../processors/igfs/IgfsBlockLocationImpl.java  |  55 ++
 .../processors/igfs/IgfsDataManager.java        |  12 +-
 .../internal/processors/igfs/IgfsFileImpl.java  |   5 +-
 .../internal/processors/igfs/IgfsImpl.java      |  26 +-
 .../processors/igfs/IgfsIpcHandler.java         |   4 -
 .../processors/igfs/IgfsKernalContextAware.java |  32 -
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../internal/processors/igfs/IgfsProcessor.java |   3 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |  10 +-
 .../igfs/IgfsSecondaryFileSystemV2.java         |  40 -
 .../internal/processors/igfs/IgfsUtils.java     | 105 +++
 .../igfs/client/IgfsClientSizeCallable.java     |   1 -
 .../local/LocalFileSystemBlockKey.java          | 103 +++
 .../local/LocalFileSystemIgfsFile.java          |  10 +-
 .../LocalFileSystemPositionedReadable.java      |  65 ++
 ...fsSecondaryFileSystemPositionedReadable.java |  65 --
 .../processors/job/GridJobProcessor.java        |   2 +-
 .../internal/processors/job/GridJobWorker.java  |   7 +-
 .../marshaller/ClientRequestFuture.java         | 183 ++++
 .../GridMarshallerMappingProcessor.java         | 348 +++++++
 .../processors/marshaller/MappedName.java       |  63 ++
 .../marshaller/MappingAcceptedMessage.java      |  71 ++
 .../marshaller/MappingExchangeResult.java       |  96 ++
 .../marshaller/MappingProposedMessage.java      | 137 +++
 .../marshaller/MarshallerMappingItem.java       |  99 ++
 .../marshaller/MarshallerMappingTransport.java  | 212 +++++
 .../MissingMappingRequestMessage.java           | 146 +++
 .../MissingMappingResponseMessage.java          | 169 ++++
 .../internal/processors/odbc/IgniteTypes.java   |  69 ++
 .../odbc/OdbcQueryExecuteRequest.java           |   6 +-
 .../internal/processors/odbc/OdbcTypes.java     | 131 +++
 .../internal/processors/odbc/OdbcUtils.java     |  85 ++
 .../processors/odbc/escape/OdbcEscapeUtils.java |  52 +-
 .../platform/PlatformAbstractBootstrap.java     |   3 +-
 .../platform/PlatformConfigurationEx.java       |   3 +-
 .../platform/PlatformNativeException.java       |   3 +-
 .../platform/PlatformNoopProcessor.java         |   5 +
 .../platform/PlatformPluginExtension.java       |  39 +
 .../processors/platform/PlatformProcessor.java  |   7 +
 .../platform/PlatformProcessorImpl.java         |  62 +-
 .../platform/cache/PlatformCache.java           |  23 +-
 .../callback/PlatformCallbackGateway.java       |  96 ++
 .../platform/callback/PlatformCallbackOp.java   |  18 +
 .../plugin/PlatformPluginProcessor.java         |  46 +
 .../cache/PlatformCachePluginConfiguration.java |  58 ++
 .../cache/PlatformCachePluginProvider.java      | 123 +++
 .../transactions/PlatformTransactions.java      |   9 +
 .../utils/PlatformConfigurationUtils.java       | 158 +++-
 .../platform/utils/PlatformUtils.java           |  58 +-
 .../processors/plugin/CachePluginManager.java   |   8 +-
 .../plugin/IgnitePluginProcessor.java           |  84 +-
 .../internal/processors/pool/PoolProcessor.java |  15 +-
 .../processors/query/GridQueryProcessor.java    |  39 +-
 .../processors/resource/GridResourceIoc.java    |   6 +-
 .../resource/GridResourceProcessor.java         |  32 +-
 .../processors/rest/GridRestResponse.java       |   2 +-
 .../handlers/cache/GridCacheCommandHandler.java |   2 -
 .../string/GridRedisSetCommandHandler.java      |   2 +-
 .../service/GridServiceProcessor.java           |   9 +-
 .../processors/task/GridTaskWorker.java         |   2 +-
 .../internal/util/GridStripedSpinBusyLock.java  |   2 +-
 .../ignite/internal/util/HostAndPortRange.java  |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |  73 +-
 .../ignite/internal/util/StripedExecutor.java   |   2 +-
 .../internal/util/future/GridFutureAdapter.java |   2 +-
 .../util/lang/GridMetadataAwareAdapter.java     |   2 +-
 .../ignite/internal/util/nio/GridNioServer.java | 159 +++-
 .../util/tostring/GridToStringBuilder.java      | 656 ++++++++++++--
 .../util/tostring/GridToStringInclude.java      |  12 +-
 .../util/tostring/GridToStringThreadLocal.java  |  12 +-
 .../visor/node/VisorIgfsConfiguration.java      |   1 -
 .../query/VisorQueryScanSubstringFilter.java    |   5 +-
 .../internal/visor/query/VisorQueryUtils.java   |  60 ++
 .../ignite/marshaller/MarshallerContext.java    |  23 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |   1 -
 .../optimized/OptimizedClassDescriptor.java     |   2 +-
 .../optimized/OptimizedMarshallerUtils.java     |  16 +-
 .../ignite/resources/FileSystemResource.java    |  62 ++
 .../communication/tcp/TcpCommunicationSpi.java  |  22 +-
 .../tcp/TcpCommunicationSpiMBean.java           |   6 +-
 .../ignite/spi/discovery/DiscoveryDataBag.java  | 299 ++++++
 .../spi/discovery/DiscoverySpiDataExchange.java |  15 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  30 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 105 +--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  71 +-
 .../tcp/internal/DiscoveryDataPacket.java       | 345 +++++++
 .../tcp/internal/TcpDiscoveryStatistics.java    |   8 +-
 .../TcpDiscoveryJoinRequestMessage.java         |  18 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |  10 +-
 .../messages/TcpDiscoveryNodeAddedMessage.java  |  78 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   3 +
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  16 +-
 .../resources/META-INF/classnames.properties    |   5 +-
 .../org.apache.ignite.plugin.PluginProvider     |   1 +
 .../CacheJdbcPojoStoreAbstractSelfTest.java     |  21 +-
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |  48 +-
 ...eJdbcStoreAbstractMultithreadedSelfTest.java |  17 +-
 .../ignite/cache/store/jdbc/model/Gender.java   |  41 +
 .../ignite/cache/store/jdbc/model/Person.java   |  33 +-
 .../internal/ClusterGroupHostsSelfTest.java     |   6 +-
 .../ComputeJobCancelWithServiceSelfTest.java    |   2 +-
 .../ignite/internal/GridAffinitySelfTest.java   |   2 +-
 .../internal/GridContinuousTaskSelfTest.java    |   4 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |   2 +-
 .../IgniteClientReconnectApiExceptionTest.java  |   6 +-
 ...niteClientReconnectFailoverAbstractTest.java |   4 +-
 .../MarshallerContextLockingSelfTest.java       |  61 +-
 .../internal/binary/BinaryEnumsSelfTest.java    |  18 +
 .../binary/BinaryMarshallerSelfTest.java        |  20 +-
 ...GridBinaryMarshallerCtxDisabledSelfTest.java |  42 +-
 ...mmunicationBalancePairedConnectionsTest.java |  28 +
 .../IgniteCommunicationBalanceTest.java         |  25 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   3 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |   4 +-
 .../BinaryObjectOffHeapUnswapTemporaryTest.java |   1 -
 .../cache/CacheAffinityCallSelfTest.java        |   8 +-
 .../cache/CacheEntryProcessorCopySelfTest.java  |   6 +-
 .../cache/CacheGetEntryAbstractTest.java        |   2 +-
 .../CacheSerializableTransactionsTest.java      |   6 +-
 .../CacheStartupInDeploymentModesTest.java      |   7 +-
 .../cache/CacheStopAndDestroySelfTest.java      |   6 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   6 +-
 .../cache/GridCacheBasicStoreAbstractTest.java  |   2 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java |   8 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java |  48 +-
 .../processors/cache/GridCacheMvccSelfTest.java | 208 ++---
 .../processors/cache/GridCacheStopSelfTest.java |   4 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   3 +-
 .../cache/GridCacheUtilsSelfTest.java           |   2 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java |   8 +-
 .../IgniteCacheConfigVariationsFullApiTest.java |   8 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |   8 +-
 .../cache/IgniteCacheStoreCollectionTest.java   |   1 -
 .../cache/IgniteInternalCacheTypesTest.java     |  11 -
 ...iteMarshallerCacheClassNameConflictTest.java | 273 ++++++
 ...lerCacheClientRequestsMappingOnMissTest.java | 345 +++++++
 .../cache/IgniteSystemCacheOnClientTest.java    |  23 +-
 ...gniteTopologyValidatorAbstractCacheTest.java |   6 +-
 .../processors/cache/IgniteTxAbstractTest.java  |   3 +-
 .../cache/IgniteTxConfigCacheSelfTest.java      |   2 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |   3 +-
 .../IgniteTxMultiThreadedAbstractTest.java      |   4 +-
 .../CacheKeepBinaryWithInterceptorTest.java     |   1 -
 .../GridBinaryCacheEntryMemorySizeSelfTest.java |  21 +-
 ...yAtomicEntryProcessorDeploymentSelfTest.java |   4 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   7 +-
 ...IgniteCacheAbstractExecutionContextTest.java |   2 -
 ...eAbstractDataStructuresFailoverSelfTest.java |  16 +-
 ...actQueueFailoverDataConsistencySelfTest.java |   2 +-
 .../IgniteLockAbstractSelfTest.java             |  24 +-
 .../CacheGetFutureHangsSelfTest.java            |   6 +-
 .../CacheLateAffinityAssignmentTest.java        |   2 +-
 ...ridCachePartitionNotLoadedEventSelfTest.java |   4 +-
 ...cMessageRecoveryNoPairedConnectionsTest.java |  47 -
 ...micMessageRecoveryPairedConnectionsTest.java |  47 +
 ...niteCacheClientNodeChangingTopologyTest.java |  20 +-
 .../IgniteCacheSystemTransactionsSelfTest.java  |  18 +-
 ...teSynchronizationModesMultithreadedTest.java |   8 +-
 .../GridCacheDhtEvictionsDisabledSelfTest.java  |   5 +-
 .../IgniteCacheCrossCacheTxFailoverTest.java    |   6 +-
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |   4 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |   4 +-
 .../AtomicPutAllChangingTopologyTest.java       | 212 +++++
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   4 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   4 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   8 +-
 .../GridCacheReplicatedNodeRestartSelfTest.java |   3 +-
 .../GridCacheReplicatedPreloadSelfTest.java     |   1 -
 .../IgniteCacheExpiryPolicyAbstractTest.java    |   4 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java | 100 ++-
 .../IgniteCacheTxExpiryPolicyWithStoreTest.java |  21 +
 .../IgniteCacheQueryCacheDestroySelfTest.java   | 142 +++
 .../cache/query/IndexingSpiQuerySelfTest.java   | 199 +++-
 ...ContinuousQueryFailoverAbstractSelfTest.java |   6 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   2 +-
 ...dCacheContinuousQueryNodesFilteringTest.java |   2 +-
 ...IgniteCacheContinuousQueryReconnectTest.java |  12 +-
 .../TxDeadlockDetectionNoHangsTest.java         |   2 +-
 .../transactions/TxDeadlockDetectionTest.java   |   2 +-
 .../TxOptimisticDeadlockDetectionTest.java      |   2 +-
 .../TxPessimisticDeadlockDetectionTest.java     |   2 +-
 ...ComputeJobExecutionErrorToLogManualTest.java |   4 +-
 .../DataStreamProcessorSelfTest.java            | 104 +++
 .../datastreamer/DataStreamerImplSelfTest.java  |   8 +-
 .../datastreamer/DataStreamerTimeoutTest.java   |   6 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  50 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |  26 +-
 .../processors/igfs/IgfsFileInfoSelfTest.java   |   4 +-
 ...SecondaryFileSystemDualAbstractSelfTest.java |   5 -
 ...fsLocalSecondaryFileSystemProxySelfTest.java |  81 ++
 ...IgfsLocalSecondaryFileSystemTestAdapter.java |   7 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |   3 +-
 .../igfs/IgfsModeResolverSelfTest.java          |  10 +-
 ...gfsSecondaryFileSystemInjectionSelfTest.java | 270 ++++++
 .../processors/igfs/IgfsTaskSelfTest.java       |  87 +-
 .../igfs/benchmark/IgfsBenchmark.java           |   4 +-
 .../odbc/OdbcEscapeSequenceSelfTest.java        | 131 +++
 .../cache/GridCacheCommandHandlerSelfTest.java  |   1 -
 ...rviceDeploymentExceptionPropagationTest.java |   2 +-
 .../GridServiceProcessorStopSelfTest.java       |   2 +-
 .../service/IgniteServiceReassignmentTest.java  |   2 +-
 .../internal/util/IgniteUtilsSelfTest.java      |  71 +-
 .../unsafe/GridOffheapSnapTreeSelfTest.java     |   7 +-
 .../tostring/GridToStringBuilderSelfTest.java   |  33 +-
 .../lang/GridMetadataAwareAdapterSelfTest.java  |   4 +-
 .../marshaller/MarshallerContextSelfTest.java   | 144 ++-
 .../marshaller/MarshallerContextTestImpl.java   |  34 +-
 .../OptimizedMarshallerEnumSelfTest.java        |  25 +-
 .../p2p/GridP2PSameClassLoaderSelfTest.java     |  14 +-
 .../platform/PlatformComputeEchoTask.java       |  12 +
 .../platform/PlatformEventsWriteEventTask.java  |   9 -
 .../ignite/platform/PlatformExceptionTask.java  |  78 ++
 .../platform/plugin/PlatformTestPlugin.java     |  27 +
 .../plugin/PlatformTestPluginExtension.java     |  51 ++
 .../plugin/PlatformTestPluginProvider.java      |  97 ++
 .../plugin/PlatformTestPluginTarget.java        |  90 ++
 .../local/GridLocalDeploymentSpiSelfTest.java   |   6 +-
 .../discovery/AbstractDiscoverySelfTest.java    |   7 +-
 ...lientDiscoverySpiFailureTimeoutSelfTest.java |   4 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 111 ++-
 .../TcpDiscoverySpiFailureTimeoutSelfTest.java  |  14 +-
 .../tcp/TcpDiscoverySpiStartStopSelfTest.java   |  11 +-
 .../GridSwapSpaceSpiAbstractSelfTest.java       |   3 +-
 .../testframework/GridSpiTestContext.java       |   5 +-
 .../ignite/testframework/IgniteTestSuite.java   |   2 +-
 .../testframework/junits/GridAbstractTest.java  |  13 +-
 .../junits/GridTestKernalContext.java           |  12 +-
 .../junits/spi/GridSpiAbstractTest.java         |   9 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |   5 +
 .../IgniteCacheDataStructuresSelfTestSuite.java |   2 -
 .../IgniteCacheFailoverTestSuite.java           |   3 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   6 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   3 +
 ...essagingConfigVariationFullApiTestSuite.java |   4 +-
 .../ignite/thread/IgniteThreadPoolSizeTest.java |   8 -
 .../ignite/util/mbeans/GridMBeanSelfTest.java   |  10 +-
 .../CacheNoValueClassOnServerTestClient.java    |   3 +-
 .../CacheConfigurationP2PTestClient.java        |   3 +-
 .../apache/ignite/stream/flume/IgniteSink.java  |   7 +-
 modules/gce/pom.xml                             |   4 +-
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |   4 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   6 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  23 +-
 .../io/BytesWritablePartiallyRawComparator.java |  51 ++
 .../hadoop/io/TextPartiallyRawComparator.java   |  68 +-
 .../mapreduce/IgniteHadoopMapReducePlanner.java | 416 ---------
 .../IgniteHadoopWeightedMapReducePlanner.java   |  15 +-
 .../planner/HadoopAbstractMapReducePlanner.java | 118 +++
 .../planner/HadoopTestRoundRobinMrPlanner.java  |  75 ++
 .../processors/hadoop/HadoopCommonUtils.java    |   1 +
 .../processors/hadoop/HadoopContext.java        |   2 +
 .../processors/hadoop/HadoopExternalSplit.java  |   1 +
 .../processors/hadoop/HadoopProcessor.java      |   4 +-
 .../processors/hadoop/HadoopSplitWrapper.java   |   1 +
 .../HadoopFileSystemCounterWriterDelegate.java  |   4 +-
 .../HadoopIgfsSecondaryFileSystemDelegate.java  |   4 +-
 .../hadoop/igfs/HadoopIgfsEndpoint.java         |  42 +-
 .../processors/hadoop/impl/HadoopUtils.java     |  66 ++
 .../HadoopBasicFileSystemFactoryDelegate.java   |   2 +-
 ...doopFileSystemCounterWriterDelegateImpl.java |   4 +-
 ...doopIgfsSecondaryFileSystemDelegateImpl.java |  61 +-
 .../hadoop/impl/igfs/HadoopIgfsInProc.java      |  40 +-
 .../hadoop/impl/igfs/HadoopIgfsJclLogger.java   |   9 +-
 .../hadoop/impl/igfs/HadoopIgfsOutProc.java     |  24 +-
 .../hadoop/impl/igfs/HadoopIgfsWrapper.java     |  33 +-
 .../hadoop/impl/v1/HadoopV1MapTask.java         |  95 +-
 .../hadoop/impl/v1/HadoopV1ReduceTask.java      |  73 +-
 .../hadoop/impl/v1/HadoopV1Splitter.java        |   2 +-
 .../hadoop/impl/v2/HadoopV2Context.java         |  14 +-
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |  15 +-
 .../hadoop/impl/v2/HadoopV2MapTask.java         |  18 +-
 .../hadoop/impl/v2/HadoopV2ReduceTask.java      |  14 +
 .../hadoop/impl/v2/HadoopV2Splitter.java        |   2 +-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |  85 +-
 .../hadoop/jobtracker/HadoopJobMetadata.java    |   4 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |  32 +-
 .../planner/HadoopAbstractMapReducePlanner.java | 116 ---
 .../planner/HadoopDefaultMapReducePlan.java     |   4 +-
 .../hadoop/shuffle/HadoopShuffle.java           |   4 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |  73 +-
 .../shuffle/direct/HadoopDirectDataInput.java   |  77 +-
 .../shuffle/direct/HadoopDirectDataOutput.java  |  14 +
 .../direct/HadoopDirectDataOutputContext.java   |  48 +-
 .../direct/HadoopDirectDataOutputState.java     |  14 +-
 .../shuffle/streams/HadoopDataInStream.java     |  34 +-
 .../shuffle/streams/HadoopOffheapBuffer.java    |  18 +
 .../HadoopEmbeddedTaskExecutor.java             |   8 +-
 .../taskexecutor/HadoopExecutorService.java     |   6 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  18 +-
 .../taskexecutor/HadoopTaskExecutorAdapter.java |   8 +-
 .../external/HadoopExternalTaskExecutor.java    |  16 +-
 .../child/HadoopChildProcessRunner.java         |  14 +-
 .../resources/META-INF/classnames.properties    |   4 +-
 .../impl/HadoopAbstractMapReduceTest.java       |   4 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |   3 +-
 .../hadoop/impl/HadoopCommandLineTest.java      |   4 +-
 .../HadoopDefaultMapReducePlannerSelfTest.java  | 619 -------------
 .../hadoop/impl/HadoopErrorSimulator.java       |  12 +-
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |   1 +
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |   6 +-
 .../hadoop/impl/HadoopPlannerMockJob.java       |  10 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |  19 +-
 .../hadoop/impl/HadoopTasksAllVersionsTest.java |  16 +-
 .../hadoop/impl/HadoopTasksV1Test.java          |   4 +-
 .../hadoop/impl/HadoopTasksV2Test.java          |   4 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |  32 +-
 .../impl/HadoopTestRoundRobinMrPlanner.java     |  75 --
 .../hadoop/impl/HadoopTestTaskContext.java      |   6 +-
 .../hadoop/impl/HadoopV2JobSelfTest.java        |   6 +-
 .../HadoopWeightedMapReducePlannerTest.java     |   6 +-
 .../client/HadoopClientProtocolSelfTest.java    |   2 +-
 .../impl/igfs/Hadoop1DualAbstractTest.java      |   7 +-
 .../impl/igfs/Hadoop1OverIgfsProxyTest.java     |  67 ++
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   6 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   3 +-
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |   2 +-
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |   2 +-
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   6 +-
 ...oopSecondaryFileSystemConfigurationTest.java |   4 +-
 .../hadoop/impl/igfs/IgfsEventsTestSuite.java   |   4 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   4 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |   8 +-
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |  32 +-
 ...niteHadoopFileSystemLoggerStateSelfTest.java |   6 +-
 ...condaryFileSystemInitializationSelfTest.java |   4 +-
 .../collections/HadoopAbstractMapTest.java      |   4 +-
 .../streams/HadoopDataStreamSelfTest.java       | 177 +++-
 .../testsuites/IgniteHadoopTestSuite.java       |   4 +-
 .../query/h2/GridH2ResultSetIterator.java       |   3 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   6 +-
 .../query/h2/opt/GridH2IndexBase.java           |   6 +-
 .../processors/query/h2/sql/DmlAstUtils.java    |   2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   8 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   7 +-
 .../query/h2/twostep/msg/GridH2Geometry.java    |   2 +-
 .../cache/BinarySerializationQuerySelfTest.java |   2 -
 .../CacheRandomOperationsMultithreadedTest.java |   4 +-
 .../ClientReconnectAfterClusterRestartTest.java |   4 +-
 .../GridCacheQueryIndexDisabledSelfTest.java    |  20 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |  14 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |   3 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |   6 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  18 +-
 ...eLockPartitionOnAffinityRunAbstractTest.java |   1 -
 ...PartitionOnAffinityRunAtomicCacheOpTest.java |  11 +-
 ...niteCacheLockPartitionOnAffinityRunTest.java |  51 +-
 ...titionOnAffinityRunWithCollisionSpiTest.java |  10 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  15 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java |   4 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   4 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |   4 +-
 ...niteCacheDistributedQueryCancelSelfTest.java |   4 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   4 +-
 .../query/IgniteSqlQueryDedicatedPoolTest.java  | 110 +++
 .../query/IgniteSqlSchemaIndexingTest.java      |  15 +-
 .../query/IgniteSqlSplitterSelfTest.java        |   4 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   4 +-
 .../query/h2/sql/GridQueryParsingTest.java      |   4 -
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +
 .../org/apache/ignite/logger/jcl/JclLogger.java |   9 +-
 .../apache/ignite/stream/jms11/JmsStreamer.java |   4 +-
 .../ignite/stream/kafka/KafkaStreamer.java      |   6 +-
 .../ignite/stream/kafka/TestKafkaBroker.java    |   2 +-
 .../connect/IgniteSourceConnectorTest.java      |   2 +-
 .../stream/mqtt/IgniteMqttStreamerTest.java     |   4 +-
 .../BundleDelegatingClassLoader.java            |   4 +-
 .../classloaders/ContainerSweepClassLoader.java |   4 +-
 modules/platforms/cpp/binary/Makefile.am        |   5 +-
 .../platforms/cpp/binary/include/Makefile.am    |   6 +
 .../cpp/binary/include/ignite/binary/binary.h   |  15 +-
 .../binary/binary_array_identity_resolver.h     |  64 ++
 .../ignite/binary/binary_identity_resolver.h    |  61 ++
 .../include/ignite/binary/binary_object.h       |  78 ++
 .../binary/include/ignite/binary/binary_type.h  |  16 -
 .../ignite/impl/binary/binary_object_header.h   | 250 ++++++
 .../ignite/impl/binary/binary_object_impl.h     | 109 +++
 .../include/ignite/impl/binary/binary_schema.h  |   2 +-
 .../ignite/impl/binary/binary_type_impl.h       | 149 +++
 .../ignite/impl/binary/binary_type_manager.h    |   2 +-
 .../ignite/impl/binary/binary_type_snapshot.h   |  18 +-
 .../include/ignite/impl/binary/binary_utils.h   | 236 ++---
 .../ignite/impl/binary/binary_writer_impl.h     |  11 +-
 .../ignite/impl/interop/interop_output_stream.h |  10 +-
 .../cpp/binary/project/vs/binary.vcxproj        |   9 +
 .../binary/project/vs/binary.vcxproj.filters    |  27 +
 .../binary/binary_array_identity_resolver.cpp   |  42 +
 .../cpp/binary/src/binary/binary_type.cpp       |   4 +-
 .../src/impl/binary/binary_object_header.cpp    |  51 ++
 .../src/impl/binary/binary_object_impl.cpp      |  56 ++
 .../src/impl/binary/binary_type_manager.cpp     |  16 +-
 .../cpp/binary/src/impl/binary/binary_utils.cpp | 169 ++--
 .../src/impl/binary/binary_writer_impl.cpp      |   4 +-
 .../binary/src/impl/interop/interop_memory.cpp  |   2 +-
 .../src/impl/interop/interop_output_stream.cpp  |   5 +
 modules/platforms/cpp/common/Makefile.am        |   7 +-
 .../platforms/cpp/common/include/Makefile.am    |   1 +
 .../include/ignite/common/dynamic_size_array.h  |   2 +-
 .../include/ignite/common/platform_utils.h      |  79 ++
 .../cpp/common/include/ignite/common/utils.h    | 249 +++--
 .../os/linux/src/common/platform_utils.cpp      |  81 ++
 .../cpp/common/os/linux/src/common/utils.cpp    | 136 ---
 .../common/os/win/src/common/platform_utils.cpp |  88 ++
 .../cpp/common/os/win/src/common/utils.cpp      | 142 ---
 .../cpp/common/project/vs/common.vcxproj        |   5 +-
 .../common/project/vs/common.vcxproj.filters    |  12 +-
 .../platforms/cpp/common/src/common/utils.cpp   | 153 ++++
 modules/platforms/cpp/configure.ac              |   2 +-
 modules/platforms/cpp/configure.acrel           |   2 +-
 modules/platforms/cpp/core-test/Makefile.am     |   7 +-
 .../cpp/core-test/config/cache-identity.xml     | 121 +++
 .../cpp/core-test/config/cache-query.xml        |  10 -
 .../core-test/include/ignite/binary_test_defs.h |  25 +
 .../cpp/core-test/include/ignite/complex_type.h | 135 +++
 .../cpp/core-test/include/ignite/test_type.h    | 186 ++++
 .../cpp/core-test/include/ignite/test_utils.h   |  63 ++
 .../cpp/core-test/project/vs/core-test.vcxproj  |   7 +
 .../project/vs/core-test.vcxproj.filters        |  29 +-
 .../src/binary_identity_resolver_test.cpp       | 522 +++++++++++
 .../cpp/core-test/src/binary_object_test.cpp    | 282 ++++++
 .../cpp/core-test/src/binary_session_test.cpp   |   4 +-
 .../cpp/core-test/src/cache_query_test.cpp      | 157 ++--
 .../platforms/cpp/core-test/src/cache_test.cpp  |  43 +-
 .../cpp/core-test/src/continuous_query_test.cpp |  31 +-
 .../cpp/core-test/src/ignition_test.cpp         |  19 +-
 .../cpp/core-test/src/interop_test.cpp          |  42 +-
 .../cpp/core-test/src/reference_test.cpp        |  14 +-
 .../platforms/cpp/core-test/src/test_utils.cpp  |  79 ++
 .../cpp/core-test/src/transactions_test.cpp     |  23 +-
 modules/platforms/cpp/core/Makefile.am          |   2 +-
 .../cache/query/continuous/continuous_query.h   |   8 +-
 .../include/ignite/cache/query/query_cursor.h   |   4 +-
 .../query/continuous/continuous_query_impl.h    |   8 +-
 .../core/include/ignite/impl/handle_registry.h  |   2 +-
 .../cpp/core/include/ignite/impl/operations.h   |   4 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   5 +-
 .../cpp/core/src/impl/handle_registry.cpp       |   2 +-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |   4 +-
 modules/platforms/cpp/examples/configure.ac     |   2 +-
 .../cpp/examples/odbc-example/Makefile.am       |   2 +-
 .../cpp/examples/putget-example/Makefile.am     |   2 +-
 .../cpp/examples/query-example/Makefile.am      |   2 +-
 modules/platforms/cpp/ignite/Makefile.am        |   2 +-
 modules/platforms/cpp/jni/Makefile.am           |   2 +-
 .../cpp/jni/include/ignite/jni/exports.h        |   1 +
 .../platforms/cpp/jni/include/ignite/jni/java.h |   2 +
 .../platforms/cpp/jni/project/vs/jni.vcxproj    |   1 +
 modules/platforms/cpp/jni/project/vs/module.def |   1 +
 modules/platforms/cpp/jni/src/exports.cpp       |   4 +
 modules/platforms/cpp/jni/src/java.cpp          |  15 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |   3 +-
 .../cpp/odbc-test/config/queries-default.xml    |   1 +
 .../odbc-test/include/sql_test_suite_fixture.h  |  16 +
 .../platforms/cpp/odbc-test/include/test_type.h |  20 +-
 .../cpp/odbc-test/include/test_utils.h          |  51 +-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   1 +
 .../project/vs/odbc-test.vcxproj.filters        |   3 +
 .../cpp/odbc-test/src/api_robustness_test.cpp   | 150 +++-
 .../src/application_data_buffer_test.cpp        |  24 +-
 .../platforms/cpp/odbc-test/src/column_test.cpp | 189 ++++
 .../cpp/odbc-test/src/queries_test.cpp          |  71 +-
 .../src/sql_aggregate_functions_test.cpp        |   5 +-
 .../src/sql_date_time_functions_test.cpp        |  26 +-
 .../src/sql_esc_convert_function_test.cpp       | 160 ++++
 .../src/sql_numeric_functions_test.cpp          |   1 +
 .../cpp/odbc-test/src/sql_outer_join_test.cpp   |   1 +
 .../odbc-test/src/sql_string_functions_test.cpp |   1 +
 .../odbc-test/src/sql_test_suite_fixture.cpp    | 105 ++-
 .../cpp/odbc-test/src/sql_types_test.cpp        | 256 ++++++
 .../platforms/cpp/odbc-test/src/test_utils.cpp  |  60 +-
 modules/platforms/cpp/odbc/Makefile.am          |   5 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |   3 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |  29 +-
 .../ignite/odbc/config/connection_info.h        |   2 -
 .../cpp/odbc/include/ignite/odbc/log.h          | 123 +++
 .../odbc/include/ignite/odbc/meta/column_meta.h |   2 -
 .../cpp/odbc/include/ignite/odbc/statement.h    | 143 ++-
 .../cpp/odbc/include/ignite/odbc/utility.h      |  26 +-
 .../cpp/odbc/install/ignite-odbc-amd64.wxs      |   2 +-
 .../cpp/odbc/install/ignite-odbc-x86.wxs        |   2 +-
 .../odbc/os/linux/src/system/socket_client.cpp  |   9 +-
 .../odbc/os/win/src/system/socket_client.cpp    |   9 +-
 .../src/system/ui/dsn_configuration_window.cpp  |  18 +-
 .../cpp/odbc/os/win/src/system_dsn.cpp          |  17 +-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |   2 +
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |   6 +
 .../odbc/src/app/application_data_buffer.cpp    |  66 +-
 .../platforms/cpp/odbc/src/app/parameter.cpp    |   4 +-
 modules/platforms/cpp/odbc/src/column.cpp       |  28 +-
 .../cpp/odbc/src/config/connection_info.cpp     | 267 +++++-
 modules/platforms/cpp/odbc/src/connection.cpp   |  21 +-
 .../odbc/src/diagnostic/diagnosable_adapter.cpp |   3 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |  38 +
 modules/platforms/cpp/odbc/src/entry_points.cpp |  52 +-
 modules/platforms/cpp/odbc/src/log.cpp          |  83 ++
 .../platforms/cpp/odbc/src/meta/column_meta.cpp |   4 -
 modules/platforms/cpp/odbc/src/odbc.cpp         | 286 +++---
 .../odbc/src/query/column_metadata_query.cpp    |  13 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |  23 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |  12 +-
 modules/platforms/cpp/odbc/src/statement.cpp    | 187 +++-
 modules/platforms/cpp/odbc/src/utility.cpp      |  29 +-
 .../Apache.Ignite.AspNet.Tests.csproj           |   7 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.AspNet.csproj                 |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Benchmarks.csproj             |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Core.Tests.csproj             |  19 +
 .../Binary/BinaryBuilderSelfTest.cs             | 179 ++--
 .../BinaryBuilderSelfTestArrayIdentity.cs       |  34 +
 .../Binary/BinaryEqualityComparerTest.cs        | 279 ++++++
 .../Binary/BinaryStructureTest.cs               | 122 +++
 .../Binary/IO/BinaryStreamsTest.cs              |  19 +
 .../Cache/CacheAbstractTest.cs                  | 648 +------------
 .../Cache/CacheAbstractTransactionalTest.cs     | 899 +++++++++++++++++++
 .../Cache/CacheConfigurationTest.cs             |  16 +-
 .../Cache/CacheLocalAtomicTest.cs               |   5 -
 .../Cache/CacheLocalTest.cs                     |   6 +-
 .../CachePartitionedAtomicNearEnabledTest.cs    |   5 -
 .../Cache/CachePartitionedAtomicTest.cs         |   5 -
 .../Cache/CachePartitionedNearEnabledTest.cs    |   7 +-
 .../Cache/CachePartitionedTest.cs               |   7 +-
 .../Cache/CacheReplicatedAtomicTest.cs          |   5 -
 .../Cache/CacheReplicatedTest.cs                |   7 +-
 .../Cache/Query/CacheDmlQueriesTest.cs          | 296 ++++++
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   4 +-
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |   9 +-
 .../Cache/Store/CacheStoreSessionTest.cs        |  22 +-
 .../Cache/Store/CacheStoreTest.cs               | 333 ++++---
 .../Cache/Store/CacheTestStore.cs               |  14 +
 .../Cache/Store/NamedNodeCacheStoreTest.cs      |  34 +
 .../Compute/ComputeApiTest.cs                   |  48 +
 .../Examples/Example.cs                         |   5 +-
 .../Examples/ExamplesTest.cs                    | 107 ++-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  38 +
 .../IgniteConfigurationSerializerTest.cs        |  71 +-
 .../IgniteConfigurationTest.cs                  |  39 +-
 .../Log/ConcurrentMemoryTarget.cs               |  73 ++
 .../Log/NLogLoggerTest.cs                       |   5 +-
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |   6 +-
 .../Plugin/Cache/CachePlugin.cs                 | 127 +++
 .../Plugin/Cache/CachePluginConfiguration.cs    |  40 +
 .../Plugin/Cache/CachePluginTest.cs             | 215 +++++
 .../Plugin/PluginTest.cs                        | 254 ++++++
 .../Plugin/TestIgnitePlugin.cs                  |  39 +
 .../Plugin/TestIgnitePluginConfiguration.cs     |  33 +
 .../Plugin/TestIgnitePluginProvider.cs          | 132 +++
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |  26 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   1 -
 .../Apache.Ignite.Core.csproj                   |  41 +-
 .../Binary/BinaryArrayEqualityComparer.cs       | 149 +++
 .../Binary/BinaryConfiguration.cs               |  24 +
 .../Binary/BinaryTypeConfiguration.cs           |  14 +
 .../Cache/Configuration/CacheConfiguration.cs   |  34 +
 .../Cache/Configuration/QueryEntity.cs          |  33 +-
 .../Cache/Configuration/QueryField.cs           |   6 +
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |   4 +-
 .../Communication/Tcp/TcpCommunicationSpi.cs    |   2 +-
 .../Discovery/Tcp/TcpDiscoverySpi.cs            |   2 +-
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |  10 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   | 107 ++-
 .../IgniteConfigurationSection.xsd              |  61 ++
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |  28 +-
 .../Binary/BinaryEqualityComparerSerializer.cs  |  99 ++
 .../Impl/Binary/BinaryFieldEqualityComparer.cs  | 138 +++
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |  21 +-
 .../Impl/Binary/BinaryObject.cs                 |  31 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |  64 +-
 .../Impl/Binary/BinaryObjectHeader.cs           |  21 +-
 .../Impl/Binary/BinaryObjectSchemaHolder.cs     |  22 +
 .../Impl/Binary/BinaryReader.cs                 | 153 ++--
 .../Binary/BinarySurrogateTypeDescriptor.cs     |   6 +
 .../Impl/Binary/BinarySystemHandlers.cs         |  20 +-
 .../Impl/Binary/BinaryWriter.cs                 |  17 +-
 .../Impl/Binary/DateTimeHolder.cs               |  35 +-
 .../Impl/Binary/IBinaryEqualityComparer.cs      |  53 ++
 .../Impl/Binary/IBinaryTypeDescriptor.cs        |   5 +
 .../Impl/Binary/Io/BinaryHeapStream.cs          |   9 +
 .../Impl/Binary/Io/BinaryStreamBase.cs          |  13 +
 .../Impl/Binary/Io/IBinaryStream.cs             |  11 +-
 .../Impl/Binary/Io/IBinaryStreamProcessor.cs    |  36 +
 .../Impl/Binary/Marshaller.cs                   |  22 +-
 .../Impl/Binary/SerializableObjectHolder.cs     |  16 +
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  | 107 ++-
 .../Common/IgniteConfigurationXmlSerializer.cs  |   8 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   6 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  26 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |   6 +
 .../Impl/Memory/PlatformMemoryStream.cs         |  16 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  70 +-
 .../Impl/Plugin/Cache/CachePluginContext.cs     |  82 ++
 .../Impl/Plugin/Cache/CachePluginProcessor.cs   |  77 ++
 .../Plugin/Cache/CachePluginProviderProxy.cs    |  75 ++
 .../Plugin/Cache/ICachePluginProviderProxy.cs   |  52 ++
 .../Impl/Plugin/IPluginProviderProxy.cs         |  74 ++
 .../Impl/Plugin/PluginContext.cs                |  72 ++
 .../Impl/Plugin/PluginProcessor.cs              | 235 +++++
 .../Impl/Plugin/PluginProviderProxy.cs          |  90 ++
 .../Transactions/CacheTransactionManager.cs     | 161 ++++
 .../Impl/Transactions/Transaction.cs            |   8 +
 .../Impl/Transactions/TransactionImpl.cs        |  13 +
 .../Impl/Transactions/TransactionsImpl.cs       |  28 +
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |   3 +
 .../Impl/Unmanaged/UnmanagedCallbackOp.cs       |   8 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |  83 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  11 +-
 .../Interop/IPlatformTarget.cs                  |  91 ++
 .../Cache/CachePluginProviderTypeAttribute.cs   |  52 ++
 .../Plugin/Cache/ICachePluginConfiguration.cs   |  50 ++
 .../Plugin/Cache/ICachePluginContext.cs         |  47 +
 .../Plugin/Cache/ICachePluginProvider.cs        |  52 ++
 .../Plugin/IPluginConfiguration.cs              |  48 +
 .../Apache.Ignite.Core/Plugin/IPluginContext.cs |  52 ++
 .../Plugin/IPluginProvider.cs                   |  69 ++
 .../Plugin/PluginNotFoundException.cs           |  65 ++
 .../Plugin/PluginProviderTypeAttribute.cs       |  52 ++
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Transactions/ITransactions.cs               |  19 +-
 .../Apache.Ignite.EntityFramework.Tests.csproj  |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.EntityFramework.csproj        |   9 +-
 .../Impl/ArrayDbDataReader.cs                   |   5 +
 .../Impl/DataReaderResult.cs                    |   2 +
 .../Impl/DbCommandInfo.cs                       |  11 +
 .../Impl/DbCommandProxy.cs                      |   1 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Linq.csproj                   |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Log4Net.csproj                |   5 +
 .../IgniteLog4NetLogger.cs                      |  16 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.NLog.csproj                   |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 modules/platforms/dotnet/Apache.Ignite.sln      |   3 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |   5 +
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   4 +-
 modules/platforms/dotnet/README.md              |   6 +-
 modules/platforms/dotnet/build.ps1              |   2 +-
 .../Apache.Ignite.Examples.csproj               |   4 +
 .../examples/Apache.Ignite.Examples/App.config  |   2 +
 .../DataStructures/AtomicSequenceExample.cs     |  26 +-
 .../Datagrid/QueryDmlExample.cs                 | 162 ++++
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.ExamplesDll.csproj            |   4 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Services/MapService.cs                      |   2 -
 .../apache/ignite/logger/slf4j/Slf4jLogger.java |  11 +-
 .../spark/JavaStandaloneIgniteRDDSelfTest.java  |   3 +-
 .../util/spring/IgniteSpringHelperImpl.java     |   8 +-
 .../src/test/config/jdbc-pojo-store-builtin.xml |   8 +
 .../src/test/config/jdbc-pojo-store-obj.xml     |   8 +
 .../ignite/internal/GridFactorySelfTest.java    |   2 +-
 .../internal/IgniteDynamicCacheConfigTest.java  |   4 +-
 .../GridSpringTransactionManagerSelfTest.java   |   2 +-
 .../ignite/tools/classgen/ClassesGenerator.java |   2 +-
 .../twitter/IgniteTwitterStreamerTest.java      |   4 +-
 .../GridUriDeploymentFileProcessorSelfTest.java |   3 +-
 .../scala/org/apache/ignite/visor/visor.scala   | 115 ++-
 .../commands/mem/VisorMemoryCommandSpec.scala   |  33 +
 modules/web-console/backend/app/agent.js        |  12 +-
 modules/web-console/backend/app/mongo.js        |  19 +
 .../generator/AbstractTransformer.js            |   5 +
 .../modules/configuration/generator/Beans.js    |   4 +
 .../generator/ConfigurationGenerator.js         |  45 +-
 .../states/configuration/caches/affinity.jade   |  82 ++
 .../states/configuration/caches/memory.jade     |   4 +-
 .../frontend/views/configuration/caches.jade    |   1 +
 .../web-agent/bin/ignite-web-agent.bat          |   4 +-
 .../web-agent/bin/ignite-web-agent.sh           |   2 +
 .../ignite/console/agent/AgentLauncher.java     |  90 +-
 .../ignite/cache/websession/WebSessionV2.java   |   2 +-
 .../internal/websession/WebSessionSelfTest.java |   6 +-
 modules/yardstick/DEVNOTES.txt                  |  16 +
 modules/yardstick/README.txt                    |  10 +-
 modules/yardstick/pom-standalone.xml            | 209 +++++
 modules/yardstick/pom.xml                       |  10 +-
 .../yardstick/cache/IgniteIoTestBenchmark.java  |  73 --
 .../apache/ignite/yarn/ApplicationMaster.java   |   4 +-
 .../org/apache/ignite/yarn/IgniteProvider.java  |   4 +-
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  |   2 +-
 parent/pom.xml                                  |   2 +-
 pom.xml                                         |  95 +-
 893 files changed, 25767 insertions(+), 8085 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5982e39a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/5982e39a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
----------------------------------------------------------------------


[02/31] ignite git commit: Handle Dates correctly when parsing Date string without query

Posted by vo...@apache.org.
Handle Dates correctly when parsing Date string without query


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

Branch: refs/heads/ignite-6022-proto
Commit: 8c785ccc6c3b4fde3c6580fab7f5f5c0f8d3a154
Parents: d362da5
Author: Alexander Paschenko <al...@gmail.com>
Authored: Wed Dec 28 11:54:51 2016 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Wed Dec 28 11:54:51 2016 +0300

----------------------------------------------------------------------
 .../processors/query/h2/DmlStatementsProcessor.java         | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8c785ccc/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index f660148..70bd2bd 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -680,7 +680,14 @@ public class DmlStatementsProcessor {
 
         Value h2Val = desc.wrap(val, objType);
 
-        return h2Val.convertTo(type).getObject();
+        Object res = h2Val.convertTo(type).getObject();
+
+        if (res instanceof Date && res.getClass() != Date.class && expCls == Date.class) {
+            // We can get a Timestamp instead of Date when converting a String to Date without query - let's handle this
+            return new Date(((Date) res).getTime());
+        }
+
+        return res;
     }
 
     /**


[05/31] ignite git commit: Merge remote-tracking branch 'apache/master' into ignite-4490

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/336ad01a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 0abe2cb,98d123f..09b4a27
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@@ -58,8 -57,9 +57,10 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.query.GridQueryProperty;
  import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
  import org.apache.ignite.internal.processors.query.IgniteSQLException;
+ import org.apache.ignite.internal.processors.query.QueryUtils;
 +import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
  import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
+ import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode;
  import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
  import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
  import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
@@@ -238,72 -350,42 +351,69 @@@ public class DmlStatementsProcessor 
          if (plan.fastUpdateArgs != null) {
              assert F.isEmpty(failedKeys) && errKeysPos == null;
  
-             return new UpdateResult(doSingleUpdate(plan, params), X.EMPTY_OBJECT_ARRAY);
+             return doFastUpdate(plan, fieldsQry.getArgs());
          }
  
 -        assert !F.isEmpty(plan.selectQry);
 +        assert !F.isEmpty(plan.rows) ^ !F.isEmpty(plan.selectQry);
  
 -        QueryCursorImpl<List<?>> cur;
 +        Iterable<List<?>> cur;
  
          // Do a two-step query only if locality flag is not set AND if plan's SELECT corresponds to an actual
          // subquery and not some dummy stuff like "select 1, 2, 3;"
          if (!loc && !plan.isLocSubqry) {
 +            assert !F.isEmpty(plan.selectQry);
 +
              SqlFieldsQuery newFieldsQry = new SqlFieldsQuery(plan.selectQry, fieldsQry.isCollocated())
-                 .setArgs(params)
+                 .setArgs(fieldsQry.getArgs())
                  .setDistributedJoins(fieldsQry.isDistributedJoins())
                  .setEnforceJoinOrder(fieldsQry.isEnforceJoinOrder())
                  .setLocal(fieldsQry.isLocal())
                  .setPageSize(fieldsQry.getPageSize())
                  .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
  
-             cur = indexing.queryTwoStep(cctx, newFieldsQry, cancel);
+             cur = (QueryCursorImpl<List<?>>) idx.queryDistributedSqlFields(schemaName, newFieldsQry, true, cancel,
+                 mainCacheId);
          }
 -        else {
 +        else if (F.isEmpty(plan.rows)) {
-             final GridQueryFieldsResult res = indexing.queryLocalSqlFields(cctx.name(), plan.selectQry, F.asList(params),
-                 filters, fieldsQry.isEnforceJoinOrder(), fieldsQry.getTimeout(), cancel);
+             final GridQueryFieldsResult res = idx.queryLocalSqlFields(schemaName, plan.selectQry,
+                 F.asList(fieldsQry.getArgs()), filters, fieldsQry.isEnforceJoinOrder(), fieldsQry.getTimeout(), cancel);
  
-             QueryCursorImpl<List<?>> resCur = new QueryCursorImpl<>(new Iterable<List<?>>() {
+             cur = new QueryCursorImpl<>(new Iterable<List<?>>() {
                  @Override public Iterator<List<?>> iterator() {
                      try {
-                         return new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepBinary());
+                         return new GridQueryCacheObjectsIterator(res.iterator(), idx.objectContext(), true);
                      }
                      catch (IgniteCheckedException e) {
                          throw new IgniteException(e);
                      }
                  }
              }, cancel);
- 
-             resCur.fieldsMeta(res.metaData());
- 
-             cur = resCur;
          }
 +        else {
 +            assert plan.rowsNum > 0 && !F.isEmpty(plan.colNames);
 +
 +            List<List<?>> args = new ArrayList<>(plan.rowsNum);
 +
 +            GridH2RowDescriptor desc = plan.tbl.rowDescriptor();
 +
 +            for (List<FastUpdateArgument> argRow : plan.rows) {
 +                List<Object> row = new ArrayList<>();
 +
 +                for (int j = 0; j < plan.colNames.length; j++) {
 +                    Object colVal = argRow.get(j).apply(fieldsQry.getArgs());
 +
 +                    if (j == plan.keyColIdx || j == plan.valColIdx)
-                         colVal = convert(colVal, j == plan.keyColIdx ? desc.type().keyClass() : desc.type().valueClass(),
-                             desc);
++                        colVal = convert(colVal, desc, j == plan.keyColIdx ? desc.type().keyClass() :
++                            desc.type().valueClass(), plan.colTypes[j]);
 +
 +                    row.add(colVal);
 +                }
 +
 +                args.add(row);
 +            }
 +
 +            cur = args;
 +        }
  
          int pageSize = loc ? 0 : fieldsQry.getPageSize();
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/336ad01a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
index 056dfaa,cb47704..257014c
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
@@@ -50,40 -50,4 +50,40 @@@ public final class FastUpdateArguments 
              return null;
          }
      };
 +
 +    /** Simple constant value based operand. */
 +    public final static class ValueArgument implements FastUpdateArgument {
 +        /** Value to return. */
 +        private final Object val;
 +
 +        /** */
 +        public ValueArgument(Object val) {
 +            this.val = val;
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
 +            return val;
 +        }
 +    }
 +
-     /** Simple constant value based operand. */
++    /** User given param value operand. */
 +    public final static class ParamArgument implements FastUpdateArgument {
-         /** Value to return. */
++        /** Index of param to take. */
 +        private final int paramIdx;
 +
 +        /** */
 +        public ParamArgument(int paramIdx) {
 +            assert paramIdx >= 0;
 +
 +            this.paramIdx = paramIdx;
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
 +            assert arg.length > paramIdx;
 +
 +            return arg[paramIdx];
 +        }
 +    }
  }

http://git-wip-us.apache.org/repos/asf/ignite/blob/336ad01a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
index 9bd1ecf,b81ac60..534a164
--- 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
@@@ -53,9 -58,6 +59,9 @@@ public final class UpdatePlan 
      /** Subquery flag - {@code true} if {@link #selectQry} is an actual subquery that retrieves data from some cache. */
      public final boolean isLocSubqry;
  
 +    /** */
-     public final Iterable<List<FastUpdateArgument>> rows;
++    public final List<List<FastUpdateArgument>> rows;
 +
      /** Number of rows in rows based MERGE or INSERT. */
      public final int rowsNum;
  
@@@ -63,11 -65,11 +69,12 @@@
      public final FastUpdateArguments fastUpdateArgs;
  
      /** */
-     private UpdatePlan(UpdateMode mode, GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+     private UpdatePlan(UpdateMode mode, GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
          KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-         Iterable<List<FastUpdateArgument>> rows, int rowsNum, FastUpdateArguments fastUpdateArgs) {
 -        int rowsNum, FastUpdateArguments fastUpdateArgs) {
++        List<List<FastUpdateArgument>> rows, int rowsNum, FastUpdateArguments fastUpdateArgs) {
          this.colNames = colNames;
+         this.colTypes = colTypes;
 +        this.rows = rows;
          this.rowsNum = rowsNum;
          assert mode != null;
          assert tbl != null;
@@@ -84,23 -86,22 +91,23 @@@
      }
  
      /** */
-     public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+     public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
          KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-         Iterable<List<FastUpdateArgument>> rows, int rowsNum) {
 -        int rowsNum) {
++        List<List<FastUpdateArgument>> rows, int rowsNum) {
          assert !F.isEmpty(colNames);
  
-         return new UpdatePlan(UpdateMode.MERGE, tbl, colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+         return new UpdatePlan(UpdateMode.MERGE, tbl, colNames, colTypes, keySupplier, valSupplier, keyColIdx, valColIdx,
 -            selectQry, isLocSubqry, rowsNum, null);
 +            selectQry, isLocSubqry, rows, rowsNum, null);
      }
  
      /** */
-     public static UpdatePlan forInsert(GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+     public static UpdatePlan forInsert(GridH2Table tbl, String[] colNames, int[] colTypes, KeyValueSupplier keySupplier,
 -        KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry, int rowsNum) {
 +        KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
-         Iterable<List<FastUpdateArgument>> rows, int rowsNum) {
++        List<List<FastUpdateArgument>> rows, int rowsNum) {
          assert !F.isEmpty(colNames);
  
-         return new UpdatePlan(UpdateMode.INSERT, tbl, colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+         return new UpdatePlan(UpdateMode.INSERT, tbl, colNames, colTypes, keySupplier, valSupplier, keyColIdx, valColIdx,
 -            selectQry, isLocSubqry, rowsNum, null);
 +            selectQry, isLocSubqry, rows, rowsNum, null);
      }
  
      /** */
@@@ -108,20 -109,20 +115,20 @@@
          int valColIdx, String selectQry) {
          assert !F.isEmpty(colNames);
  
-         return new UpdatePlan(UpdateMode.UPDATE, tbl, colNames, null, valSupplier, -1, valColIdx, selectQry,
+         return new UpdatePlan(UpdateMode.UPDATE, tbl, colNames, colTypes, null, valSupplier, -1, valColIdx, selectQry,
 -            false, 0, null);
 +            false, null, 0, null);
      }
  
      /** */
      public static UpdatePlan forDelete(GridH2Table tbl, String selectQry) {
-         return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, -1, -1, selectQry, false, null, 0, null);
 -        return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, null, -1, -1, selectQry, false, 0, null);
++        return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, null, -1, -1, selectQry, false, null, 0, null);
      }
  
      /** */
      public static UpdatePlan forFastUpdate(UpdateMode mode, GridH2Table tbl, FastUpdateArguments fastUpdateArgs) {
          assert mode == UpdateMode.UPDATE || mode == UpdateMode.DELETE;
  
-         return new UpdatePlan(mode, tbl, null, null, null, -1, -1, null, false, null, 0, fastUpdateArgs);
 -        return new UpdatePlan(mode, tbl, null, null, null, null, -1, -1, null, false, 0, fastUpdateArgs);
++        return new UpdatePlan(mode, tbl, null, null, null, null, -1, -1, null, false, null, 0, fastUpdateArgs);
      }
  
  }

http://git-wip-us.apache.org/repos/asf/ignite/blob/336ad01a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index 8700b9a,b304109..cf6cd88
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@@ -230,11 -197,11 +234,11 @@@ public final class UpdatePlanBuilder 
          KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false, false);
  
          if (stmt instanceof GridSqlMerge)
-             return UpdatePlan.forMerge(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx,
+             return UpdatePlan.forMerge(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
 -                valColIdx, sel.getSQL(), !isTwoStepSubqry, rowsNum);
 +                valColIdx, sel != null ? sel.getSQL() : null, !isTwoStepSubqry, rows, rowsNum);
          else
-             return UpdatePlan.forInsert(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx,
+             return UpdatePlan.forInsert(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
 -                valColIdx, sel.getSQL(), !isTwoStepSubqry, rowsNum);
 +                valColIdx, sel != null ? sel.getSQL() : null, !isTwoStepSubqry, rows, rowsNum);
      }
  
      /**
@@@ -296,7 -265,10 +302,10 @@@
                  for (int i = 0; i < updatedCols.size(); i++) {
                      colNames[i] = updatedCols.get(i).columnName();
  
-                     if (VAL_FIELD_NAME.equals(colNames[i]))
+                     colTypes[i] = updatedCols.get(i).resultType().type();
+ 
 -                    Column column = updatedCols.get(i).column();
 -                    if (desc.isValueColumn(column.getColumnId()))
++                    Column col = updatedCols.get(i).column();
++                    if (desc.isValueColumn(col.getColumnId()))
                          valColIdx = i;
                  }
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/336ad01a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
index efc7f32,5a1d412..2cdb93c
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
@@@ -21,9 -21,10 +21,9 @@@ import java.util.ArrayList
  import java.util.HashSet;
  import java.util.List;
  import java.util.Set;
 -import org.apache.ignite.IgniteCheckedException;
  import org.apache.ignite.IgniteException;
  import org.apache.ignite.internal.processors.query.IgniteSQLException;
- import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+ import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
  import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
  import org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow;
  import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;


[04/31] ignite git commit: Merge remote-tracking branch 'apache/ignite-2.0' into ignite-4490

Posted by vo...@apache.org.
Merge remote-tracking branch 'apache/ignite-2.0' into ignite-4490

# Conflicts:
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
#	modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java


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

Branch: refs/heads/ignite-6022-proto
Commit: e526948e3d557787c5160bda672ca86eeead2d0d
Parents: 5982e39 9ce099c
Author: Alexander Paschenko <al...@gmail.com>
Authored: Thu Feb 9 15:23:46 2017 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Thu Feb 9 15:25:16 2017 +0300

----------------------------------------------------------------------
 assembly/LICENSE_FABRIC                         |  35 ++-
 assembly/LICENSE_HADOOP                         |  17 +-
 examples/schema-import/bin/h2-server.bat        |   2 +-
 examples/schema-import/bin/h2-server.sh         |   4 +-
 .../schema-import/bin/schema-import.properties  |   2 +-
 .../src/main/resources/META-INF/licenses.txt.vm |   7 +-
 .../apache/ignite/IgniteSystemProperties.java   |  11 +
 .../configuration/CacheConfiguration.java       |  16 +-
 .../processors/cache/GridCacheProcessor.java    |  85 +++++++
 .../distributed/dht/GridDhtCacheAdapter.java    |  10 +-
 .../distributed/dht/GridDhtLocalPartition.java  | 120 ++++++++--
 .../processors/query/GridQueryProcessor.java    |  81 +++++--
 .../processors/query/GridQueryProperty.java     |  21 +-
 .../ignite/lang/IgniteProductVersion.java       |   2 +-
 .../cache/CacheDeferredDeleteQueueTest.java     | 134 +++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../query/h2/DmlStatementsProcessor.java        |  70 +++---
 .../processors/query/h2/IgniteH2Indexing.java   |   8 +-
 .../query/h2/dml/UpdatePlanBuilder.java         | 105 ++++++---
 .../h2/twostep/GridReduceQueryExecutor.java     |   2 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |   4 +
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |   2 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  22 ++
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  24 ++
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |  61 ++++-
 .../IgniteSqlEntryCacheModeAgnosticTest.java    | 140 +++++++++++
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   5 +
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 .../Cache/CacheAbstractTransactionalTest.cs     | 105 +++++----
 .../Cache/Configuration/QueryEntity.cs          |   2 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   9 +-
 .../Apache.Ignite.Examples.csproj               |   1 +
 .../Datagrid/TransactionExample.cs              |  75 ++++--
 modules/web-console/backend/app/agent.js        |  10 +-
 modules/web-console/backend/app/mongo.js        |  49 ++--
 modules/web-console/backend/app/routes.js       |   5 +-
 .../web-console/backend/routes/activities.js    |  52 +++++
 modules/web-console/backend/routes/admin.js     |   2 +-
 modules/web-console/backend/routes/agent.js     |  10 +-
 modules/web-console/backend/routes/public.js    |   1 -
 .../web-console/backend/services/activities.js  | 136 +++++++++++
 modules/web-console/backend/services/users.js   |  13 +-
 modules/web-console/frontend/app/app.config.js  |   9 +
 modules/web-console/frontend/app/app.js         |  29 ++-
 .../activities-user-dialog.controller.js        |  60 +++++
 .../activities-user-dialog.jade                 |  36 +++
 .../components/activities-user-dialog/index.js  |  36 +++
 .../form-field-datepicker.jade                  |  55 +++++
 .../form-field-datepicker.scss                  |  20 ++
 .../list-of-registered-users/index.js           |  28 +++
 .../list-of-registered-users.categories.js      |  30 +++
 .../list-of-registered-users.column-defs.js     |  80 +++++++
 .../list-of-registered-users.controller.js      | 207 ++++++++++++++++
 .../list-of-registered-users.jade               |  54 +++++
 .../ui-grid-header/ui-grid-header.jade          |  27 +++
 .../ui-grid-header/ui-grid-header.scss          |  84 +++++++
 .../ui-grid-settings/ui-grid-settings.jade      |  33 +++
 .../ui-grid-settings/ui-grid-settings.scss      |  70 ++++++
 .../app/core/activities/Activities.data.js      |  39 ++++
 .../frontend/app/core/admin/Admin.data.js       |  77 ++++++
 modules/web-console/frontend/app/core/index.js  |  25 ++
 modules/web-console/frontend/app/data/i18n.js   |  38 +++
 .../ui-grid-settings/ui-grid-settings.jade      |  33 ---
 .../ui-grid-settings/ui-grid-settings.scss      |  38 ---
 .../app/filters/uiGridSubcategories.filter.js   |  24 ++
 .../frontend/app/modules/Demo/Demo.module.js    | 166 -------------
 .../frontend/app/modules/demo/Demo.module.js    | 172 ++++++++++++++
 .../frontend/app/modules/sql/sql.controller.js  |  14 +-
 .../frontend/app/modules/sql/sql.module.js      |   2 +-
 .../frontend/app/modules/states/admin.state.js  |   2 +-
 .../configuration/summary/summary.controller.js |   6 +-
 .../app/modules/user/AclRoute.provider.js       |  31 +--
 .../frontend/app/modules/user/Auth.service.js   |   2 +-
 .../frontend/app/modules/user/permissions.js    |   2 +-
 .../frontend/app/modules/user/user.module.js    |   6 +-
 modules/web-console/frontend/app/vendor.js      |   1 +
 .../frontend/controllers/admin-controller.js    | 234 -------------------
 .../frontend/controllers/domains-controller.js  |  12 +-
 modules/web-console/frontend/package.json       |   1 +
 .../stylesheets/_font-awesome-custom.scss       |  28 +++
 .../frontend/public/stylesheets/style.scss      |  39 +++-
 .../frontend/public/stylesheets/variables.scss  |   1 +
 .../frontend/views/settings/admin.jade          |  32 +--
 modules/web-console/frontend/views/sql/sql.jade |   4 +-
 .../views/templates/agent-download.jade         |   2 +-
 modules/web-console/web-agent/README.txt        |   2 +-
 .../cache/IgnitePutRemoveBenchmark.java         |  42 ++++
 .../cache/IgnitePutRemoveTxBenchmark.java       |  30 +++
 88 files changed, 2609 insertions(+), 818 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e526948e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 70bd2bd,7995083..0abe2cb
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@@ -58,9 -58,7 +58,8 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.query.GridQueryProperty;
  import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
  import org.apache.ignite.internal.processors.query.IgniteSQLException;
 +import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
  import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
- import org.apache.ignite.internal.processors.query.h2.dml.KeyValueSupplier;
  import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
  import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
  import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
@@@ -913,13 -841,32 +894,32 @@@ public class DmlStatementsProcessor 
          if (val == null)
              throw new IgniteSQLException("Value for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_VALUE);
  
-         GridQueryTypeDescriptor desc = rowDesc.type();
+         GridQueryTypeDescriptor desc = plan.tbl.rowDescriptor().type();
  
-         for (int i = 0; i < cols.length; i++) {
-             if (i == keyColIdx || i == valColIdx)
+         Map<String, Object> newColVals = new HashMap<>();
+ 
+         for (int i = 0; i < plan.colNames.length; i++) {
+             if (i == plan.keyColIdx || i == plan.valColIdx)
                  continue;
  
-             desc.setValue(cols[i], key, val, convert(row[i], cols[i], rowDesc));
+             newColVals.put(plan.colNames[i], convert(row.get(i), plan.colNames[i],
 -                plan.tbl.rowDescriptor(), plan.colTypes[i]));
++                plan.tbl.rowDescriptor()));
+         }
+ 
+         // We update columns in the order specified by the table for a reason - table's
+         // column order preserves their precedence for correct update of nested properties.
+         Column[] cols = plan.tbl.getColumns();
+ 
+         // First 2 columns are _key and _val, skip 'em.
+         for (int i = 2; i < cols.length; i++) {
+             String colName = cols[i].getName();
+ 
+             if (!newColVals.containsKey(colName))
+                 continue;
+ 
+             Object colVal = newColVals.get(colName);
+ 
+             desc.setValue(colName, key, val, colVal);
          }
  
          if (cctx.binaryMarshaller()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e526948e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index 0303fa4,ce2971a..8700b9a
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@@ -225,15 -193,15 +226,15 @@@ public final class UpdatePlanBuilder 
                  hasValProps = true;
          }
  
-         KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, true);
-         KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false);
+         KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, true, false);
+         KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false, false);
  
          if (stmt instanceof GridSqlMerge)
 -            return UpdatePlan.forMerge(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
 -                valColIdx, sel.getSQL(), !isTwoStepSubqry, rowsNum);
 +            return UpdatePlan.forMerge(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx,
 +                valColIdx, sel != null ? sel.getSQL() : null, !isTwoStepSubqry, rows, rowsNum);
          else
 -            return UpdatePlan.forInsert(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
 -                valColIdx, sel.getSQL(), !isTwoStepSubqry, rowsNum);
 +            return UpdatePlan.forInsert(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx,
 +                valColIdx, sel != null ? sel.getSQL() : null, !isTwoStepSubqry, rows, rowsNum);
      }
  
      /**


[11/31] ignite git commit: Merge remote-tracking branch 'apache/master' into ignite-4490

Posted by vo...@apache.org.
Merge remote-tracking branch 'apache/master' into ignite-4490

# Conflicts:
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java


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

Branch: refs/heads/ignite-6022-proto
Commit: 799098c6793d5351895b84fb7c5b80c2eefcc621
Parents: e66b664 e358ae2
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Dec 15 14:38:40 2017 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Fri Dec 15 14:38:40 2017 +0300

----------------------------------------------------------------------
 .gitignore                                      |     3 +
 README.md                                       |   323 +-
 README.txt                                      |     1 +
 RELEASE_NOTES.txt                               |   108 +
 assembly/dependencies-fabric-lgpl.xml           |     4 +
 assembly/dependencies-fabric.xml                |     4 +
 assembly/dependencies-sqlline.xml               |    58 +
 assembly/release-fabric-base.xml                |     3 +-
 bin/control.bat                                 |     1 +
 bin/control.sh                                  |     1 +
 examples/config/example-data-regions.xml        |   106 +
 examples/config/example-memory-policies.xml     |   108 -
 .../example-persistent-store.xml                |    37 +-
 examples/config/redis/example-redis.xml         |    74 +
 examples/pom-standalone-lgpl.xml                |    32 +-
 examples/pom-standalone.xml                     |    32 +-
 examples/pom.xml                                |    46 +-
 examples/redis/redis-example.py                 |    16 +-
 .../examples/datagrid/CacheQueryDdlExample.java |     2 +-
 .../examples/datagrid/DataRegionsExample.java   |   113 +
 .../ignite/examples/datagrid/JdbcExample.java   |     2 +-
 .../datagrid/MemoryPoliciesExample.java         |   114 -
 .../datastructures/IgniteSemaphoreExample.java  |    21 +-
 .../persistentstore/PersistentStoreExample.java |    27 +-
 .../clustering/DatasetWithObviousStructure.java |   105 +
 .../ml/clustering/FuzzyCMeansExample.java       |   125 +
 .../KMeansDistributedClustererExample.java      |    95 +
 .../clustering/KMeansLocalClustererExample.java |   106 +
 .../examples/ml/clustering/package-info.java    |    22 +
 .../KNNClassificationExample.java               |   151 +
 .../ml/knn/classification/package-info.java     |    22 +
 .../ignite/examples/ml/knn/package-info.java    |    22 +
 .../ml/knn/regression/KNNRegressionExample.java |   152 +
 .../ml/knn/regression/package-info.java         |    22 +
 .../decompositions/QRDecompositionExample.java  |    82 +
 .../ml/math/matrix/CacheMatrixExample.java      |     4 +-
 .../ml/math/matrix/ExampleMatrixStorage.java    |    12 +-
 .../DistributedRegressionExample.java           |   149 -
 .../ml/math/vector/CacheVectorExample.java      |     4 +-
 .../apache/ignite/examples/ml/package-info.java |    22 +
 .../DistributedRegressionExample.java           |   149 +
 .../DistributedRegressionModelExample.java      |   134 +
 .../examples/ml/regression/package-info.java    |    22 +
 .../ignite/examples/ml/trees/MNISTExample.java  |   261 +
 .../ignite/examples/ml/trees/package-info.java  |    22 +
 .../src/main/resources/datasets/knn/README.md   |     2 +
 .../resources/datasets/knn/cleared_machines.txt |   209 +
 .../src/main/resources/datasets/knn/iris.txt    |   150 +
 .../examples/ScalarCreditRiskExample.scala      |     4 +-
 .../ignite/examples/CacheExamplesSelfTest.java  |     6 +-
 modules/aop/pom.xml                             |     2 +-
 modules/apache-license-gen/pom.xml              |     2 +-
 modules/aws/pom.xml                             |     2 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |    81 +-
 .../spi/checkpoint/s3/S3CheckpointSpiMBean.java |    12 +
 .../tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java |    65 +-
 .../s3/S3CheckpointManagerSelfTest.java         |     2 +-
 .../checkpoint/s3/S3CheckpointSpiSelfTest.java  |    26 +-
 ...pointSpiStartStopBucketEndpointSelfTest.java |    50 +
 ...ckpointSpiStartStopSSEAlgorithmSelfTest.java |    49 +
 .../s3/S3CheckpointSpiStartStopSelfTest.java    |     2 +-
 .../s3/S3SessionCheckpointSelfTest.java         |     2 +-
 .../TcpDiscoveryS3IpFinderAbstractSelfTest.java |    66 +-
 ...3IpFinderAwsCredentialsProviderSelfTest.java |     1 +
 ...scoveryS3IpFinderAwsCredentialsSelfTest.java |     1 +
 ...scoveryS3IpFinderBucketEndpointSelfTest.java |    55 +
 ...DiscoveryS3IpFinderSSEAlgorithmSelfTest.java |    48 +
 .../ignite/testsuites/IgniteS3TestSuite.java    |     8 +
 modules/benchmarks/pom.xml                      |     2 +-
 .../benchmarks/jmh/tree/BPlusTreeBenchmark.java |     8 +-
 modules/camel/pom.xml                           |     2 +-
 modules/cassandra/pom.xml                       |     2 +-
 modules/cassandra/serializers/pom.xml           |     4 +-
 modules/cassandra/store/pom.xml                 |     4 +-
 modules/clients/pom.xml                         |    16 +-
 modules/clients/src/test/config/jdbc-config.xml |     1 +
 .../jdbc2/JdbcConnectionReopenTest.java         |    51 +
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |    49 +-
 .../jdbc2/JdbcDeleteStatementSelfTest.java      |    22 +
 .../jdbc2/JdbcDynamicIndexAbstractSelfTest.java |    39 +-
 .../internal/jdbc2/JdbcErrorsSelfTest.java      |    67 +
 .../jdbc2/JdbcInsertStatementSelfTest.java      |   168 +-
 .../internal/jdbc2/JdbcLocalCachesSelfTest.java |    28 +
 .../jdbc2/JdbcMergeStatementSelfTest.java       |    41 +
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |   184 +-
 .../jdbc2/JdbcStatementBatchingSelfTest.java    |   133 +
 .../internal/jdbc2/JdbcStatementSelfTest.java   |   130 +-
 .../jdbc2/JdbcUpdateStatementSelfTest.java      |    24 +
 .../JettyRestProcessorAbstractSelfTest.java     |    46 +
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java |   659 +
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    35 +-
 .../JdbcThinAbstractDmlStatementSelfTest.java   |    16 +-
 .../jdbc/thin/JdbcThinAbstractSelfTest.java     |    71 +-
 .../thin/JdbcThinAutoCloseServerCursorTest.java |    12 +-
 .../ignite/jdbc/thin/JdbcThinBatchSelfTest.java |     2 +-
 .../thin/JdbcThinComplexDmlDdlSelfTest.java     |   493 +
 ...omplexDmlDdlSkipReducerOnUpdateSelfTest.java |    33 +
 .../jdbc/thin/JdbcThinComplexQuerySelfTest.java |     2 +-
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   |  1560 ++-
 .../JdbcThinDynamicIndexAbstractSelfTest.java   |     6 +-
 .../jdbc/thin/JdbcThinEmptyCacheSelfTest.java   |     2 +-
 .../jdbc/thin/JdbcThinErrorsSelfTest.java       |   108 +
 .../thin/JdbcThinInsertStatementSelfTest.java   |     3 +-
 ...ertStatementSkipReducerOnUpdateSelfTest.java |    33 +
 ...rgeStatementSkipReducerOnUpdateSelfTest.java |    33 +
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |   362 +-
 .../JdbcThinMissingLongArrayResultsTest.java    |   341 +
 .../jdbc/thin/JdbcThinNoDefaultSchemaTest.java  |    85 +-
 .../thin/JdbcThinPreparedStatementSelfTest.java |   263 +-
 .../jdbc/thin/JdbcThinResultSetSelfTest.java    |  1197 +-
 .../jdbc/thin/JdbcThinSchemaCaseTest.java       |   129 +
 .../thin/JdbcThinSelectAfterAlterTable.java     |   173 +
 .../jdbc/thin/JdbcThinStatementSelfTest.java    |   916 +-
 ...ateStatementSkipReducerOnUpdateSelfTest.java |    33 +
 modules/clients/src/test/keystore/ca/node01.jks |   Bin 0 -> 3719 bytes
 modules/clients/src/test/keystore/ca/node02.jks |   Bin 0 -> 4598 bytes
 modules/clients/src/test/keystore/ca/node03.jks |   Bin 0 -> 3754 bytes
 modules/clients/src/test/keystore/ca/oneca.cnf  |    31 +
 modules/clients/src/test/keystore/ca/oneca.key  |    28 +
 .../clients/src/test/keystore/ca/oneindex.txt   |     1 +
 .../src/test/keystore/ca/oneindex.txt.attr      |     1 +
 modules/clients/src/test/keystore/ca/oneserial  |     1 +
 .../clients/src/test/keystore/ca/trust-both.jks |   Bin 0 -> 1718 bytes
 .../clients/src/test/keystore/ca/trust-one.jks  |   Bin 0 -> 877 bytes
 .../clients/src/test/keystore/ca/trust-two.jks  |   Bin 0 -> 891 bytes
 modules/clients/src/test/keystore/ca/twoca.cnf  |    31 +
 modules/clients/src/test/keystore/ca/twoca.key  |    28 +
 .../clients/src/test/keystore/ca/twoindex.txt   |     2 +
 .../src/test/keystore/ca/twoindex.txt.attr      |     1 +
 modules/clients/src/test/keystore/ca/twoserial  |     1 +
 modules/cloud/pom.xml                           |     2 +-
 modules/codegen/pom.xml                         |     2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |     2 +
 modules/compatibility/README.txt                |     5 +
 modules/compatibility/licenses/apache-2.0.txt   |   202 +
 modules/compatibility/pom.xml                   |   126 +
 .../ignite/compatibility/package-info.java      |    22 +
 .../DummyPersistenceCompatibilityTest.java      |   361 +
 .../FoldersReuseCompatibilityTest.java          |   262 +
 ...itePersistenceCompatibilityAbstractTest.java |    94 +
 ...tingToWalV2SerializerWithCompactionTest.java |   250 +
 .../compatibility/persistence/package-info.java |    22 +
 .../junits/CompatibilityTestsFacade.java        |    46 +
 .../testframework/junits/Dependency.java        |   117 +
 .../junits/IgniteCompatibilityAbstractTest.java |   329 +
 .../junits/IgniteCompatibilityNodeRunner.java   |   221 +
 .../logger/ListenedGridTestLog4jLogger.java     |    91 +
 .../junits/logger/package-info.java             |    22 +
 .../testframework/junits/package-info.java      |    22 +
 .../testframework/package-info.java             |    22 +
 .../plugins/DisabledValidationProcessor.java    |    42 +
 .../TestCompatibilityPluginProvider.java        |   119 +
 .../testframework/plugins/package-info.java     |    22 +
 .../testframework/util/MavenUtils.java          |   196 +
 .../testframework/util/package-info.java        |    22 +
 .../IgniteCompatibilityBasicTestSuite.java      |    44 +
 .../compatibility/testsuites/package-info.java  |    22 +
 .../org.apache.ignite.plugin.PluginProvider     |     1 +
 modules/core/pom.xml                            |     2 +-
 .../org/apache/ignite/DataRegionMetrics.java    |   130 +
 .../apache/ignite/DataRegionMetricsAdapter.java |   106 +
 .../org/apache/ignite/DataStorageMetrics.java   |   114 +
 .../ignite/DataStorageMetricsAdapter.java       |   101 +
 .../src/main/java/org/apache/ignite/Ignite.java |    49 +-
 .../org/apache/ignite/IgniteJdbcDriver.java     |    46 +-
 .../org/apache/ignite/IgniteJdbcThinDriver.java |    53 +-
 .../java/org/apache/ignite/IgniteServices.java  |   106 +-
 .../apache/ignite/IgniteSystemProperties.java   |    47 +
 .../java/org/apache/ignite/MemoryMetrics.java   |    11 +-
 .../org/apache/ignite/PersistenceMetrics.java   |     4 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |    46 +-
 .../java/org/apache/ignite/cache/CacheMode.java |     2 +-
 .../org/apache/ignite/cache/QueryEntity.java    |    59 +-
 .../org/apache/ignite/cache/QueryIndex.java     |    41 +-
 .../eviction/AbstractEvictionPolicyFactory.java |   106 +
 .../fifo/FifoEvictionPolicyFactory.java         |    72 +
 .../eviction/lru/LruEvictionPolicyFactory.java  |    72 +
 .../sorted/SortedEvictionPolicyFactory.java     |    98 +
 .../ignite/cache/query/SqlFieldsQuery.java      |    63 +-
 .../query/annotations/QueryGroupIndex.java      |    20 +
 .../cache/query/annotations/QuerySqlField.java  |    30 +
 .../apache/ignite/cluster/ClusterMetrics.java   |    12 +-
 .../configuration/CacheConfiguration.java       |    95 +-
 .../configuration/CheckpointWriteOrder.java     |    51 +
 .../ClientConnectorConfiguration.java           |   275 +
 .../configuration/DataPageEvictionMode.java     |     8 +-
 .../configuration/DataRegionConfiguration.java  |   434 +
 .../configuration/DataStorageConfiguration.java |   880 ++
 .../configuration/IgniteConfiguration.java      |    91 +-
 .../configuration/MemoryConfiguration.java      |    19 +-
 .../MemoryPolicyConfiguration.java              |     9 +-
 .../configuration/NearCacheConfiguration.java   |    37 +
 .../ignite/configuration/OdbcConfiguration.java |     4 +-
 .../PersistentStoreConfiguration.java           |    67 +-
 .../SqlConnectorConfiguration.java              |     7 +-
 .../apache/ignite/configuration/WALMode.java    |    15 +-
 .../org/apache/ignite/igfs/IgfsMetrics.java     |     4 +-
 .../ClusterLocalNodeMetricsMXBeanImpl.java      |    69 +-
 .../internal/ClusterMetricsMXBeanImpl.java      |   438 +
 .../ignite/internal/ClusterMetricsSnapshot.java |    41 +-
 .../internal/DuplicateTypeIdException.java      |    74 +
 .../internal/GridEventConsumeHandler.java       |     9 +-
 .../ignite/internal/GridJobExecuteRequest.java  |   133 +-
 .../ignite/internal/GridKernalContext.java      |    11 +-
 .../ignite/internal/GridKernalContextImpl.java  |    28 +-
 .../apache/ignite/internal/GridLoggerProxy.java |     9 +
 .../ignite/internal/GridTaskSessionImpl.java    |    18 +
 .../ignite/internal/IgniteComputeImpl.java      |     8 +-
 .../internal/IgniteDiagnosticMessage.java       |     2 +-
 .../IgniteDiagnosticPrepareContext.java         |     4 +-
 .../org/apache/ignite/internal/IgniteEx.java    |    14 +
 .../apache/ignite/internal/IgniteKernal.java    |   191 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    10 +
 .../ignite/internal/IgniteServicesImpl.java     |    63 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   169 +-
 .../ignite/internal/MarshallerContextImpl.java  |    60 +-
 .../internal/MarshallerMappingFileStore.java    |   115 +-
 .../internal/binary/BinaryClassDescriptor.java  |     7 +-
 .../ignite/internal/binary/BinaryContext.java   |    67 +-
 .../ignite/internal/binary/BinaryMetadata.java  |     8 +-
 .../ignite/internal/binary/BinaryUtils.java     |    26 +
 .../binary/builder/BinaryObjectBuilderImpl.java |    19 +-
 .../GridClientConnectionManagerAdapter.java     |     1 -
 .../connection/GridClientNioTcpConnection.java  |     4 +-
 .../internal/commandline/CommandHandler.java    |     2 +-
 .../internal/jdbc/JdbcDatabaseMetadata.java     |     2 +-
 .../jdbc/thin/ConnectionProperties.java         |   148 +
 .../jdbc/thin/ConnectionPropertiesImpl.java     |   637 +
 .../internal/jdbc/thin/JdbcThinConnection.java  |   296 +-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     |  1545 +++
 .../jdbc/thin/JdbcThinParameterMetadata.java    |   115 +
 .../jdbc/thin/JdbcThinPreparedStatement.java    |    75 +-
 .../internal/jdbc/thin/JdbcThinResultSet.java   |   707 +-
 .../internal/jdbc/thin/JdbcThinStatement.java   |   305 +-
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |   407 +-
 .../internal/jdbc/thin/JdbcThinUtils.java       |    67 +-
 .../internal/jdbc2/JdbcBatchUpdateTask.java     |   224 +
 .../ignite/internal/jdbc2/JdbcConnection.java   |   129 +-
 .../internal/jdbc2/JdbcDatabaseMetadata.java    |   724 +-
 .../internal/jdbc2/JdbcPreparedStatement.java   |    25 +-
 .../jdbc2/JdbcQueryMultipleStatementsTask.java  |   168 +
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |   180 +-
 .../internal/jdbc2/JdbcQueryTaskResult.java     |   120 +
 .../ignite/internal/jdbc2/JdbcQueryTaskV2.java  |   107 +
 .../ignite/internal/jdbc2/JdbcQueryTaskV3.java  |   105 +
 .../ignite/internal/jdbc2/JdbcResultSet.java    |   193 +-
 .../internal/jdbc2/JdbcSqlFieldsQuery.java      |   105 -
 .../ignite/internal/jdbc2/JdbcStatement.java    |   340 +-
 .../internal/jdbc2/JdbcStatementResultInfo.java |    73 +
 .../jdbc2/JdbcStreamedPreparedStatement.java    |    19 +-
 .../apache/ignite/internal/jdbc2/JdbcUtils.java |    42 +
 .../logger/platform/PlatformLogger.java         |    15 +
 .../managers/communication/GridIoManager.java   |    11 +-
 .../communication/GridIoMessageFactory.java     |    23 +-
 .../managers/deployment/GridDeploymentInfo.java |     1 +
 .../deployment/GridDeploymentInfoBean.java      |    21 +-
 .../GridDeploymentPerVersionStore.java          |    44 +-
 .../managers/discovery/ConsistentIdMapper.java  |   101 +
 .../internal/managers/discovery/DiscoCache.java |   160 +-
 .../discovery/DiscoveryCustomMessage.java       |    13 +-
 .../discovery/GridDiscoveryManager.java         |   410 +-
 .../optimized/OptimizedObjectInputStream.java   |    22 +
 .../optimized/OptimizedObjectOutputStream.java  |    17 +-
 .../ignite/internal/pagemem/PageIdUtils.java    |    32 +
 .../ignite/internal/pagemem/PageUtils.java      |    26 +
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |    34 +-
 .../pagemem/store/IgnitePageStoreManager.java   |     4 +-
 .../internal/pagemem/store/PageStore.java       |     5 +-
 .../pagemem/wal/IgniteWriteAheadLogManager.java |     8 +
 .../ignite/internal/pagemem/wal/WALPointer.java |     4 +-
 .../pagemem/wal/record/CheckpointRecord.java    |     6 +
 .../internal/pagemem/wal/record/DataEntry.java  |    28 +-
 .../internal/pagemem/wal/record/DataRecord.java |    26 +-
 .../pagemem/wal/record/FilteredRecord.java      |    31 +
 .../pagemem/wal/record/LazyDataEntry.java       |    38 +-
 .../pagemem/wal/record/MarshalledRecord.java    |    61 +
 .../wal/record/MemoryRecoveryRecord.java        |     7 +
 .../pagemem/wal/record/PageSnapshot.java        |    23 +-
 .../pagemem/wal/record/SnapshotRecord.java      |    58 +
 .../pagemem/wal/record/SwitchSegmentRecord.java |    28 +
 .../pagemem/wal/record/TimeStampRecord.java     |    57 +
 .../internal/pagemem/wal/record/TxRecord.java   |   163 +-
 .../pagemem/wal/record/UnwrapDataEntry.java     |   126 +
 .../internal/pagemem/wal/record/WALRecord.java  |     6 +-
 .../delta/DataPageInsertFragmentRecord.java     |    13 +-
 .../wal/record/delta/DataPageInsertRecord.java  |     6 +
 .../wal/record/delta/DataPageRemoveRecord.java  |     2 +-
 .../delta/DataPageSetFreeListPageRecord.java    |     6 +
 .../wal/record/delta/DataPageUpdateRecord.java  |     6 +
 .../wal/record/delta/FixCountRecord.java        |     9 +
 .../record/delta/FixLeftmostChildRecord.java    |     9 +
 .../pagemem/wal/record/delta/FixRemoveId.java   |     6 +
 .../wal/record/delta/InitNewPageRecord.java     |     2 -
 .../wal/record/delta/InnerReplaceRecord.java    |     6 +
 .../pagemem/wal/record/delta/InsertRecord.java  |     2 +-
 .../pagemem/wal/record/delta/MergeRecord.java   |     2 +-
 .../wal/record/delta/MetaPageAddRootRecord.java |     9 +
 .../wal/record/delta/MetaPageCutRootRecord.java |     6 +
 .../wal/record/delta/MetaPageInitRecord.java    |     6 +
 .../delta/MetaPageInitRootInlineRecord.java     |     7 +-
 .../record/delta/MetaPageInitRootRecord.java    |     6 +
 .../delta/MetaPageUpdateLastAllocatedIndex.java |     6 +
 ...aPageUpdateLastSuccessfulFullSnapshotId.java |     6 +
 .../MetaPageUpdateLastSuccessfulSnapshotId.java |     6 +
 .../delta/MetaPageUpdateNextSnapshotId.java     |     6 +
 .../MetaPageUpdatePartitionDataRecord.java      |     2 +-
 .../wal/record/delta/NewRootInitRecord.java     |     6 +
 .../delta/PageListMetaResetCountRecord.java     |     6 +
 .../delta/PagesListInitNewPageRecord.java       |     4 +-
 .../record/delta/PagesListRemovePageRecord.java |     3 +-
 .../record/delta/PagesListSetNextRecord.java    |     6 +
 .../delta/PagesListSetPreviousRecord.java       |     6 +
 .../record/delta/PartitionDestroyRecord.java    |     6 +
 .../record/delta/PartitionMetaStateRecord.java  |     6 +
 .../pagemem/wal/record/delta/RecycleRecord.java |    15 +
 .../pagemem/wal/record/delta/RemoveRecord.java  |     2 +-
 .../pagemem/wal/record/delta/ReplaceRecord.java |     6 +
 .../record/delta/SplitExistingPageRecord.java   |     2 +-
 .../record/delta/SplitForwardPageRecord.java    |    24 +
 .../record/delta/TrackingPageDeltaRecord.java   |    14 +-
 .../processors/affinity/AffinityAssignment.java |     5 +
 .../affinity/AffinityTopologyVersion.java       |     7 +
 .../affinity/GridAffinityAssignment.java        |    45 +-
 .../affinity/GridAffinityAssignmentCache.java   |    60 +-
 .../affinity/GridAffinityProcessor.java         |     8 +-
 .../affinity/HistoryAffinityAssignment.java     |    15 +
 .../cache/CacheAffinityChangeMessage.java       |     8 +
 .../cache/CacheAffinitySharedManager.java       |   789 +-
 .../cache/CacheClusterMetricsMXBeanImpl.java    |    32 +-
 .../processors/cache/CacheGroupContext.java     |    64 +-
 .../processors/cache/CacheGroupData.java        |    17 +-
 .../processors/cache/CacheGroupDescriptor.java  |    19 +-
 .../cache/CacheGroupMetricsMXBeanImpl.java      |   255 +
 .../cache/CacheLocalMetricsMXBeanImpl.java      |    32 +-
 .../processors/cache/CacheMetricsImpl.java      |    81 +-
 .../processors/cache/CacheMetricsSnapshot.java  |    48 +
 .../processors/cache/CacheObjectUtils.java      |    65 +-
 .../cache/CachePartitionExchangeWorkerTask.java |     5 +-
 .../ClientCacheChangeDiscoveryMessage.java      |     9 +
 .../ClientCacheChangeDummyDiscoveryMessage.java |    14 +
 .../cache/ClientCacheUpdateTimeout.java         |     5 +
 .../processors/cache/ClusterCachesInfo.java     |    55 +-
 .../cache/DynamicCacheChangeBatch.java          |     9 +
 .../cache/DynamicCacheChangeRequest.java        |    29 +-
 .../processors/cache/ExchangeContext.java       |   131 +
 .../cache/ExchangeDiscoveryEvents.java          |   262 +
 .../cache/GatewayProtectedCacheProxy.java       |     5 -
 .../processors/cache/GridCacheAdapter.java      |   299 +-
 .../cache/GridCacheAffinityManager.java         |     9 +-
 .../processors/cache/GridCacheAttributes.java   |    19 +
 .../processors/cache/GridCacheContext.java      |    38 +-
 .../cache/GridCacheDeploymentManager.java       |    54 +-
 .../processors/cache/GridCacheEntryEx.java      |    18 +-
 .../cache/GridCacheEvictionManager.java         |    15 +-
 .../processors/cache/GridCacheIoManager.java    |    66 +-
 .../processors/cache/GridCacheLockCallback.java |    51 +
 .../processors/cache/GridCacheMapEntry.java     |    98 +-
 .../processors/cache/GridCacheMvccCallback.java |    51 -
 .../processors/cache/GridCacheMvccFuture.java   |    37 -
 .../processors/cache/GridCacheMvccManager.java  |    51 +-
 .../GridCachePartitionExchangeManager.java      |   517 +-
 .../processors/cache/GridCacheProcessor.java    |   408 +-
 .../processors/cache/GridCacheProxyImpl.java    |    12 -
 .../cache/GridCacheSharedContext.java           |    22 +-
 .../processors/cache/GridCacheTtlManager.java   |     3 +-
 .../processors/cache/GridCacheUtils.java        |   343 +-
 .../cache/GridCacheVersionedFuture.java         |    37 +
 .../cache/IgniteCacheOffheapManager.java        |    23 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  1305 +-
 .../processors/cache/IgniteCacheProxy.java      |     8 -
 .../processors/cache/IgniteCacheProxyImpl.java  |    50 +-
 .../processors/cache/IgniteInternalCache.java   |     9 -
 .../processors/cache/StoredCacheData.java       |     9 +
 .../cache/binary/BinaryMetadataFileStore.java   |    81 +-
 .../cache/binary/BinaryMetadataTransport.java   |    38 +-
 .../binary/CacheObjectBinaryProcessor.java      |    20 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |    59 +-
 .../binary/MetadataUpdateAcceptedMessage.java   |     9 +
 .../binary/MetadataUpdateProposedMessage.java   |     9 +
 .../CacheDataStructuresManager.java             |    12 +
 .../GridDistributedTxRemoteAdapter.java         |     7 +-
 .../dht/CacheDistributedGetFutureAdapter.java   |     5 +-
 .../dht/ClientCacheDhtTopologyFuture.java       |    12 +-
 .../dht/GridClientPartitionTopology.java        |   225 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |    24 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |     2 -
 .../distributed/dht/GridDhtGetSingleFuture.java |     2 -
 .../distributed/dht/GridDhtLocalPartition.java  |   111 +-
 .../distributed/dht/GridDhtLockFuture.java      |     6 +-
 .../dht/GridDhtPartitionTopology.java           |    67 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  1377 +-
 .../dht/GridDhtPartitionsReservation.java       |     2 +-
 .../distributed/dht/GridDhtTopologyFuture.java  |    36 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |    97 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |     2 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |    54 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |     4 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   462 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |     5 +
 .../dht/GridPartitionedGetFuture.java           |    62 +-
 .../dht/GridPartitionedSingleGetFuture.java     |    43 +-
 .../GridDhtAtomicAbstractUpdateFuture.java      |     2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   181 +-
 .../GridNearAtomicSingleUpdateFuture.java       |    10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |     9 +-
 .../dht/colocated/GridDhtColocatedCache.java    |    30 +-
 .../colocated/GridDhtColocatedLockFuture.java   |    87 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |     4 +-
 .../preloader/CacheGroupAffinityMessage.java    |   339 +
 .../CachePartitionFullCountersMap.java          |   135 +
 .../CachePartitionPartialCountersMap.java       |   184 +
 .../preloader/ForceRebalanceExchangeTask.java   |     5 +
 .../dht/preloader/GridDhtPartitionDemander.java |    42 +-
 .../preloader/GridDhtPartitionExchangeId.java   |    11 +
 .../dht/preloader/GridDhtPartitionMap.java      |    30 +-
 .../dht/preloader/GridDhtPartitionSupplier.java |     6 +-
 .../GridDhtPartitionsAbstractMessage.java       |    39 +-
 .../GridDhtPartitionsExchangeFuture.java        |  2138 +++-
 .../preloader/GridDhtPartitionsFullMessage.java |   228 +-
 .../GridDhtPartitionsSingleMessage.java         |   108 +-
 .../GridDhtPartitionsSingleRequest.java         |    55 +-
 .../dht/preloader/GridDhtPreloader.java         |    46 +-
 .../IgniteDhtPartitionCountersMap.java          |     7 +
 .../IgniteDhtPartitionCountersMap2.java         |    69 +
 .../dht/preloader/InitNewCoordinatorFuture.java |   346 +
 .../RebalanceReassignExchangeTask.java          |     5 +
 .../distributed/near/GridNearAtomicCache.java   |     2 -
 .../distributed/near/GridNearCacheAdapter.java  |     5 +-
 .../distributed/near/GridNearCacheEntry.java    |     7 +-
 .../distributed/near/GridNearGetFuture.java     |    47 +-
 .../distributed/near/GridNearGetRequest.java    |     2 +-
 .../distributed/near/GridNearLockFuture.java    |   124 +-
 .../distributed/near/GridNearLockRequest.java   |     2 +-
 ...arOptimisticSerializableTxPrepareFuture.java |     5 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |    22 +-
 .../GridNearPessimisticTxPrepareFuture.java     |     3 +-
 .../near/GridNearTransactionalCache.java        |    12 +-
 .../near/GridNearTxFastFinishFuture.java        |    82 +
 .../near/GridNearTxFinishFuture.java            |    23 +-
 .../cache/distributed/near/GridNearTxLocal.java |   344 +-
 .../near/GridNearTxPrepareFutureAdapter.java    |     4 +-
 .../near/GridNearTxPrepareRequest.java          |    16 +-
 .../distributed/near/GridNearTxRemote.java      |     5 +
 .../distributed/near/NearTxFinishFuture.java    |    31 +
 .../processors/cache/local/GridLocalCache.java  |     5 +-
 .../cache/local/GridLocalLockFuture.java        |     4 +-
 .../local/atomic/GridLocalAtomicCache.java      |   196 +-
 .../cache/persistence/CacheDataRowAdapter.java  |     7 +-
 .../cache/persistence/DataRegion.java           |    84 +
 .../persistence/DataRegionMetricsImpl.java      |   307 +
 .../DataRegionMetricsMXBeanImpl.java            |   136 +
 .../persistence/DataRegionMetricsSnapshot.java  |   121 +
 .../persistence/DataStorageMetricsImpl.java     |   297 +
 .../persistence/DataStorageMetricsSnapshot.java |   144 +
 .../GridCacheDatabaseSharedManager.java         |   714 +-
 .../persistence/GridCacheOffheapManager.java    |    67 +-
 .../IgniteCacheDatabaseSharedManager.java       |   599 +-
 .../cache/persistence/MemoryMetricsImpl.java    |   284 -
 .../persistence/MemoryMetricsMXBeanImpl.java    |   131 -
 .../persistence/MemoryMetricsSnapshot.java      |   112 -
 .../cache/persistence/MemoryPolicy.java         |    83 -
 .../cache/persistence/MetadataStorage.java      |    21 +-
 .../persistence/PersistenceMetricsImpl.java     |   297 -
 .../persistence/PersistenceMetricsSnapshot.java |   144 -
 .../processors/cache/persistence/RowStore.java  |    40 +-
 .../evict/FairFifoPageEvictionTracker.java      |     6 +-
 .../evict/PageAbstractEvictionTracker.java      |     6 +-
 .../evict/Random2LruPageEvictionTracker.java    |     8 +-
 .../evict/RandomLruPageEvictionTracker.java     |     8 +-
 .../cache/persistence/file/AsyncFileIO.java     |   218 +
 .../persistence/file/AsyncFileIOFactory.java    |    66 +
 .../cache/persistence/file/FileIODecorator.java |     1 -
 .../cache/persistence/file/FileIOFactory.java   |    25 +-
 .../cache/persistence/file/FilePageStore.java   |    47 +-
 .../persistence/file/FilePageStoreManager.java  |   124 +-
 .../cache/persistence/file/FilePageStoreV2.java |     4 +-
 .../file/FileVersionCheckingFactory.java        |     8 +-
 .../persistence/file/RandomAccessFileIO.java    |    48 +-
 .../file/RandomAccessFileIOFactory.java         |    14 +-
 .../filename/PdsConsistentIdProcessor.java      |   569 +
 .../persistence/filename/PdsFolderSettings.java |   138 +
 .../filename/PdsFoldersResolver.java            |    33 +
 .../persistence/freelist/FreeListImpl.java      |    37 +-
 .../cache/persistence/freelist/PagesList.java   |    42 +-
 .../freelist/io/PagesListMetaIO.java            |    21 +
 .../freelist/io/PagesListNodeIO.java            |    19 +-
 .../persistence/pagemem/PageMemoryImpl.java     |   272 +-
 .../persistence/pagemem/PagesWriteThrottle.java |   105 +
 .../snapshot/IgniteCacheSnapshotManager.java    |     9 +-
 .../persistence/snapshot/SnapshotOperation.java |    13 +-
 .../cache/persistence/tree/BPlusTree.java       |   179 +-
 .../cache/persistence/tree/io/BPlusIO.java      |    12 +
 .../cache/persistence/tree/io/BPlusMetaIO.java  |    12 +
 .../cache/persistence/tree/io/DataPageIO.java   |    46 +-
 .../cache/persistence/tree/io/IOVersions.java   |     3 +
 .../cache/persistence/tree/io/PageIO.java       |    52 +-
 .../cache/persistence/tree/io/PageMetaIO.java   |    14 +
 .../tree/io/PagePartitionCountersIO.java        |    27 +
 .../tree/io/PagePartitionMetaIO.java            |    19 +
 .../persistence/tree/io/TrackingPageIO.java     |    31 +
 .../wal/AbstractWalRecordsIterator.java         |   147 +-
 .../persistence/wal/ByteBufferExpander.java     |    27 +-
 .../cache/persistence/wal/FileInput.java        |    15 +-
 .../cache/persistence/wal/FileWALPointer.java   |     3 +
 .../wal/FileWriteAheadLogManager.java           |   961 +-
 .../cache/persistence/wal/RecordSerializer.java |    57 -
 .../SingleSegmentLogicalRecordsIterator.java    |   141 +
 .../wal/WalSegmentTailReachedException.java     |    37 +
 .../wal/reader/IgniteWalIteratorFactory.java    |   130 +-
 .../wal/reader/StandaloneGridKernalContext.java |   145 +-
 .../reader/StandaloneIgnitePluginProcessor.java |     6 +-
 .../reader/StandaloneNoopCommunicationSpi.java  |    83 +
 .../wal/reader/StandaloneNoopDiscoverySpi.java  |   127 +
 .../reader/StandaloneWalRecordsIterator.java    |   171 +-
 .../persistence/wal/record/HeaderRecord.java    |    13 +-
 .../persistence/wal/record/RecordTypes.java     |    69 +
 .../wal/serializer/RecordDataSerializer.java    |    59 +
 .../wal/serializer/RecordDataV1Serializer.java  |  1571 +++
 .../wal/serializer/RecordDataV2Serializer.java  |   122 +
 .../wal/serializer/RecordSerializer.java        |    63 +
 .../wal/serializer/RecordSerializerFactory.java |    71 +
 .../serializer/RecordSerializerFactoryImpl.java |   133 +
 .../wal/serializer/RecordV1Serializer.java      |  1670 +--
 .../wal/serializer/RecordV2Serializer.java      |   248 +
 .../wal/serializer/TxRecordSerializer.java      |   227 +
 .../persistence/wal/serializer/io/RecordIO.java |    60 +
 .../processors/cache/query/CacheQueryEntry.java |    58 +
 .../query/GridCacheDistributedQueryManager.java |    16 +-
 .../cache/query/GridCacheQueryAdapter.java      |    59 +-
 .../cache/query/GridCacheQueryManager.java      |   635 +-
 .../query/GridCacheQuerySqlMetadataJobV2.java   |   154 +
 .../query/GridCacheQuerySqlMetadataV2.java      |   101 +
 .../cache/query/GridCacheSqlMetadata.java       |     8 +
 .../cache/query/GridCacheSqlQuery.java          |    24 +
 .../cache/query/IgniteQueryErrorCode.java       |    50 +-
 .../cache/query/QueryEntityIndexDescriptor.java |     2 +
 .../cache/query/QueryEntityTypeDescriptor.java  |    36 +-
 .../cache/query/SqlFieldsQueryEx.java           |   158 +
 .../continuous/CacheContinuousQueryHandler.java |     9 +-
 .../cache/ratemetrics/HitRateMetrics.java       |     2 +-
 .../cache/store/CacheStoreManager.java          |    12 +-
 .../store/GridCacheStoreManagerAdapter.java     |   101 +-
 .../cache/store/GridCacheWriteBehindStore.java  |    10 +-
 .../cache/transactions/IgniteInternalTx.java    |     2 +-
 .../cache/transactions/IgniteTxAdapter.java     |    60 +-
 .../cache/transactions/IgniteTxHandler.java     |   241 +-
 .../IgniteTxImplicitSingleStateImpl.java        |     5 +-
 .../transactions/IgniteTxLocalAdapter.java      |    14 +-
 .../cache/transactions/IgniteTxLocalEx.java     |     7 +-
 .../cache/transactions/IgniteTxManager.java     |   100 +-
 .../cache/tree/AbstractDataInnerIO.java         |   109 +
 .../cache/tree/AbstractDataLeafIO.java          |   108 +
 .../cache/tree/AbstractPendingEntryInnerIO.java |    99 +
 .../cache/tree/AbstractPendingEntryLeafIO.java  |    98 +
 .../cache/tree/CacheDataRowStore.java           |    80 +
 .../processors/cache/tree/CacheDataTree.java    |   258 +
 .../cache/tree/CacheIdAwareDataInnerIO.java     |    48 +
 .../cache/tree/CacheIdAwareDataLeafIO.java      |    48 +
 .../tree/CacheIdAwarePendingEntryInnerIO.java   |    48 +
 .../tree/CacheIdAwarePendingEntryLeafIO.java    |    48 +
 .../processors/cache/tree/DataInnerIO.java      |    48 +
 .../processors/cache/tree/DataLeafIO.java       |    48 +
 .../internal/processors/cache/tree/DataRow.java |   105 +
 .../cache/tree/PendingEntriesTree.java          |   120 +
 .../cache/tree/PendingEntryInnerIO.java         |    48 +
 .../cache/tree/PendingEntryLeafIO.java          |    48 +
 .../processors/cache/tree/PendingRow.java       |    83 +
 .../processors/cache/tree/PendingRowIO.java     |    44 +
 .../processors/cache/tree/RowLinkIO.java        |    44 +
 .../processors/cache/tree/SearchRow.java        |    76 +
 .../IgniteCacheObjectProcessorImpl.java         |   164 -
 .../UserCacheObjectByteArrayImpl.java           |    59 +
 .../cacheobject/UserCacheObjectImpl.java        |    82 +
 .../cacheobject/UserKeyCacheObjectImpl.java     |   101 +
 .../closure/GridClosureProcessor.java           |    36 +-
 .../cluster/ChangeGlobalStateFinishMessage.java |     9 +
 .../cluster/ChangeGlobalStateMessage.java       |    11 +-
 .../processors/cluster/ClusterProcessor.java    |    39 +-
 .../cluster/GridClusterStateProcessor.java      |    69 +-
 .../processors/cluster/GridUpdateNotifier.java  |   224 +-
 .../cluster/HttpIgniteUpdatesChecker.java       |    29 +-
 .../continuous/AbstractContinuousMessage.java   |    10 +
 .../continuous/GridContinuousProcessor.java     |   135 +-
 .../StartRoutineAckDiscoveryMessage.java        |     2 +-
 .../StartRoutineDiscoveryMessage.java           |     2 +-
 .../StopRoutineAckDiscoveryMessage.java         |     2 +-
 .../continuous/StopRoutineDiscoveryMessage.java |     2 +-
 .../datastreamer/DataStreamProcessor.java       |    57 +-
 .../datastreamer/DataStreamerImpl.java          |   171 +-
 .../AtomicDataStructureProxy.java               |   195 +
 .../datastructures/DataStructuresProcessor.java |    38 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   120 +-
 .../GridCacheAtomicReferenceImpl.java           |   114 +-
 .../GridCacheAtomicSequenceImpl.java            |   113 +-
 .../GridCacheAtomicStampedImpl.java             |   115 +-
 .../GridCacheCountDownLatchImpl.java            |    73 +-
 .../datastructures/GridCacheLockImpl.java       |    91 +-
 .../datastructures/GridCacheQueueAdapter.java   |     5 +
 .../datastructures/GridCacheRemovable.java      |     6 +
 .../datastructures/GridCacheSemaphoreImpl.java  |    85 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |    15 +-
 .../processors/hadoop/HadoopJobInfo.java        |     7 +
 .../processors/igfs/IgfsDataManager.java        |     4 +-
 .../processors/igfs/IgfsHelperImpl.java         |     8 +-
 .../internal/processors/igfs/IgfsImpl.java      |     4 +-
 .../processors/job/GridJobProcessor.java        |     8 +
 .../internal/processors/job/GridJobWorker.java  |    19 -
 .../processors/jobmetrics/GridJobMetrics.java   |    21 +-
 .../jobmetrics/GridJobMetricsProcessor.java     |     9 +-
 .../marshaller/MappingAcceptedMessage.java      |     9 +
 .../marshaller/MappingProposedMessage.java      |     9 +
 .../odbc/ClientConnectorConfigurationEx.java    |    27 +
 .../odbc/ClientListenerBufferedParser.java      |    81 +
 .../odbc/ClientListenerConnectionContext.java   |    62 +
 .../odbc/ClientListenerMessageParser.java       |    39 +
 .../odbc/ClientListenerNioListener.java         |   245 +
 .../odbc/ClientListenerNioServerBuffer.java     |   114 +
 .../odbc/ClientListenerProcessor.java           |   320 +
 .../odbc/ClientListenerProtocolVersion.java     |   116 +
 .../processors/odbc/ClientListenerRequest.java  |    31 +
 .../odbc/ClientListenerRequestHandler.java      |    49 +
 .../odbc/ClientListenerRequestNoId.java         |    28 +
 .../processors/odbc/ClientListenerResponse.java |    76 +
 .../odbc/SqlListenerBufferedParser.java         |    81 -
 .../odbc/SqlListenerConnectionContext.java      |    56 -
 .../odbc/SqlListenerMessageParser.java          |    39 -
 .../processors/odbc/SqlListenerNioListener.java |   269 -
 .../odbc/SqlListenerNioServerBuffer.java        |   114 -
 .../processors/odbc/SqlListenerProcessor.java   |   278 -
 .../odbc/SqlListenerProtocolVersion.java        |   116 -
 .../processors/odbc/SqlListenerRequest.java     |    43 -
 .../odbc/SqlListenerRequestHandler.java         |    39 -
 .../processors/odbc/SqlListenerResponse.java    |    76 -
 .../processors/odbc/SqlListenerUtils.java       |     6 +
 .../internal/processors/odbc/SqlStateCode.java  |    69 +
 .../odbc/jdbc/JdbcBatchExecuteRequest.java      |    20 +-
 .../odbc/jdbc/JdbcBatchExecuteResult.java       |     6 +
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |    85 +-
 .../processors/odbc/jdbc/JdbcColumnMetaV2.java  |    74 +
 .../odbc/jdbc/JdbcConnectionContext.java        |   132 +
 .../processors/odbc/jdbc/JdbcIndexMeta.java     |   192 +
 .../processors/odbc/jdbc/JdbcMessageParser.java |    12 +-
 .../odbc/jdbc/JdbcMetaColumnsRequest.java       |   102 +
 .../odbc/jdbc/JdbcMetaColumnsResult.java        |   125 +
 .../odbc/jdbc/JdbcMetaColumnsResultV2.java      |    50 +
 .../odbc/jdbc/JdbcMetaIndexesRequest.java       |    88 +
 .../odbc/jdbc/JdbcMetaIndexesResult.java        |    98 +
 .../odbc/jdbc/JdbcMetaParamsRequest.java        |    87 +
 .../odbc/jdbc/JdbcMetaParamsResult.java         |    97 +
 .../odbc/jdbc/JdbcMetaPrimaryKeysRequest.java   |    88 +
 .../odbc/jdbc/JdbcMetaPrimaryKeysResult.java    |    99 +
 .../odbc/jdbc/JdbcMetaSchemasRequest.java       |    73 +
 .../odbc/jdbc/JdbcMetaSchemasResult.java        |    73 +
 .../odbc/jdbc/JdbcMetaTablesRequest.java        |    87 +
 .../odbc/jdbc/JdbcMetaTablesResult.java         |    97 +
 .../processors/odbc/jdbc/JdbcParameterMeta.java |   163 +
 .../odbc/jdbc/JdbcPrimaryKeyMeta.java           |   131 +
 .../odbc/jdbc/JdbcQueryCloseRequest.java        |     4 +-
 ...dbcQueryExecuteMultipleStatementsResult.java |   134 +
 .../odbc/jdbc/JdbcQueryExecuteRequest.java      |    37 +-
 .../odbc/jdbc/JdbcQueryExecuteResult.java       |    12 +-
 .../odbc/jdbc/JdbcQueryFetchRequest.java        |     4 +-
 .../odbc/jdbc/JdbcQueryFetchResult.java         |    12 +-
 .../odbc/jdbc/JdbcQueryMetadataRequest.java     |    18 +-
 .../odbc/jdbc/JdbcQueryMetadataResult.java      |    14 +-
 .../processors/odbc/jdbc/JdbcRequest.java       |    71 +-
 .../odbc/jdbc/JdbcRequestHandler.java           |   519 +-
 .../processors/odbc/jdbc/JdbcResponse.java      |     4 +-
 .../processors/odbc/jdbc/JdbcResult.java        |    74 +-
 .../processors/odbc/jdbc/JdbcResultInfo.java    |    95 +
 .../processors/odbc/jdbc/JdbcStatementType.java |    45 +
 .../processors/odbc/jdbc/JdbcTableMeta.java     |    82 +
 .../processors/odbc/jdbc/JdbcUtils.java         |    37 +-
 .../odbc/odbc/OdbcConnectionContext.java        |   134 +
 .../processors/odbc/odbc/OdbcMessageParser.java |   109 +-
 .../odbc/odbc/OdbcQueryExecuteBatchRequest.java |    16 +-
 .../odbc/odbc/OdbcQueryExecuteBatchResult.java  |    34 +-
 .../odbc/odbc/OdbcQueryExecuteRequest.java      |    16 +-
 .../odbc/odbc/OdbcQueryExecuteResult.java       |    25 +-
 .../odbc/odbc/OdbcQueryMoreResultsRequest.java  |    61 +
 .../odbc/odbc/OdbcQueryMoreResultsResult.java   |    66 +
 .../processors/odbc/odbc/OdbcQueryResults.java  |   106 +
 .../processors/odbc/odbc/OdbcRequest.java       |     7 +-
 .../odbc/odbc/OdbcRequestHandler.java           |   342 +-
 .../processors/odbc/odbc/OdbcResponse.java      |     4 +-
 .../processors/odbc/odbc/OdbcResultSet.java     |   101 +
 .../processors/odbc/odbc/OdbcUtils.java         |    73 +
 .../platform/PlatformAbstractBootstrap.java     |    18 +-
 .../processors/platform/PlatformBootstrap.java  |     7 +-
 .../platform/PlatformContextImpl.java           |   125 +-
 .../processors/platform/PlatformIgnition.java   |     4 +-
 .../platform/cache/PlatformCache.java           |     4 +-
 .../query/PlatformAbstractQueryCursor.java      |     9 +-
 .../callback/PlatformCallbackGateway.java       |    15 +
 .../platform/callback/PlatformCallbackOp.java   |     3 +
 .../platform/client/ClientBooleanResponse.java  |    46 +
 .../client/ClientCloseableResource.java         |    28 +
 .../client/ClientConnectionContext.java         |   140 +
 .../platform/client/ClientIntResponse.java      |    46 +
 .../platform/client/ClientLongResponse.java     |    46 +
 .../platform/client/ClientMessageParser.java    |   367 +
 .../platform/client/ClientObjectResponse.java   |    46 +
 .../platform/client/ClientRawRequest.java       |    45 +
 .../platform/client/ClientRequest.java          |    61 +
 .../platform/client/ClientRequestHandler.java   |    63 +
 .../client/ClientResourceCloseRequest.java      |    46 +
 .../platform/client/ClientResourceRegistry.java |    98 +
 .../platform/client/ClientResponse.java         |    86 +
 .../platform/client/ClientStatus.java           |    48 +
 .../platform/client/ClientStringResponse.java   |    47 +
 .../platform/client/IgniteClientException.java  |    65 +
 .../binary/ClientBinaryTypeGetRequest.java      |    50 +
 .../binary/ClientBinaryTypeGetResponse.java     |    55 +
 .../binary/ClientBinaryTypeNameGetRequest.java  |    61 +
 .../binary/ClientBinaryTypeNamePutRequest.java  |    65 +
 .../binary/ClientBinaryTypePutRequest.java      |    56 +
 .../cache/ClientCacheClearKeyRequest.java       |    44 +
 .../cache/ClientCacheClearKeysRequest.java      |    44 +
 .../client/cache/ClientCacheClearRequest.java   |    44 +
 .../ClientCacheConfigurationSerializer.java     |   180 +
 .../cache/ClientCacheContainsKeyRequest.java    |    45 +
 .../cache/ClientCacheContainsKeysRequest.java   |    45 +
 ...ientCacheCreateWithConfigurationRequest.java |    58 +
 .../cache/ClientCacheCreateWithNameRequest.java |    56 +
 .../client/cache/ClientCacheDestroyRequest.java |    51 +
 .../cache/ClientCacheEntryQueryCursor.java      |    46 +
 .../cache/ClientCacheFieldsQueryCursor.java     |    53 +
 .../client/cache/ClientCacheGetAllRequest.java  |    46 +
 .../client/cache/ClientCacheGetAllResponse.java |    57 +
 .../ClientCacheGetAndPutIfAbsentRequest.java    |    45 +
 .../cache/ClientCacheGetAndPutRequest.java      |    45 +
 .../cache/ClientCacheGetAndRemoveRequest.java   |    45 +
 .../cache/ClientCacheGetAndReplaceRequest.java  |    45 +
 .../ClientCacheGetConfigurationRequest.java     |    47 +
 .../ClientCacheGetConfigurationResponse.java    |    51 +
 .../cache/ClientCacheGetNamesRequest.java       |    46 +
 .../cache/ClientCacheGetNamesResponse.java      |    56 +
 ...acheGetOrCreateWithConfigurationRequest.java |    58 +
 .../ClientCacheGetOrCreateWithNameRequest.java  |    49 +
 .../client/cache/ClientCacheGetRequest.java     |    45 +
 .../client/cache/ClientCacheGetSizeRequest.java |    57 +
 .../client/cache/ClientCacheKeyRequest.java     |    48 +
 .../cache/ClientCacheKeyValueRequest.java       |    48 +
 .../client/cache/ClientCacheKeysRequest.java    |    68 +
 .../client/cache/ClientCachePutAllRequest.java  |    57 +
 .../cache/ClientCachePutIfAbsentRequest.java    |    45 +
 .../client/cache/ClientCachePutRequest.java     |    44 +
 .../client/cache/ClientCacheQueryCursor.java    |   141 +
 .../cache/ClientCacheQueryNextPageRequest.java  |    49 +
 .../cache/ClientCacheQueryNextPageResponse.java |    50 +
 .../client/cache/ClientCacheQueryResponse.java  |    52 +
 .../cache/ClientCacheRemoveAllRequest.java      |    44 +
 .../cache/ClientCacheRemoveIfEqualsRequest.java |    45 +
 .../cache/ClientCacheRemoveKeyRequest.java      |    45 +
 .../cache/ClientCacheRemoveKeysRequest.java     |    44 +
 .../ClientCacheReplaceIfEqualsRequest.java      |    50 +
 .../client/cache/ClientCacheReplaceRequest.java |    45 +
 .../client/cache/ClientCacheRequest.java        |   122 +
 .../cache/ClientCacheScanQueryRequest.java      |   142 +
 .../cache/ClientCacheSqlFieldsQueryRequest.java |   126 +
 .../ClientCacheSqlFieldsQueryResponse.java      |    75 +
 .../cache/ClientCacheSqlQueryRequest.java       |    79 +
 .../platform/cluster/PlatformClusterGroup.java  |    90 +
 .../cluster/PlatformClusterNodeFilterImpl.java  |     7 +
 .../datastreamer/PlatformDataStreamer.java      |    17 +-
 .../dotnet/PlatformDotNetBootstrap.java         |    26 +-
 .../PlatformDotNetConfigurationClosure.java     |    22 +-
 .../dotnet/PlatformDotNetServiceImpl.java       |     7 +
 .../platform/events/PlatformEvents.java         |    17 +-
 .../events/PlatformLocalEventListener.java      |    88 +
 .../services/PlatformAbstractService.java       |     8 +
 .../platform/services/PlatformServices.java     |   178 +-
 .../utils/PlatformConfigurationUtils.java       |   358 +-
 .../platform/utils/PlatformUtils.java           |   206 +-
 .../processors/query/GridQueryIndexing.java     |    74 +-
 .../processors/query/GridQueryProcessor.java    |   461 +-
 .../processors/query/GridQueryProperty.java     |     7 +
 .../query/GridQueryTypeDescriptor.java          |    17 +
 .../processors/query/IgniteSQLException.java    |    84 +-
 .../processors/query/QueryEntityEx.java         |    98 +
 .../internal/processors/query/QueryField.java   |    76 +
 .../internal/processors/query/QuerySchema.java  |    74 +-
 .../query/QueryTypeDescriptorImpl.java          |    88 +-
 .../internal/processors/query/QueryUtils.java   |   138 +-
 .../messages/GridQueryNextPageResponse.java     |    36 +-
 .../query/property/QueryBinaryProperty.java     |    13 +-
 .../query/property/QueryClassProperty.java      |    16 +-
 .../query/schema/SchemaExchangeWorkerTask.java  |     5 +
 .../query/schema/SchemaIndexCacheFilter.java    |    33 +
 .../schema/SchemaIndexCacheVisitorClosure.java  |    14 +-
 .../schema/SchemaIndexCacheVisitorImpl.java     |   221 +-
 .../SchemaNodeLeaveExchangeWorkerTask.java      |     5 +
 .../query/schema/SchemaOperationWorker.java     |     2 +-
 .../message/SchemaAbstractDiscoveryMessage.java |    10 +
 .../message/SchemaProposeDiscoveryMessage.java  |     3 +-
 .../SchemaAbstractAlterTableOperation.java      |    39 +
 .../SchemaAlterTableAddColumnOperation.java     |    96 +
 .../operation/SchemaIndexCreateOperation.java   |    16 +-
 .../processors/rest/GridRestCommand.java        |     2 +-
 .../processors/rest/GridRestProcessor.java      |     5 +-
 .../handlers/cache/GridCacheCommandHandler.java |    12 +-
 .../GridServiceDeploymentCompoundFuture.java    |    94 +
 .../service/GridServiceProcessor.java           |   582 +-
 .../service/PreparedConfigurations.java         |    47 +
 .../session/GridTaskSessionProcessor.java       |     6 +
 .../processors/task/GridTaskProcessor.java      |    15 +-
 .../task/GridTaskThreadContextKey.java          |     3 +
 .../processors/task/GridTaskWorker.java         |    32 +-
 .../timeout/GridTimeoutProcessor.java           |    23 +-
 .../apache/ignite/internal/sql/SqlKeyword.java  |   243 +
 .../apache/ignite/internal/sql/SqlLexer.java    |   219 +
 .../internal/sql/SqlLexerLookAheadToken.java    |    75 +
 .../ignite/internal/sql/SqlLexerToken.java      |    48 +
 .../ignite/internal/sql/SqlLexerTokenType.java  |   112 +
 .../ignite/internal/sql/SqlParseException.java  |    99 +
 .../apache/ignite/internal/sql/SqlParser.java   |   181 +
 .../ignite/internal/sql/SqlParserUtils.java     |   369 +
 .../ignite/internal/sql/command/SqlCommand.java |    43 +
 .../sql/command/SqlCreateIndexCommand.java      |   304 +
 .../sql/command/SqlDropIndexCommand.java        |    80 +
 .../internal/sql/command/SqlIndexColumn.java    |    61 +
 .../internal/sql/command/SqlQualifiedName.java  |    70 +
 .../internal/util/GridSpinReadWriteLock.java    |     2 +-
 .../apache/ignite/internal/util/GridUnsafe.java |    14 +
 .../apache/ignite/internal/util/IgniteTree.java |    12 +
 .../ignite/internal/util/IgniteUtils.java       |   211 +-
 .../util/Jsr310Java8DateTimeApiUtils.java       |    80 +
 .../apache/ignite/internal/util/MpscQueue.java  |   240 +
 .../internal/util/OffheapReadWriteLock.java     |     8 +-
 .../ignite/internal/util/StripedExecutor.java   |   304 +-
 .../util/future/GridCompoundFuture.java         |    55 +-
 .../internal/util/future/GridFutureAdapter.java |    16 +
 .../ignite/internal/util/lang/GridFunc.java     |     3 +-
 .../internal/util/lang/GridTreePrinter.java     |    19 +-
 .../nio/GridAbstractCommunicationClient.java    |     2 +-
 .../internal/util/nio/GridNioKeyAttachment.java |    33 +
 .../util/nio/GridNioRecoveryDescriptor.java     |     3 +-
 .../ignite/internal/util/nio/GridNioServer.java |   256 +-
 .../util/nio/GridSelectorNioSessionImpl.java    |    28 +-
 .../internal/util/nio/ssl/GridNioSslFilter.java |    12 +-
 .../offheap/unsafe/GridOffHeapSmartPointer.java |    38 -
 .../unsafe/GridOffHeapSmartPointerFactory.java  |    29 -
 .../offheap/unsafe/GridOffHeapSnapTreeMap.java  |  4525 -------
 .../unsafe/GridUnsafeCompoundMemory.java        |    35 -
 .../util/offheap/unsafe/GridUnsafeGuard.java    |   376 -
 .../apache/ignite/internal/util/typedef/X.java  |     4 +-
 .../internal/visor/VisorDataTransferObject.java |     6 +
 .../ignite/internal/visor/cache/VisorCache.java |    94 +-
 .../cache/VisorCacheAggregatedMetrics.java      |    24 +-
 .../visor/cache/VisorCacheConfiguration.java    |    17 +-
 .../cache/VisorCacheEvictionConfiguration.java  |     4 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |     5 +-
 .../visor/cache/VisorCacheModifyTask.java       |   113 +
 .../visor/cache/VisorCacheModifyTaskArg.java    |   114 +
 .../visor/cache/VisorCacheModifyTaskResult.java |   101 +
 .../cache/VisorCacheNearConfiguration.java      |     7 +-
 .../visor/cache/VisorCacheStopTask.java         |    22 +-
 .../visor/cache/VisorCacheStopTaskArg.java      |    27 +
 .../visor/cache/VisorMemoryMetrics.java         |     6 +-
 .../visor/cache/VisorModifyCacheMode.java       |    47 +
 .../visor/file/VisorLatestTextFilesTask.java    |    13 +-
 .../internal/visor/log/VisorLogSearchTask.java  |    25 +-
 .../node/VisorClientConnectorConfiguration.java |   166 +
 .../node/VisorDataRegionConfiguration.java      |   238 +
 .../node/VisorDataStorageConfiguration.java     |   452 +
 .../node/VisorExecutorServiceConfiguration.java |    18 +-
 .../visor/node/VisorGridConfiguration.java      |    57 +-
 .../visor/node/VisorMemoryConfiguration.java    |    26 +-
 .../node/VisorMemoryPolicyConfiguration.java    |    10 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |    13 +-
 .../node/VisorNodeDataCollectorJobResult.java   |     4 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |     2 +-
 .../node/VisorNodeDataCollectorTaskArg.java     |    47 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |    35 +-
 .../visor/node/VisorPersistenceMetrics.java     |     8 +-
 .../node/VisorPersistentStoreConfiguration.java |    21 +-
 .../node/VisorSqlConnectorConfiguration.java    |     3 +
 .../internal/visor/query/VisorQueryTask.java    |    21 +-
 .../internal/visor/query/VisorQueryTaskArg.java |    41 +-
 .../internal/visor/query/VisorQueryUtils.java   |    25 +-
 .../internal/visor/util/VisorTaskUtils.java     |   196 +-
 .../org/apache/ignite/logger/NullLogger.java    |     6 +
 .../apache/ignite/logger/java/JavaLogger.java   |    17 +
 .../ignite/mxbean/CacheGroupMetricsMXBean.java  |   142 +
 .../ignite/mxbean/CacheMetricsMXBean.java       |    10 +-
 .../mxbean/ClusterLocalNodeMetricsMXBean.java   |   253 -
 .../ignite/mxbean/ClusterMetricsMXBean.java     |   317 +
 .../ignite/mxbean/DataRegionMetricsMXBean.java  |   143 +
 .../ignite/mxbean/DataStorageMetricsMXBean.java |   121 +
 .../org/apache/ignite/mxbean/IgniteMXBean.java  |    21 +
 .../ignite/mxbean/MemoryMetricsMXBean.java      |     2 +
 .../ignite/mxbean/PersistenceMetricsMXBean.java |     2 +
 .../services/ServiceDeploymentException.java    |    77 +
 .../tcp/TcpCommunicationMetricsListener.java    |   224 +
 .../communication/tcp/TcpCommunicationSpi.java  |  1008 +-
 .../tcp/TcpCommunicationSpiMBean.java           |    34 +
 .../tcp/messages/HandshakeMessage.java          |   157 +
 .../tcp/messages/HandshakeMessage2.java         |    95 +
 .../tcp/messages/NodeIdMessage.java             |   128 +
 .../messages/RecoveryLastReceivedMessage.java   |   114 +
 .../tcp/messages/package-info.java              |    22 +
 .../spi/discovery/IgniteDiscoveryThread.java    |    23 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    66 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   439 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |     4 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    54 +-
 .../tcp/internal/TcpDiscoveryNode.java          |    35 +
 .../sharedfs/TcpDiscoverySharedFsIpFinder.java  |    34 +-
 .../spi/indexing/IndexingQueryCacheFilter.java  |    82 +
 .../spi/indexing/IndexingQueryFilter.java       |    12 +-
 .../spi/indexing/IndexingQueryFilterImpl.java   |    79 +
 .../org/apache/ignite/thread/IgniteThread.java  |    11 +-
 .../resources/META-INF/classnames.properties    |   156 +-
 .../core/src/main/resources/ignite.properties   |     2 +-
 .../core/src/test/config/examples.properties    |     2 +-
 .../core/src/test/config/spark/spark-config.xml |     8 +-
 modules/core/src/test/config/tests.properties   |     9 +
 .../org.apache.ignite.plugin.PluginProvider     |     1 -
 ...atformCachePluginConfigurationClosureFactory |     1 -
 ...rm.PlatformPluginConfigurationClosureFactory |     1 -
 .../ignite/cache/LargeEntryUpdateTest.java      |     6 +-
 ...istenerRWThroughDisabledAtomicCacheTest.java |    33 +
 ...RWThroughDisabledTransactionalCacheTest.java |   138 +
 .../store/CacheStoreReadFromBackupTest.java     |   238 +
 ...ionListenerReadWriteThroughDisabledTest.java |   291 +
 ...reSessionListenerWriteBehindEnabledTest.java |   405 +
 ...cheTransactionalStoreReadFromBackupTest.java |    32 +
 .../cache/store/GridStoreLoadCacheTest.java     |   120 +
 .../internal/ClusterNodeMetricsSelfTest.java    |   188 +-
 .../ignite/internal/GridDiscoverySelfTest.java  |    30 -
 .../internal/GridJobStealingSelfTest.java       |     7 +-
 .../GridMultithreadedJobStealingSelfTest.java   |   176 +-
 ...entConnectAfterCommunicationFailureTest.java |   156 +
 .../IgniteClientReconnectBinaryContexTest.java  |   115 +
 .../IgniteClientReconnectCacheTest.java         |    13 +-
 .../IgniteClientReconnectCollectionsTest.java   |   131 +-
 .../IgniteSlowClientDetectionSelfTest.java      |     6 +-
 .../internal/TestDelayingCommunicationSpi.java  |    63 +
 .../internal/TestRecordingCommunicationSpi.java |    12 +
 .../binary/BinaryMarshallerSelfTest.java        |   154 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |    48 +
 .../IgniteCommunicationBalanceTest.java         |    13 +
 .../IgniteCommunicationSslBalanceTest.java      |    28 +
 .../OptimizedObjectStreamSelfTest.java          |   108 +-
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |    18 +-
 .../AtomicCacheAffinityConfigurationTest.java   |     7 +-
 .../cache/CacheClientStoreSelfTest.java         |     4 +-
 .../processors/cache/CacheComparatorTest.java   |    48 +
 .../cache/CacheConfigurationLeakTest.java       |    15 +-
 .../cache/CacheDataRegionConfigurationTest.java |   172 +
 ...CacheExchangeMessageDuplicatedStateTest.java |    36 +-
 .../cache/CacheGroupMetricsMBeanTest.java       |   266 +
 .../cache/CacheGroupsMetricsRebalanceTest.java  |   167 +-
 .../CacheMemoryPolicyConfigurationTest.java     |   172 -
 .../CacheMetricsForClusterGroupSelfTest.java    |   119 +-
 .../cache/CacheRebalancingSelfTest.java         |    91 +-
 .../cache/CacheStopAndDestroySelfTest.java      |    10 +-
 .../processors/cache/CacheTxFastFinishTest.java |     9 +-
 .../cache/CacheValidatorMetricsTest.java        |   121 +
 .../cache/ClusterStateAbstractTest.java         |   129 +-
 .../cache/ConcurrentCacheStartTest.java         |    67 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |    45 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    |    17 +-
 ...idCacheConfigurationConsistencySelfTest.java |    52 +
 ...ridCacheQuerySqlFieldInlineSizeSelfTest.java |   160 +
 .../processors/cache/GridCacheTestEntryEx.java  |    14 +-
 ...StorageConfigurationConsistencySelfTest.java |    79 +
 .../cache/IgniteCacheDynamicStopSelfTest.java   |     2 +-
 .../IgniteCacheTopologySafeGetSelfTest.java     |   222 -
 ...IgniteClientCacheInitializationFailTest.java |    43 +-
 .../IgniteClientCacheStartFailoverTest.java     |     4 +-
 .../IgniteClusterActivateDeactivateTest.java    |    36 +-
 ...erActivateDeactivateTestWithPersistence.java |    30 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |    26 +-
 .../IgniteMarshallerCacheFSRestoreTest.java     |    71 +-
 .../cache/IgniteOutOfMemoryPropagationTest.java |   251 +
 ...niteTopologyValidatorGridSplitCacheTest.java |   547 +-
 .../cache/IgniteTxConfigCacheSelfTest.java      |    14 +
 .../cache/MemoryPolicyConfigValidationTest.java |    24 +-
 ...AffinityCoordinatorDynamicStartStopTest.java |    17 +-
 ...heapCacheMetricsForClusterGroupSelfTest.java |    19 +-
 .../binary/BinaryMetadataUpdatesFlowTest.java   |    48 +
 ...eAbstractDataStructuresFailoverSelfTest.java |     7 +-
 .../GridCacheQueueClientDisconnectTest.java     |   117 +
 ...ionedAtomicQueueCreateMultiNodeSelfTest.java |     4 -
 ...PartitionedQueueCreateMultiNodeSelfTest.java |     2 -
 ...eplicatedDataStructuresFailoverSelfTest.java |    14 +
 .../distributed/Cache64kPartitionsTest.java     |    94 +
 .../distributed/CacheExchangeMergeTest.java     |  1528 +++
 .../CacheLateAffinityAssignmentTest.java        |   609 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java |    11 +
 .../CacheLockReleaseNodeLeaveTest.java          |    13 +-
 .../distributed/CachePartitionStateTest.java    |    18 +-
 .../cache/distributed/CacheStartOnJoinTest.java |     9 +-
 ...ncurrentGridStartSelfTestAllowOverwrite.java |    33 +
 ...titionEvictionDuringReadThroughSelfTest.java |     2 +
 .../distributed/IgniteCache150ClientsTest.java  |     4 +-
 ...niteCacheClientNodeChangingTopologyTest.java |     5 +-
 ...teCacheClientNodePartitionsExchangeTest.java |    52 +-
 .../IgniteCacheMessageWriteTimeoutTest.java     |     4 +-
 .../IgniteCacheSystemTransactionsSelfTest.java  |    18 +
 .../IgniteCacheThreadLocalTxTest.java           |   223 +
 .../IgniteOptimisticTxSuspendResumeTest.java    |    17 +-
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |   111 +-
 .../IgniteCacheTopologySplitAbstractTest.java   |   266 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |    36 +-
 ...ionedMultiNodeLongTxTimeout2FullApiTest.java |    34 +
 .../near/GridCacheNearClientHitTest.java        |   154 +
 .../GridCacheNearEvictionEventSelfTest.java     |     5 -
 .../EvictionPolicyFactoryAbstractTest.java      |  1073 ++
 .../fifo/FifoEvictionPolicyFactorySelfTest.java |   261 +
 .../lru/LruEvictionPolicyFactorySelfTest.java   |   352 +
 .../paged/PageEvictionAbstractTest.java         |    25 +-
 .../paged/PageEvictionDataStreamerTest.java     |    65 +
 .../PageEvictionMultinodeMixedRegionsTest.java  |    59 +
 .../paged/PageEvictionMultinodeTest.java        |     6 +-
 .../SortedEvictionPolicyFactorySelfTest.java    |   264 +
 .../expiry/IgniteCacheLargeValueExpireTest.java |     6 +-
 .../local/GridCacheLocalFullApiSelfTest.java    |    76 +-
 .../cache/persistence/DummyPageIO.java          |    41 +
 .../IgniteDataStorageMetricsSelfTest.java       |   237 +
 ...tePdsBinaryMetadataOnClusterRestartTest.java |    10 +-
 .../IgnitePdsBinarySortObjectFieldsTest.java    |   150 +
 ...gnitePdsCacheAssignmentNodeRestartsTest.java |   257 +
 .../IgnitePdsCacheRebalancingAbstractTest.java  |   167 +-
 .../IgnitePdsClientNearCachePutGetTest.java     |    12 +-
 .../IgnitePdsContinuousRestartTest.java         |    70 +-
 ...ousRestartTestWithSharedGroupAndIndexes.java |    67 +
 .../persistence/IgnitePdsDynamicCacheTest.java  |    75 +-
 .../IgnitePdsExchangeDuringCheckpointTest.java  |   136 +
 ...MarshallerMappingRestoreOnNodeStartTest.java |   116 +
 .../IgnitePdsMultiNodePutGetRestartTest.java    |    30 +-
 .../persistence/IgnitePdsPageSizesTest.java     |    35 +-
 ...gnitePdsRecoveryAfterFileCorruptionTest.java |   126 +-
 .../IgnitePdsRemoveDuringRebalancingTest.java   |    45 +-
 ...gnitePdsSingleNodePutGetPersistenceTest.java |    12 +-
 .../IgnitePersistenceMetricsSelfTest.java       |   225 -
 ...nitePersistenceSequentialCheckpointTest.java |    44 +
 .../IgnitePersistentStoreCacheGroupsTest.java   |    49 +-
 ...IgnitePersistentStoreDataStructuresTest.java |    23 +-
 .../MemoryPolicyInitializationTest.java         |    24 +-
 .../db/IgnitePdsCacheRestoreTest.java           |    45 +-
 .../db/IgnitePdsMultiNodePutGetRestartTest.java |    29 +-
 ...PdsPageEvictionDuringPartitionClearTest.java |    35 +-
 .../db/IgnitePdsPageEvictionTest.java           |    36 +-
 ...tePdsRebalancingOnNotStableTopologyTest.java |    34 +-
 .../db/IgnitePdsTransactionsHangTest.java       |    31 +-
 .../db/IgnitePdsWholeClusterRestartTest.java    |    32 +-
 ...faultPageSizeBackwardsCompatibilityTest.java |   146 +
 .../db/file/IgnitePdsCacheIntegrationTest.java  |    35 +-
 ...ckpointSimulationWithRealCpDisabledTest.java |    86 +-
 .../db/file/IgnitePdsEvictionTest.java          |    62 +-
 .../file/IgnitePdsNoActualWalHistoryTest.java   |    28 +-
 .../file/IgnitePdsThreadInterruptionTest.java   |   186 +
 .../IgniteUidAsConsistentIdMigrationTest.java   |   708 ++
 .../persistence/db/wal/IgnitePdsWalTlbTest.java |    42 +-
 .../wal/IgniteWalFlushBackgroundSelfTest.java   |    35 +
 .../db/wal/IgniteWalFlushDefaultSelfTest.java   |    35 +
 .../db/wal/IgniteWalFlushFailoverTest.java      |    68 +-
 .../db/wal/IgniteWalFlushLogOnlySelfTest.java   |    35 +
 ...lFlushMultiNodeFailoverAbstractSelfTest.java |   246 +
 .../wal/IgniteWalHistoryReservationsTest.java   |    83 +-
 .../db/wal/IgniteWalRecoveryPPCTest.java        |   321 +
 .../IgniteWalRecoverySeveralRestartsTest.java   |    35 +-
 .../db/wal/IgniteWalRecoveryTest.java           |   310 +-
 .../IgniteWalRecoveryWithCompactionTest.java    |    33 +
 .../db/wal/IgniteWalSerializerVersionTest.java  |   318 +
 .../persistence/db/wal/WalCompactionTest.java   |   312 +
 .../db/wal/WalRecoveryTxLogicalRecordsTest.java |    24 +-
 .../db/wal/crc/IgniteDataIntegrityTests.java    |    97 +-
 .../db/wal/reader/IgniteWalReaderTest.java      |  1167 +-
 .../db/wal/reader/IndexedObject.java            |    79 +
 .../db/wal/reader/MockWalIteratorFactory.java   |    50 +-
 .../pagemem/BPlusTreePageMemoryImplTest.java    |     8 +-
 .../BPlusTreeReuseListPageMemoryImplTest.java   |     7 +-
 .../MetadataStoragePageMemoryImplTest.java      |     8 +-
 .../pagemem/NoOpPageStoreManager.java           |     2 +-
 .../persistence/pagemem/NoOpWALManager.java     |     5 +
 .../pagemem/PageMemoryImplNoLoadTest.java       |     8 +-
 .../persistence/pagemem/PageMemoryImplTest.java |     8 +-
 .../pagemem/PagesWriteThrottleSandboxTest.java  |   260 +
 .../pagemem/PagesWriteThrottleSmokeTest.java    |   322 +
 .../AbstractNodeJoinTemplate.java               |    13 +-
 .../IgniteChangeGlobalStateAbstractTest.java    |    31 +-
 .../IgniteChangeGlobalStateServiceTest.java     |     2 +
 .../IgniteChangeGlobalStateTest.java            |    11 +-
 .../IgniteStandByClusterTest.java               |    66 +-
 .../extended/GridActivateExtensionTest.java     |    34 +-
 .../join/JoinInActiveNodeToActiveCluster.java   |     4 +-
 ...gniteAbstractStandByClientReconnectTest.java |    14 +-
 .../GridCacheQueryTransformerSelfTest.java      |    41 +
 ...ContinuousQueryFailoverAbstractSelfTest.java |    11 +-
 ...niteCacheContinuousQueryBackupQueueTest.java |     6 +-
 .../AbstractDeadlockDetectionTest.java          |   246 +
 .../cache/transactions/TxDeadlockCauseTest.java |    15 +-
 .../TxOptimisticDeadlockDetectionTest.java      |   322 +-
 .../TxPessimisticDeadlockDetectionTest.java     |   244 +-
 .../TxRollbackOnTimeoutNearCacheTest.java       |    28 +
 ...ollbackOnTimeoutNoDeadlockDetectionTest.java |    47 +
 .../transactions/TxRollbackOnTimeoutTest.java   |   839 ++
 .../closure/GridClosureProcessorRemoteTest.java |    97 +-
 .../cluster/GridUpdateNotifierSelfTest.java     |    50 +-
 .../processors/database/BPlusTreeSelfTest.java  |   866 +-
 .../database/DataRegionMetricsSelfTest.java     |   348 +
 .../database/FreeListImplSelfTest.java          |    16 +-
 .../database/IgniteDbAbstractTest.java          |    21 +-
 .../database/IgniteDbDynamicCacheSelfTest.java  |    18 +-
 .../IgniteDbMemoryLeakAbstractTest.java         |    14 +-
 .../database/MemoryMetricsSelfTest.java         |   348 -
 .../database/MetadataStorageSelfTest.java       |     8 +-
 .../database/SwapPathConstructionSelfTest.java  |    28 +-
 .../datastreamer/DataStreamerImplSelfTest.java  |   203 +-
 .../processors/igfs/IgfsIgniteMock.java         |    35 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |    12 +-
 .../OdbcConfigurationValidationSelfTest.java    |     1 -
 ...ServiceDeploymentCompoundFutureSelfTest.java |   127 +
 ...GridServiceProcessorBatchDeploySelfTest.java |   671 +
 .../GridServiceProcessorMultiNodeSelfTest.java  |    71 +-
 .../internal/sql/SqlParserAbstractSelfTest.java |    46 +
 .../sql/SqlParserCreateIndexSelfTest.java       |   261 +
 .../sql/SqlParserDropIndexSelfTest.java         |    99 +
 .../internal/util/nio/GridNioSelfTest.java      |     2 +-
 .../unsafe/GridOffheapSnapTreeSelfTest.java     |   326 -
 .../unsafe/GridUnsafeMemoryPerformanceTest.java |    65 -
 .../unsafe/GridUnsafeMemorySelfTest.java        |   192 -
 .../ignite/logger/java/JavaLoggerTest.java      |     5 +
 .../p2p/DeploymentClassLoaderCallableTest.java  |   144 +
 .../apache/ignite/p2p/SharedDeploymentTest.java |   128 +
 .../platform/PlatformCacheWriteMetricsTask.java |    31 +-
 .../ignite/platform/PlatformExceptionTask.java  |     8 +
 .../platform/plugin/PlatformTestPlugin.java     |    27 -
 .../plugin/PlatformTestPluginConfiguration.java |    63 -
 ...rmTestPluginConfigurationClosureFactory.java |    61 -
 .../plugin/PlatformTestPluginException.java     |    34 -
 .../plugin/PlatformTestPluginExtension.java     |    51 -
 .../plugin/PlatformTestPluginProvider.java      |   103 -
 .../plugin/PlatformTestPluginTarget.java        |   236 -
 .../cache/PlatformGetCachePluginsTask.java      |    85 -
 .../PlatformTestCachePluginConfiguration.java   |    55 -
 ...formTestCachePluginConfigurationClosure.java |    48 -
 ...tCachePluginConfigurationClosureFactory.java |    37 -
 .../cache/PlatformTestCachePluginProvider.java  |    72 -
 .../spi/GridTcpSpiForwardingSelfTest.java       |     1 +
 .../GridAbstractCommunicationSelfTest.java      |    27 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |    28 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |    39 +-
 ...GridTcpCommunicationSpiRecoverySelfTest.java |    47 +-
 ...CommunicationRecoveryAckClosureSelfTest.java |    36 +-
 .../tcp/TcpCommunicationSpiDropNodesTest.java   |     3 +-
 ...ommunicationSpiHalfOpenedConnectionTest.java |   142 +
 .../tcp/TcpCommunicationStatisticsTest.java     |   191 +
 ...ClusterMetricsSnapshotSerializeSelfTest.java |     8 +-
 .../IgniteDiscoveryCacheReuseSelfTest.java      |    96 +
 .../discovery/tcp/IgniteClientConnectTest.java  |   163 +
 ...pClientDiscoveryMarshallerCheckSelfTest.java |    84 +-
 ...lientDiscoverySpiFailureTimeoutSelfTest.java |    20 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |   275 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   110 +
 .../tcp/TcpDiscoverySslTrustedSelfTest.java     |    42 +
 .../TcpDiscoverySslTrustedUntrustedTest.java    |   140 +
 .../tcp/TcpDiscoveryWithWrongServerTest.java    |   336 +
 .../ignite/testframework/GridTestUtils.java     |    64 +-
 .../ignite/testframework/junits/IgniteMock.java |    25 +-
 .../junits/common/GridCommonAbstractTest.java   |    22 +-
 .../junits/logger/GridTestLog4jLogger.java      |    21 +-
 .../junits/multijvm/IgniteNodeRunner.java       |     2 +-
 .../junits/multijvm/IgniteProcessProxy.java     |   125 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |     9 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |     6 +-
 .../IgniteCacheEvictionSelfTestSuite.java       |    11 +
 .../IgniteCacheFailoverTestSuite.java           |     2 -
 .../IgniteCacheFullApiSelfTestSuite.java        |     2 +
 .../IgniteCacheMetricsSelfTestSuite.java        |     5 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    13 +-
 .../testsuites/IgniteCacheTestSuite2.java       |    17 +-
 .../testsuites/IgniteCacheTestSuite4.java       |     8 +-
 .../testsuites/IgniteCacheTestSuite5.java       |     7 +-
 .../testsuites/IgniteCacheTestSuite6.java       |    19 +
 .../IgniteClientReconnectTestSuite.java         |     6 +-
 .../testsuites/IgniteKernalSelfTestSuite.java   |     4 +
 .../testsuites/IgniteP2PSelfTestSuite.java      |     4 +
 .../ignite/testsuites/IgnitePdsTestSuite.java   |     7 +
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |    30 +-
 .../testsuites/IgniteReproducingSuite.java      |    42 +
 .../IgniteSpiCommunicationSelfTestSuite.java    |     5 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |    17 +
 .../IgniteSpiIndexingSelfTestSuite.java         |    38 -
 .../ignite/testsuites/IgniteSpiTestSuite.java   |     3 -
 modules/dev-utils/pom.xml                       |    63 +
 .../development/utils/IgniteWalConverter.java   |    85 +
 .../ignite/development/utils/package-info.java  |    22 +
 modules/docker/2.2.0/Dockerfile                 |    46 +
 modules/docker/2.2.0/run.sh                     |    51 +
 modules/docker/2.3.0/Dockerfile                 |    46 +
 modules/docker/2.3.0/run.sh                     |    51 +
 modules/docker/Dockerfile                       |     4 +-
 modules/extdata/p2p/pom.xml                     |     2 +-
 .../tests/p2p/compute/ExternalCallable.java     |    59 +
 .../tests/p2p/compute/ExternalCallable1.java    |    59 +
 .../tests/p2p/compute/ExternalCallable2.java    |    59 +
 modules/extdata/platform/README.txt             |     4 +
 .../extdata/platform/licenses/apache-2.0.txt    |   202 +
 modules/extdata/platform/pom.xml                |    46 +
 .../platform/plugin/PlatformTestPlugin.java     |    27 +
 .../plugin/PlatformTestPluginConfiguration.java |    63 +
 ...rmTestPluginConfigurationClosureFactory.java |    61 +
 .../plugin/PlatformTestPluginException.java     |    34 +
 .../plugin/PlatformTestPluginExtension.java     |    51 +
 .../plugin/PlatformTestPluginProvider.java      |   103 +
 .../plugin/PlatformTestPluginTarget.java        |   236 +
 .../cache/PlatformGetCachePluginsTask.java      |    85 +
 .../PlatformTestCachePluginConfiguration.java   |    55 +
 ...formTestCachePluginConfigurationClosure.java |    48 +
 ...tCachePluginConfigurationClosureFactory.java |    37 +
 .../cache/PlatformTestCachePluginProvider.java  |    72 +
 .../platform/plugin/cache/package-info.java     |    22 +
 .../ignite/platform/plugin/package-info.java    |    22 +
 .../org.apache.ignite.plugin.PluginProvider     |     1 +
 ...atformCachePluginConfigurationClosureFactory |     1 +
 ...rm.PlatformPluginConfigurationClosureFactory |     1 +
 .../extdata/uri/modules/uri-dependency/pom.xml  |     2 +-
 modules/extdata/uri/pom.xml                     |     2 +-
 .../tests/p2p/compute/ExternalCallable.java     |    59 +
 .../tests/p2p/compute/ExternalCallable1.java    |    59 +
 .../tests/p2p/compute/ExternalCallable2.java    |    59 +
 modules/flink/pom.xml                           |     2 +-
 modules/flume/pom.xml                           |     2 +-
 modules/gce/pom.xml                             |     2 +-
 modules/geospatial/pom.xml                      |     2 +-
 .../query/h2/opt/GridH2SpatialIndex.java        |    45 +-
 modules/hadoop/pom.xml                          |     2 +-
 .../processors/hadoop/impl/HadoopUtils.java     |    45 +-
 .../impl/fs/HadoopFileSystemCacheUtils.java     |    34 +-
 .../hadoop/impl/proto/HadoopClientProtocol.java |     8 +-
 .../hadoop/impl/v1/HadoopV1MapTask.java         |     6 +-
 .../hadoop/impl/v1/HadoopV1Reporter.java        |     8 +-
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |    32 +-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |    62 +-
 .../HadoopExternalCommunication.java            |     5 +-
 .../impl/HadoopAbstractMapReduceTest.java       |     2 +-
 .../hadoop/impl/HadoopGroupingTest.java         |     2 +-
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |     4 +-
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |     2 +-
 .../hadoop/impl/HadoopPlannerMockJob.java       |     6 +
 .../hadoop/impl/HadoopSortingTest.java          |     4 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |    10 +-
 .../hadoop/impl/HadoopTasksV1Test.java          |     2 +-
 .../hadoop/impl/HadoopTasksV2Test.java          |     2 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |     2 +-
 .../hadoop/impl/HadoopTxConfigCacheTest.java    |     4 +-
 .../hadoop/impl/HadoopV2JobSelfTest.java        |     2 +-
 .../collections/HadoopAbstractMapTest.java      |     6 +
 .../HadoopExternalTaskExecutionSelfTest.java    |     4 +-
 modules/hibernate-4.2/pom.xml                   |     2 +-
 modules/hibernate-5.1/pom.xml                   |     2 +-
 modules/hibernate-core/pom.xml                  |     2 +-
 .../cache/hibernate/HibernateCacheProxy.java    |     5 -
 modules/ignored-tests/pom.xml                   |    10 +-
 modules/indexing/pom.xml                        |    50 +-
 .../cache/query/CacheQueryPartitionInfo.java    |     1 +
 .../query/h2/DmlStatementsProcessor.java        |   644 +-
 .../internal/processors/query/h2/H2Cursor.java  |    42 +-
 .../processors/query/h2/H2DatabaseType.java     |     2 +-
 .../processors/query/h2/H2DmlPlanKey.java       |    21 +-
 .../query/h2/H2ResultSetIterator.java           |     3 +-
 .../processors/query/h2/H2RowDescriptor.java    |   491 -
 .../internal/processors/query/h2/H2Schema.java  |    41 +-
 .../processors/query/h2/H2TableDescriptor.java  |    23 +-
 .../processors/query/h2/H2TableEngine.java      |     8 +-
 .../query/h2/H2TwoStepCachedQueryKey.java       |     1 +
 .../internal/processors/query/h2/H2TypeKey.java |    64 +
 .../internal/processors/query/h2/H2Utils.java   |     3 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   841 +-
 .../processors/query/h2/UpdateResult.java       |    63 +
 .../query/h2/database/H2PkHashIndex.java        |    64 +-
 .../query/h2/database/H2RowFactory.java         |     8 +-
 .../processors/query/h2/database/H2Tree.java    |    41 +-
 .../query/h2/database/H2TreeIndex.java          |   225 +-
 .../query/h2/database/InlineIndexHelper.java    |     6 +-
 .../query/h2/database/io/H2ExtrasInnerIO.java   |    20 +-
 .../query/h2/database/io/H2ExtrasLeafIO.java    |    16 +-
 .../query/h2/database/io/H2InnerIO.java         |     4 +-
 .../query/h2/database/io/H2LeafIO.java          |     4 +-
 .../query/h2/database/util/CompareUtils.java    |   332 -
 .../query/h2/ddl/DdlStatementsProcessor.java    |   239 +-
 .../processors/query/h2/dml/DmlAstUtils.java    |   639 +
 .../processors/query/h2/dml/DmlBatchSender.java |   232 +
 .../query/h2/dml/DmlDistributedPlanInfo.java    |    56 +
 .../h2/dml/DmlPageProcessingErrorResult.java    |    76 +
 .../query/h2/dml/DmlPageProcessingResult.java   |    68 +
 .../processors/query/h2/dml/DmlUtils.java       |   118 +
 .../processors/query/h2/dml/FastUpdate.java     |   175 +
 .../processors/query/h2/dml/UpdatePlan.java     |   372 +-
 .../query/h2/dml/UpdatePlanBuilder.java         |   233 +-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |   454 -
 .../query/h2/opt/GridH2CollocationModel.java    |    18 +-
 .../query/h2/opt/GridH2IndexBase.java           |   213 +-
 .../query/h2/opt/GridH2KeyRowOnheap.java        |    63 +
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   328 -
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |   199 +-
 .../query/h2/opt/GridH2MetaTable.java           |    13 +-
 .../query/h2/opt/GridH2PlainRowFactory.java     |   181 +
 .../query/h2/opt/GridH2QueryContext.java        |     4 +
 .../processors/query/h2/opt/GridH2Row.java      |   134 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |   433 +-
 .../query/h2/opt/GridH2RowFactory.java          |   194 -
 .../query/h2/opt/GridH2SearchRowAdapter.java    |   103 +
 .../processors/query/h2/opt/GridH2Table.java    |   411 +-
 .../query/h2/opt/GridLuceneDirectory.java       |    15 +-
 .../query/h2/opt/GridLuceneIndex.java           |    53 +-
 .../query/h2/opt/GridSearchRowPointer.java      |    28 -
 .../processors/query/h2/sql/DmlAstUtils.java    |   602 -
 .../h2/sql/GridSqlAlterTableAddColumn.java      |   113 +
 .../processors/query/h2/sql/GridSqlArray.java   |     8 -
 .../processors/query/h2/sql/GridSqlColumn.java  |     7 -
 .../query/h2/sql/GridSqlCreateTable.java        |   120 +
 .../query/h2/sql/GridSqlFunction.java           |     2 +-
 .../processors/query/h2/sql/GridSqlMerge.java   |     5 +-
 .../query/h2/sql/GridSqlQueryParser.java        |   442 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |    64 +-
 .../query/h2/twostep/DistributedUpdateRun.java  |   133 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |   379 +-
 .../query/h2/twostep/GridMergeIndex.java        |     8 +-
 .../h2/twostep/GridMergeIndexIterator.java      |   165 +
 .../query/h2/twostep/GridMergeIndexSorted.java  |     4 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |     4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   415 +-
 .../query/h2/twostep/MapNodeResults.java        |    52 +-
 .../query/h2/twostep/MapQueryLazyWorker.java    |   176 +
 .../query/h2/twostep/MapQueryLazyWorkerKey.java |    97 +
 .../query/h2/twostep/MapQueryResult.java        |    46 +-
 .../query/h2/twostep/MapQueryResults.java       |    26 +-
 .../query/h2/twostep/MapRequestKey.java         |    23 +-
 .../query/h2/twostep/msg/GridH2DmlRequest.java  |   516 +
 .../query/h2/twostep/msg/GridH2DmlResponse.java |   250 +
 .../query/h2/twostep/msg/GridH2Geometry.java    |     2 +-
 .../query/h2/twostep/msg/GridH2Null.java        |     1 +
 .../h2/twostep/msg/GridH2QueryRequest.java      |     9 +
 .../twostep/msg/GridH2ValueMessageFactory.java  |     6 +
 .../cache/GridCacheFullTextQuerySelfTest.java   |    30 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |    43 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |    15 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    13 +-
 ...ributedJoinPartitionedAndReplicatedTest.java |     2 +
 .../IgniteCacheInsertSqlQuerySelfTest.java      |    18 +
 ...titionOnAffinityRunWithCollisionSpiTest.java |     3 +-
 ...CheckClusterStateBeforeExecuteQueryTest.java |    89 +
 ...leNodeWithIndexingPutGetPersistenceTest.java |    12 +-
 ...stributedPartitionQueryAbstractSelfTest.java |     8 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |    83 +-
 .../IgniteCacheQueryNodeRestartTxSelfTest.java  |    36 +
 ...ldsQueryJoinNoPrimaryPartitionsSelfTest.java |   151 +
 .../cache/index/AbstractSchemaSelfTest.java     |   247 +-
 ...ynamicColumnsAbstractConcurrentSelfTest.java |  1060 ++
 .../cache/index/DynamicColumnsAbstractTest.java |   206 +
 ...umnsConcurrentAtomicPartitionedSelfTest.java |    33 +
 ...lumnsConcurrentAtomicReplicatedSelfTest.java |    33 +
 ...currentTransactionalPartitionedSelfTest.java |    33 +
 ...ncurrentTransactionalReplicatedSelfTest.java |    34 +
 .../DynamicIndexAbstractBasicSelfTest.java      |   369 +-
 .../DynamicIndexAbstractConcurrentSelfTest.java |   192 +-
 .../index/DynamicIndexAbstractSelfTest.java     |    16 +-
 .../cache/index/H2ConnectionLeaksSelfTest.java  |   169 +
 .../H2DynamicColumnsAbstractBasicSelfTest.java  |   431 +
 .../H2DynamicColumnsClientBasicSelfTest.java    |    28 +
 .../H2DynamicColumnsServerBasicSelfTest.java    |    28 +
 ...icColumnsServerCoordinatorBasicSelfTest.java |    28 +
 ...lexClientAtomicPartitionedNoBackupsTest.java |    34 +
 ...exingComplexClientAtomicPartitionedTest.java |     2 +-
 ...dexingComplexClientAtomicReplicatedTest.java |     2 +-
 ...ntTransactionalPartitionedNoBackupsTest.java |    34 +
 ...mplexClientTransactionalPartitionedTest.java |     2 +-
 ...omplexClientTransactionalReplicatedTest.java |     2 +-
 ...lexServerAtomicPartitionedNoBackupsTest.java |    34 +
 ...exingComplexServerAtomicPartitionedTest.java |     2 +-
 ...dexingComplexServerAtomicReplicatedTest.java |     2 +-
 ...erTransactionalPartitionedNoBackupsTest.java |    34 +
 ...mplexServerTransactionalPartitionedTest.java |     2 +-
 ...omplexServerTransactionalReplicatedTest.java |     2 +-
 .../index/H2DynamicIndexingComplexTest.java     |    22 +-
 .../cache/index/H2DynamicTableSelfTest.java     |   721 +-
 .../cache/index/LongIndexNameTest.java          |   214 +
 .../cache/index/SchemaExchangeSelfTest.java     |     4 +-
 ...onnectorConfigurationValidationSelfTest.java |   392 +
 ...eDbSingleNodeWithIndexingWalRestoreTest.java |    18 +-
 ...oreQueryWithMultipleClassesPerCacheTest.java |    11 +-
 .../IgnitePersistentStoreSchemaLoadTest.java    |   253 +-
 .../query/IgniteSqlNotNullConstraintTest.java   |  1240 ++
 .../query/IgniteSqlParameterizedQueryTest.java  |   392 +
 .../query/IgniteSqlSchemaIndexingTest.java      |    30 +-
 .../query/IgniteSqlSegmentedIndexSelfTest.java  |   124 +-
 ...teSqlSkipReducerOnUpdateDmlFlagSelfTest.java |   800 ++
 ...IgniteSqlSkipReducerOnUpdateDmlSelfTest.java |   755 ++
 .../processors/query/LazyQuerySelfTest.java     |   389 +
 .../MultipleStatementsSqlQuerySelfTest.java     |   154 +
 .../processors/query/SqlSchemaSelfTest.java     |   104 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |    75 +-
 .../query/h2/IgniteSqlBigIntegerKeyTest.java    |   163 +
 .../h2/database/InlineIndexHelperTest.java      |    32 +-
 .../query/h2/opt/GridH2TableSelfTest.java       |   369 -
 .../query/h2/sql/GridQueryParsingTest.java      |   305 +-
 ...onnectorConfigurationValidationSelfTest.java |     3 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    55 +
 .../IgniteCacheQuerySelfTestSuite2.java         |    17 +
 .../IgniteCacheQuerySelfTestSuite3.java         |     3 -
 .../IgniteDistributedJoinTestSuite.java         |    55 -
 .../IgnitePdsWithIndexingCoreTestSuite.java     |    11 +
 ...ityWithJsr310Java8DateTimeApiFieldsTest.java |   351 +
 ...acheQueryJsr310Java8DateTimeApiBaseTest.java |    88 +
 ...yJsr310Java8DateTimeApiSupportTestSuite.java |    38 +
 modules/jcl/pom.xml                             |     2 +-
 modules/jms11/pom.xml                           |     2 +-
 modules/jta/pom.xml                             |     2 +-
 modules/kafka/pom.xml                           |     2 +-
 modules/kubernetes/config/example-kube.xml      |     2 +-
 modules/kubernetes/pom.xml                      |     2 +-
 modules/log4j/pom.xml                           |     2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |    40 +-
 .../log4j/GridLog4jCorrectFileNameTest.java     |    13 +
 .../logger/log4j/GridLog4jInitializedTest.java  |     7 +-
 .../logger/log4j/GridLog4jLoggingFileTest.java  |    13 +-
 .../logger/log4j/GridLog4jLoggingPathTest.java  |    12 +-
 .../logger/log4j/GridLog4jLoggingUrlTest.java   |    15 +-
 .../log4j/GridLog4jNotInitializedTest.java      |     5 +
 .../ignite/testsuites/IgniteLog4jTestSuite.java |     6 +
 modules/log4j2/pom.xml                          |     2 +-
 .../ignite/logger/log4j2/Log4J2Logger.java      |   138 +-
 .../log4j2/GridLog4j2CorrectFileNameTest.java   |    94 -
 .../log4j2/GridLog4j2InitializedTest.java       |    77 -
 .../log4j2/GridLog4j2LoggingFileTest.java       |    68 -
 .../logger/log4j2/Log4j2LoggerSelfTest.java     |    26 +-
 .../log4j2/Log4j2LoggerVerboseModeSelfTest.java |    71 +-
 .../testsuites/IgniteLog4j2TestSuite.java       |     2 +
 modules/mesos/pom.xml                           |     2 +-
 modules/ml/licenses/netlib-java-bsd3.txt        |    51 +
 modules/ml/pom.xml                              |    15 +-
 .../java/org/apache/ignite/ml/Exportable.java   |    33 +
 .../java/org/apache/ignite/ml/Exporter.java     |    38 +
 .../java/org/apache/ignite/ml/FileExporter.java |    75 +
 .../ignite/ml/FuzzyCMeansModelFormat.java       |    76 +
 .../org/apache/ignite/ml/KMeansModelFormat.java |    77 +
 .../main/java/org/apache/ignite/ml/Model.java   |     7 +-
 .../main/java/org/apache/ignite/ml/Trainer.java |    37 +
 .../ml/clustering/BaseFuzzyCMeansClusterer.java |    90 +
 .../ml/clustering/BaseKMeansClusterer.java      |     2 +-
 .../FuzzyCMeansDistributedClusterer.java        |   512 +
 .../clustering/FuzzyCMeansLocalClusterer.java   |   254 +
 .../ignite/ml/clustering/FuzzyCMeansModel.java  |    88 +
 .../clustering/KMeansDistributedClusterer.java  |    76 +-
 .../ml/clustering/KMeansLocalClusterer.java     |     2 +-
 .../ignite/ml/clustering/KMeansModel.java       |    44 +-
 .../ignite/ml/clustering/WeightedClusterer.java |     2 +-
 .../apache/ignite/ml/estimators/Estimators.java |    50 +
 .../ignite/ml/estimators/package-info.java      |    22 +
 .../apache/ignite/ml/knn/models/KNNModel.java   |   233 +
 .../ignite/ml/knn/models/KNNModelFormat.java    |    92 +
 .../ignite/ml/knn/models/KNNStrategy.java       |    27 +
 .../ignite/ml/knn/models/Normalization.java     |    32 +
 .../ignite/ml/knn/models/package-info.java      |    22 +
 .../org/apache/ignite/ml/knn/package-info.java  |    22 +
 .../regression/KNNMultipleLinearRegression.java |    83 +
 .../ignite/ml/knn/regression/package-info.java  |    22 +
 .../java/org/apache/ignite/ml/math/Blas.java    |    32 +-
 .../apache/ignite/ml/math/DistanceMeasure.java  |    38 -
 .../ignite/ml/math/EuclideanDistance.java       |    47 -
 .../ignite/ml/math/IdentityValueMapper.java     |     2 +
 .../java/org/apache/ignite/ml/math/Matrix.java  |    17 +
 .../apache/ignite/ml/math/MatrixKeyMapper.java  |    30 -
 .../apache/ignite/ml/math/MatrixStorage.java    |     7 +-
 .../org/apache/ignite/ml/math/ValueMapper.java  |    37 -
 .../java/org/apache/ignite/ml/math/Vector.java  |     1 +
 .../apache/ignite/ml/math/VectorKeyMapper.java  |    29 -
 .../ml/math/decompositions/LUDecomposition.java |     2 +-
 .../ml/math/decompositions/QRDSolver.java       |   197 +
 .../ml/math/decompositions/QRDecomposition.java |    54 +-
 .../ml/math/distances/DistanceMeasure.java      |    39 +
 .../ml/math/distances/EuclideanDistance.java    |    59 +
 .../ml/math/distances/HammingDistance.java      |    65 +
 .../ml/math/distances/ManhattanDistance.java    |    59 +
 .../ignite/ml/math/distances/package-info.java  |    22 +
 .../ignite/ml/math/distributed/CacheUtils.java  |   734 ++
 .../ml/math/distributed/DistributedStorage.java |    35 +
 .../ml/math/distributed/MatrixKeyMapper.java    |    33 +
 .../ignite/ml/math/distributed/ValueMapper.java |    37 +
 .../ml/math/distributed/VectorKeyMapper.java    |    32 +
 .../distributed/keys/DataStructureCacheKey.java |    35 +
 .../math/distributed/keys/MatrixBlockKey.java   |    38 +
 .../math/distributed/keys/RowColMatrixKey.java  |    30 +
 .../math/distributed/keys/VectorBlockKey.java   |    34 +
 .../distributed/keys/impl/MatrixBlockKey.java   |   166 +
 .../distributed/keys/impl/SparseMatrixKey.java  |   119 +
 .../distributed/keys/impl/VectorBlockKey.java   |   152 +
 .../distributed/keys/impl/package-info.java     |    22 +
 .../ml/math/distributed/keys/package-info.java  |    22 +
 .../ml/math/distributed/package-info.java       |    22 +
 .../math/exceptions/knn/EmptyFileException.java |    37 +
 .../exceptions/knn/FileParsingException.java    |    39 +
 .../exceptions/knn/NoLabelVectorException.java  |    37 +
 .../knn/SmallTrainingDatasetSizeException.java  |    38 +
 .../ml/math/exceptions/knn/package-info.java    |    22 +
 .../ignite/ml/math/functions/Functions.java     |    41 +
 .../ml/math/functions/IgniteBinaryOperator.java |    29 +
 .../math/functions/IgniteCurriedBiFunction.java |    29 +
 .../ml/math/functions/IgniteSupplier.java       |    30 +
 .../math/functions/IgniteToDoubleFunction.java  |    30 +
 .../ml/math/functions/IgniteTriFunction.java    |     5 +-
 .../apache/ignite/ml/math/impls/CacheUtils.java |   559 -
 .../ml/math/impls/matrix/AbstractMatrix.java    |    57 +-
 .../ignite/ml/math/impls/matrix/BlockEntry.java |    50 -
 .../ml/math/impls/matrix/CacheMatrix.java       |     6 +-
 .../impls/matrix/DenseLocalOnHeapMatrix.java    |     4 +-
 .../ml/math/impls/matrix/MatrixBlockEntry.java  |    50 +
 .../matrix/SparseBlockDistributedMatrix.java    |   161 +-
 .../impls/matrix/SparseDistributedMatrix.java   |   165 +-
 .../storage/matrix/BaseBlockMatrixKey.java      |    41 -
 .../impls/storage/matrix/BlockMatrixKey.java    |   144 -
 .../storage/matrix/BlockMatrixStorage.java      |   302 +-
 .../storage/matrix/BlockVectorStorage.java      |   368 +
 .../storage/matrix/CacheMatrixStorage.java      |     9 +-
 .../matrix/DenseOffHeapMatrixStorage.java       |     7 +-
 .../storage/matrix/DiagonalMatrixStorage.java   |     5 +
 .../storage/matrix/FunctionMatrixStorage.java   |     5 +
 .../impls/storage/matrix/MapWrapperStorage.java |    16 +-
 .../storage/matrix/MatrixDelegateStorage.java   |     5 +
 .../storage/matrix/PivotedMatrixStorage.java    |     5 +
 .../storage/matrix/RandomMatrixStorage.java     |     5 +
 .../matrix/SparseDistributedMatrixStorage.java  |    72 +-
 .../matrix/SparseLocalOnHeapMatrixStorage.java  |     9 +-
 .../storage/vector/CacheVectorStorage.java      |     4 +-
 .../vector/SparseDistributedVectorStorage.java  |   281 +
 .../vector/SparseLocalOnHeapVectorStorage.java  |     2 +-
 .../ml/math/impls/vector/CacheVector.java       |     6 +-
 .../ml/math/impls/vector/MapWrapperVector.java  |     7 +
 .../vector/SparseBlockDistributedVector.java    |   135 +
 .../impls/vector/SparseDistributedVector.java   |   147 +
 .../ml/math/impls/vector/VectorBlockEntry.java  |    47 +
 .../apache/ignite/ml/math/util/MatrixUtil.java  |    17 +-
 .../AbstractMultipleLinearRegression.java       |    20 +
 .../OLSMultipleLinearRegression.java            |    41 +-
 .../OLSMultipleLinearRegressionModel.java       |    77 +
 .../OLSMultipleLinearRegressionModelFormat.java |    46 +
 .../OLSMultipleLinearRegressionTrainer.java     |    62 +
 .../ignite/ml/structures/LabeledDataset.java    |   475 +
 .../structures/LabeledDatasetTestTrainPair.java |   116 +
 .../ignite/ml/structures/LabeledVector.java     |    94 +
 .../ml/structures/LabeledVectorDouble.java      |    46 +
 .../ignite/ml/structures/package-info.java      |    22 +
 .../ignite/ml/trees/CategoricalRegionInfo.java  |    72 +
 .../ignite/ml/trees/CategoricalSplitInfo.java   |    68 +
 .../ignite/ml/trees/ContinuousRegionInfo.java   |    74 +
 .../ml/trees/ContinuousSplitCalculator.java     |    51 +
 .../org/apache/ignite/ml/trees/RegionInfo.java  |    62 +
 .../ml/trees/models/DecisionTreeModel.java      |    44 +
 .../ignite/ml/trees/models/package-info.java    |    22 +
 .../ml/trees/nodes/CategoricalSplitNode.java    |    50 +
 .../ml/trees/nodes/ContinuousSplitNode.java     |    56 +
 .../ignite/ml/trees/nodes/DecisionTreeNode.java |    33 +
 .../org/apache/ignite/ml/trees/nodes/Leaf.java  |    49 +
 .../apache/ignite/ml/trees/nodes/SplitNode.java |   100 +
 .../ignite/ml/trees/nodes/package-info.java     |    22 +
 .../apache/ignite/ml/trees/package-info.java    |    22 +
 .../ml/trees/trainers/columnbased/BiIndex.java  |   113 +
 ...exedCacheColumnDecisionTreeTrainerInput.java |    57 +
 .../CacheColumnDecisionTreeTrainerInput.java    |   141 +
 .../columnbased/ColumnDecisionTreeTrainer.java  |   568 +
 .../ColumnDecisionTreeTrainerInput.java         |    55 +
 .../MatrixColumnDecisionTreeTrainerInput.java   |    83 +
 .../trainers/columnbased/RegionProjection.java  |   109 +
 .../trainers/columnbased/TrainingContext.java   |   166 +
 .../columnbased/caches/ContextCache.java        |    68 +
 .../columnbased/caches/FeaturesCache.java       |   151 +
 .../columnbased/caches/ProjectionsCache.java    |   286 +
 .../trainers/columnbased/caches/SplitCache.java |   206 +
 .../columnbased/caches/package-info.java        |    22 +
 .../ContinuousSplitCalculators.java             |    34 +
 .../contsplitcalcs/GiniSplitCalculator.java     |   234 +
 .../contsplitcalcs/VarianceSplitCalculator.java |   179 +
 .../contsplitcalcs/package-info.java            |    22 +
 .../trainers/columnbased/package-info.java      |    22 +
 .../columnbased/regcalcs/RegionCalculators.java |    85 +
 .../columnbased/regcalcs/package-info.java      |    22 +
 .../vectors/CategoricalFeatureProcessor.java    |   212 +
 .../vectors/ContinuousFeatureProcessor.java     |   111 +
 .../vectors/ContinuousSplitInfo.java            |    71 +
 .../columnbased/vectors/FeatureProcessor.java   |    82 +
 .../vectors/FeatureVectorProcessorUtils.java    |    57 +
 .../columnbased/vectors/SampleInfo.java         |    80 +
 .../trainers/columnbased/vectors/SplitInfo.java |   106 +
 .../columnbased/vectors/package-info.java       |    22 +
 .../org/apache/ignite/ml/util/MnistUtils.java   |   124 +
 .../java/org/apache/ignite/ml/util/Utils.java   |    60 +
 .../org/apache/ignite/ml/util/package-info.java |    22 +
 .../org/apache/ignite/ml/IgniteMLTestSuite.java |     7 +-
 .../org/apache/ignite/ml/LocalModelsTest.java   |   166 +
 .../java/org/apache/ignite/ml/TestUtils.java    |    15 +
 .../ml/clustering/ClusteringTestSuite.java      |     7 +-
 .../FuzzyCMeansDistributedClustererTest.java    |   177 +
 .../FuzzyCMeansLocalClustererTest.java          |   203 +
 .../KMeansDistributedClustererTest.java         |   197 -
 ...KMeansDistributedClustererTestMultiNode.java |   138 +
 ...MeansDistributedClustererTestSingleNode.java |   198 +
 .../ml/clustering/KMeansLocalClustererTest.java |     2 +-
 .../apache/ignite/ml/clustering/KMeansUtil.java |     6 +-
 .../org/apache/ignite/ml/knn/BaseKNNTest.java   |    91 +
 .../ignite/ml/knn/KNNClassificationTest.java    |   153 +
 .../ml/knn/KNNMultipleLinearRegressionTest.java |   157 +
 .../org/apache/ignite/ml/knn/KNNTestSuite.java  |    33 +
 .../ignite/ml/knn/LabeledDatasetTest.java       |   266 +
 .../org/apache/ignite/ml/math/BlasTest.java     |    11 +-
 .../ml/math/MathImplDistributedTestSuite.java   |     8 +-
 .../ignite/ml/math/MathImplLocalTestSuite.java  |     6 +-
 .../ml/math/decompositions/QRDSolverTest.java   |    87 +
 .../ignite/ml/math/distances/DistanceTest.java  |    75 +
 .../ml/math/impls/matrix/CacheMatrixTest.java   |     2 +-
 .../impls/matrix/MatrixImplementationsTest.java |    17 +
 .../impls/matrix/MatrixKeyMapperForTests.java   |     2 +-
 .../SparseDistributedBlockMatrixTest.java       |   149 +-
 .../matrix/SparseDistributedMatrixTest.java     |    75 +-
 .../SparseDistributedVectorStorageTest.java     |   123 +
 .../ml/math/impls/vector/CacheVectorTest.java   |    12 +-
 .../SparseBlockDistributedVectorTest.java       |   186 +
 .../vector/SparseDistributedVectorTest.java     |   191 +
 .../math/impls/vector/VectorToMatrixTest.java   |     3 -
 ...tedBlockOLSMultipleLinearRegressionTest.java |   901 ++
 ...tributedOLSMultipleLinearRegressionTest.java |   903 ++
 .../OLSMultipleLinearRegressionModelTest.java   |    53 +
 .../OLSMultipleLinearRegressionTest.java        |     1 +
 .../ml/regressions/RegressionsTestSuite.java    |     5 +-
 .../ignite/ml/trees/BaseDecisionTreeTest.java   |    70 +
 .../ml/trees/ColumnDecisionTreeTrainerTest.java |   191 +
 .../ignite/ml/trees/DecisionTreesTestSuite.java |    33 +
 .../ml/trees/GiniSplitCalculatorTest.java       |   141 +
 .../ignite/ml/trees/SplitDataGenerator.java     |   390 +
 .../ml/trees/VarianceSplitCalculatorTest.java   |    84 +
 .../ColumnDecisionTreeTrainerBenchmark.java     |   456 +
 .../ml/src/test/resources/datasets/README.md    |     2 +
 .../resources/datasets/knn/cleared_machines.txt |   209 +
 .../src/test/resources/datasets/knn/empty.txt   |     0
 .../ml/src/test/resources/datasets/knn/iris.txt |   150 +
 .../resources/datasets/knn/iris_incorrect.txt   |   150 +
 .../resources/datasets/knn/machine.data.txt     |   209 +
 .../test/resources/datasets/knn/missed_data.txt |     3 +
 .../src/test/resources/datasets/knn/no_data.txt |     6 +
 .../trees/columntrees.manualrun.properties      |    21 +
 modules/mqtt/pom.xml                            |     2 +-
 modules/osgi-karaf/pom.xml                      |     2 +-
 modules/osgi-paxlogging/pom.xml                 |     2 +-
 modules/osgi/pom.xml                            |     2 +-
 modules/platforms/.gitignore                    |     3 +-
 modules/platforms/cpp/README.md                 |   220 +
 .../platforms/cpp/common/include/Makefile.am    |     1 +
 .../cpp/common/include/ignite/common/lazy.h     |   172 +
 .../include/ignite/common/platform_utils.h      |    20 +-
 .../cpp/common/include/ignite/common/utils.h    |    51 +
 .../os/linux/src/common/platform_utils.cpp      |    34 +-
 .../common/os/win/src/common/platform_utils.cpp |    29 +-
 .../cpp/common/project/vs/common.vcxproj        |     1 +
 .../common/project/vs/common.vcxproj.filters    |     3 +
 modules/platforms/cpp/configure.ac              |     2 +-
 modules/platforms/cpp/configure.acrel           |     2 +-
 .../platforms/cpp/core-test/config/invalid.xml  |    39 +
 .../cpp/core-test/config/isolated-32.xml        |    51 +
 .../cpp/core-test/config/isolated-default.xml   |    59 +
 .../platforms/cpp/core-test/config/isolated.xml |    32 +
 .../core-test/config/persistence-store-32.xml   |    51 +
 .../config/persistence-store-default.xml        |    36 +
 .../cpp/core-test/config/persistence-store.xml  |    32 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |    10 +
 .../project/vs/core-test.vcxproj.filters        |    19 +
 .../cpp/core-test/src/cluster_test.cpp          |    33 +
 .../cpp/core-test/src/ignition_test.cpp         |    37 +-
 .../cpp/core-test/src/interop_test.cpp          |    11 -
 .../platforms/cpp/core-test/src/test_utils.cpp  |     8 +-
 .../ignite/cache/query/query_sql_fields.h       |    71 +-
 .../impl/cache/query/query_fields_row_impl.h    |     2 +-
 .../cpp/core/include/ignite/impl/ignite_impl.h  |    25 +-
 modules/platforms/cpp/core/src/ignition.cpp     |    12 +-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |    45 +-
 .../examples/odbc-example/src/odbc_example.cpp  |     8 +-
 modules/platforms/cpp/ignite/src/ignite.cpp     |    23 +-
 .../cpp/jni/include/ignite/jni/utils.h          |    17 +-
 .../platforms/cpp/jni/os/linux/src/utils.cpp    |   120 +-
 modules/platforms/cpp/jni/os/win/src/utils.cpp  |   133 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |     6 +-
 .../odbc-test/config/queries-test-default.xml   |     4 +-
 .../odbc-test/config/queries-test-noodbc-32.xml |     2 +-
 .../odbc-test/config/queries-test-noodbc.xml    |     2 +-
 .../cpp/odbc-test/include/test_utils.h          |    16 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |    11 +-
 .../project/vs/odbc-test.vcxproj.filters        |    11 +-
 .../cpp/odbc-test/src/api_robustness_test.cpp   |    25 +-
 .../cpp/odbc-test/src/attributes_test.cpp       |   252 +
 .../cpp/odbc-test/src/configuration_test.cpp    |    36 +-
 .../cpp/odbc-test/src/connection_info_test.cpp  |     3 +-
 .../platforms/cpp/odbc-test/src/errors_test.cpp |   289 +
 .../cpp/odbc-test/src/meta_queries_test.cpp     |   113 +
 .../platforms/cpp/odbc-test/src/parser_test.cpp |     6 +-
 .../cpp/odbc-test/src/queries_test.cpp          |   544 +-
 .../cpp/odbc-test/src/sql_get_info_test.cpp     |   485 +
 .../platforms/cpp/odbc-test/src/test_utils.cpp  |    21 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |     1 +
 .../platforms/cpp/odbc/include/ignite/odbc.h    |    22 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |   134 +-
 .../include/ignite/odbc/config/configuration.h  |    56 +-
 .../ignite/odbc/config/connection_info.h        |     8 +-
 .../cpp/odbc/include/ignite/odbc/connection.h   |   131 +-
 .../cpp/odbc/include/ignite/odbc/cursor.h       |     7 +
 .../ignite/odbc/diagnostic/diagnosable.h        |     9 +
 .../odbc/diagnostic/diagnosable_adapter.h       |     8 +
 .../cpp/odbc/include/ignite/odbc/message.h      |   190 +-
 .../cpp/odbc/include/ignite/odbc/odbc_error.h   |    94 +
 .../cpp/odbc/include/ignite/odbc/parser.h       |    37 +-
 .../odbc/include/ignite/odbc/protocol_version.h |    11 +-
 .../include/ignite/odbc/query/batch_query.h     |    22 +-
 .../ignite/odbc/query/column_metadata_query.h   |    10 +
 .../odbc/include/ignite/odbc/query/data_query.h |    31 +-
 .../ignite/odbc/query/foreign_keys_query.h      |     7 +
 .../ignite/odbc/query/primary_keys_query.h      |     7 +
 .../cpp/odbc/include/ignite/odbc/query/query.h  |     7 +
 .../ignite/odbc/query/special_columns_query.h   |     7 +
 .../ignite/odbc/query/table_metadata_query.h    |    10 +
 .../include/ignite/odbc/query/type_info_query.h |    10 +
 .../cpp/odbc/include/ignite/odbc/statement.h    |    10 +-
 .../include/ignite/odbc/system/socket_client.h  |    75 +-
 .../odbc/system/ui/dsn_configuration_window.h   |     8 +
 .../cpp/odbc/install/ignite-odbc-amd64.wxs      |     2 +-
 .../cpp/odbc/install/ignite-odbc-x86.wxs        |     2 +-
 .../odbc/os/linux/src/system/socket_client.cpp  |   265 +-
 .../odbc/os/win/src/system/socket_client.cpp    |   312 +-
 .../src/system/ui/dsn_configuration_window.cpp  |    54 +-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |     1 +
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |     3 +
 modules/platforms/cpp/odbc/src/common_types.cpp |    51 +
 .../cpp/odbc/src/config/configuration.cpp       |    48 +-
 .../cpp/odbc/src/config/connection_info.cpp     |  2250 +++-
 modules/platforms/cpp/odbc/src/connection.cpp   |   259 +-
 modules/platforms/cpp/odbc/src/cursor.cpp       |     5 +
 .../odbc/src/diagnostic/diagnosable_adapter.cpp |     8 +
 .../odbc/src/diagnostic/diagnostic_record.cpp   |   111 +-
 modules/platforms/cpp/odbc/src/dsn_config.cpp   |     7 +
 modules/platforms/cpp/odbc/src/entry_points.cpp |    65 +-
 modules/platforms/cpp/odbc/src/message.cpp      |   153 +-
 .../platforms/cpp/odbc/src/meta/column_meta.cpp |     3 +
 modules/platforms/cpp/odbc/src/odbc.cpp         |    77 +-
 .../platforms/cpp/odbc/src/protocol_version.cpp |    12 +-
 .../cpp/odbc/src/query/batch_query.cpp          |   105 +-
 .../odbc/src/query/column_metadata_query.cpp    |    32 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |   159 +-
 .../cpp/odbc/src/query/foreign_keys_query.cpp   |     5 +
 .../cpp/odbc/src/query/primary_keys_query.cpp   |     5 +
 .../odbc/src/query/special_columns_query.cpp    |     5 +
 .../cpp/odbc/src/query/table_metadata_query.cpp |    32 +-
 .../cpp/odbc/src/query/type_info_query.cpp      |    21 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |   100 +-
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../Apache.Ignite.AspNet.csproj                 |     3 +-
 .../Apache.Ignite.AspNet.nuspec                 |     4 +
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../Apache.Ignite.Benchmarks.csproj             |     2 +
 .../Apache.Ignite.Benchmarks/BenchmarkRunner.cs |     1 -
 .../Interop/PlatformBenchmarkBase.cs            |    21 +
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../ThinClient/ThinClientGetBenchmark.cs        |    63 +
 .../ThinClient/ThinClientPutBenchmark.cs        |    59 +
 .../Apache.Ignite.Core.Tests.DotNetCore.csproj  |   144 +
 .../Apache.Ignite.Core.Tests.DotNetCore.sln     |    25 +
 ...Ignite.Core.Tests.DotNetCore.sln.DotSettings |    13 +
 .../Cache/CacheTest.cs                          |    56 +
 .../Common/IgnitionStartTest.cs                 |   114 +
 .../Common/Person.cs                            |    44 +
 .../Common/TestBase.cs                          |    44 +
 .../Common/TestFixtureSetUp.cs                  |    30 +
 .../Common/TestFixtureTearDown.cs               |    30 +
 .../Common/TestLogger.cs                        |    74 +
 .../Common/TestRunner.cs                        |    33 +
 .../Common/TestUtils.DotNetCore.cs              |    93 +
 .../README.txt                                  |    17 +
 .../app.config                                  |    44 +
 .../dotnet-test-windows.bat                     |    32 +
 .../dotnet-test-wsl.bat                         |    34 +
 .../nuget.config                                |    41 +
 .../Apache.Ignite.Core.Tests.NuGet.csproj       |     4 +-
 ...ache.Ignite.Core.Tests.NuGet.sln.DotSettings |     4 +-
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../TestExtensions.cs                           |     2 +
 .../Apache.Ignite.Core.Tests.csproj             |    62 +-
 .../ApiParity/BinaryParityTest.cs               |    48 +
 .../ApiParity/CacheAffinityParityTest.cs        |    39 +
 .../ApiParity/CacheConfigurationParityTest.cs   |    77 +
 .../ApiParity/CacheMetricsParityTest.cs         |    64 +
 .../ApiParity/CacheParityTest.cs                |    76 +
 .../ClientConnectorConfigurationParityTest.cs   |    39 +
 .../ApiParity/ClusterMetricsParityTest.cs       |    47 +
 .../ApiParity/ClusterParityTest.cs              |    50 +
 .../ApiParity/ComputeParityTest.cs              |    51 +
 .../DataRegionConfigurationParityTest.cs        |    39 +
 .../ApiParity/DataRegionMetricsParityTest.cs    |    54 +
 .../DataStorageConfigurationParityTest.cs       |    53 +
 .../ApiParity/DataStorageMetricsParityTest.cs   |    38 +
 .../ApiParity/EventsParityTest.cs               |    50 +
 .../ApiParity/IgniteConfigurationParityTest.cs  |    98 +
 .../ApiParity/IgniteParityTest.cs               |    79 +
 .../ApiParity/MessagingParityTest.cs            |    39 +
 .../ApiParity/ParityTest.cs                     |   201 +
 .../QueryEntityConfigurationParityTest.cs       |    49 +
 .../ApiParity/ServicesParityTest.cs             |    39 +
 .../ApiParity/StreamerParityTest.cs             |    54 +
 .../ApiParity/TransactionMetricsParityTest.cs   |    39 +
 .../ApiParity/TransactionsParityTest.cs         |    39 +
 .../AssertExtensions.cs                         |    90 +
 .../Binary/BinaryBuilderSelfTest.cs             |    10 +-
 .../Binary/BinaryDynamicRegistrationTest.cs     |    64 +-
 .../Binary/BinaryFooterTest.cs                  |     2 +-
 .../Binary/BinarySelfTest.cs                    |    97 +-
 .../Binary/EnumsTest.cs                         |     2 +-
 .../Serializable/AdvancedSerializationTest.cs   |     2 +
 .../BasicSerializableObjectsTest.cs             |     3 +-
 .../Binary/Serializable/CallbacksTest.cs        |     9 +-
 .../Serializable/GenericCollectionsTest.cs      |   112 +
 .../Binary/Serializable/SqlDmlTest.cs           |     4 +-
 .../BinaryConfigurationTest.cs                  |    14 +-
 .../Cache/Affinity/AffinityAttributeTest.cs     |   135 +
 .../Cache/Affinity/AffinityFieldTest.cs         |    45 +-
 .../Cache/CacheAbstractTest.cs                  |   153 +-
 .../Cache/CacheConfigurationTest.cs             |   114 +-
 .../Cache/DataRegionMetricsTest.cs              |   153 +
 .../Cache/DataStorageMetricsTest.cs             |   107 +
 .../Cache/MemoryMetricsTest.cs                  |     2 +
 .../Cache/PersistenceTest.cs                    |   234 +
 .../Cache/PersistentStoreTest.cs                |   224 -
 .../Cache/PersistentStoreTestObsolete.cs        |   193 +
 .../Cache/Query/CacheDmlQueriesTest.cs          |    48 +-
 .../Query/CacheDmlQueriesTestSimpleName.cs      |     2 +-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |    36 +-
 .../Cache/Query/CacheQueriesTest.cs             |    53 +-
 .../Cache/Query/CacheQueriesTestSimpleName.cs   |     2 +-
 .../Cache/Query/Linq/CacheLinqTest.Base.cs      |     5 +-
 .../Query/Linq/CacheLinqTest.Introspection.cs   |    25 +-
 .../Cache/Query/Linq/CacheLinqTest.Join.cs      |   110 +-
 .../Cache/Query/Linq/CacheLinqTest.Misc.cs      |    11 +-
 .../Cache/Query/Linq/CacheLinqTest.Strings.cs   |    26 +
 .../Cache/Query/Linq/CacheLinqTestSimpleName.cs |     2 +-
 .../Cache/Store/CacheStoreSessionTest.cs        |    13 +-
 .../Client/Cache/BinaryBuilderTest.cs           |   118 +
 .../Client/Cache/CacheTest.cs                   |   900 ++
 .../Client/Cache/CacheTestKeepBinary.cs         |   499 +
 .../Client/Cache/CacheTestNoMeta.cs             |   129 +
 .../Cache/ClientCacheConfigurationTest.cs       |   211 +
 .../Client/Cache/CreateCacheTest.cs             |   208 +
 .../Client/Cache/EmptyObject.cs                 |    54 +
 .../Client/Cache/LinqTest.cs                    |   117 +
 .../Client/Cache/Person.cs                      |    77 +
 .../Client/Cache/ScanQueryTest.cs               |   339 +
 .../Client/Cache/SqlQueryTest.cs                |   227 +
 .../Client/Cache/SqlQueryTestBase.cs            |    68 +
 .../Client/ClientConnectionTest.cs              |   175 +
 .../Client/ClientTestBase.cs                    |   181 +
 .../Client/IgniteClientConfigurationTest.cs     |    42 +
 .../Client/RawSocketTest.cs                     |   221 +
 .../Compute/AbstractTaskTest.cs                 |     2 -
 .../Compute/BinarizableClosureTaskTest.cs       |     1 -
 .../Compute/ComputeApiTest.JavaTask.cs          |   582 +
 .../Compute/ComputeApiTest.cs                   |   577 +-
 .../Compute/ComputeApiTestFullFooter.cs         |     2 +
 .../Config/Dynamic/dynamic-data.xml             |     2 +
 .../Config/cache-default.xml                    |    56 +
 .../Config/cache-query.xml                      |     6 +
 .../Config/full-config.xml                      |   157 +
 .../Config/spring-test.xml                      |    18 +-
 .../ConsoleRedirectTest.cs                      |    74 +-
 .../DataStructures/AtomicLongTest.cs            |     2 +-
 .../DataStructures/AtomicReferenceTest.cs       |     2 +-
 .../DataStructures/AtomicSequenceTest.cs        |     2 +-
 .../Dataload/DataStreamerTest.cs                |   170 +-
 .../Apache.Ignite.Core.Tests/DeploymentTest.cs  |   168 +-
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |     7 +-
 .../EventsTestLocalListeners.cs                 |   187 +
 .../Examples/Example.cs                         |    19 +-
 .../Examples/ExamplesTest.cs                    |    42 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |     2 +
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |    18 +-
 .../IgniteConfigurationSerializerTest.cs        |   370 +-
 .../IgniteConfigurationTest.cs                  |   344 +-
 .../IgniteManagerTest.cs                        |    10 +-
 .../IgniteStartStopTest.cs                      |    12 +-
 .../Log/CustomLoggerTest.cs                     |     8 +-
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |   131 +-
 .../MultiAppDomainTest.cs                       |   171 +
 .../Plugin/PluginTest.cs                        |     1 -
 .../Plugin/TestIgnitePluginConfiguration.cs     |     1 -
 .../Process/ListDataReader.cs                   |    52 +
 .../ProjectFilesTest.cs                         |    18 +-
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../Services/ServicesAsyncWrapper.cs            |    28 +-
 .../Services/ServicesTest.cs                    |   273 +-
 .../TestUtils.Common.cs                         |   366 +
 .../TestUtils.Windows.cs                        |   120 +
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   406 -
 .../Apache.Ignite.Core.csproj                   |   112 +-
 .../Apache.Ignite.Core.nuspec                   |     7 +-
 .../Binary/BinaryConfiguration.cs               |    46 +-
 .../Affinity/AffinityKeyMappedAttribute.cs      |    25 +-
 .../Cache/CachePartialUpdateException.cs        |     9 +-
 .../Cache/Configuration/CacheConfiguration.cs   |   250 +-
 .../Configuration/CacheKeyConfiguration.cs      |    84 +
 .../Cache/Configuration/DataPageEvictionMode.cs |     3 +
 .../Cache/Configuration/MemoryConfiguration.cs  |     7 +-
 .../Configuration/MemoryPolicyConfiguration.cs  |     8 +-
 .../Cache/Configuration/QueryEntity.cs          |    97 +-
 .../Cache/Configuration/QueryField.cs           |    45 +
 .../Cache/Configuration/QueryIndex.cs           |    50 +-
 .../Configuration/QuerySqlFieldAttribute.cs     |    20 +
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |     9 +-
 .../Apache.Ignite.Core/Cache/IMemoryMetrics.cs  |     6 +-
 .../Cache/Query/IFieldsQueryCursor.cs           |    34 +
 .../Apache.Ignite.Core/Cache/Query/ScanQuery.cs |     2 +-
 .../Cache/Query/SqlFieldsQuery.cs               |    17 +-
 .../Client/Cache/CacheClientConfiguration.cs    |   420 +
 .../Client/Cache/ICacheClient.cs                |   254 +
 .../Client/ClientStatusCode.cs                  |    57 +
 .../Apache.Ignite.Core/Client/IIgniteClient.cs  |   103 +
 .../Client/IgniteClientConfiguration.cs         |   104 +
 .../Client/IgniteClientException.cs             |   116 +
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |    31 +-
 .../Configuration/CheckpointWriteOrder.cs       |    37 +
 .../ClientConnectorConfiguration.cs             |   159 +
 .../Configuration/DataPageEvictionMode.cs       |    59 +
 .../Configuration/DataRegionConfiguration.cs    |   222 +
 .../Configuration/DataStorageConfiguration.cs   |   473 +
 .../Configuration/SqlConnectorConfiguration.cs  |     2 +
 .../Apache.Ignite.Core/Configuration/WalMode.cs |    45 +
 .../Datastream/DataStreamerDefaults.cs          |    46 +
 .../Datastream/IDataStreamer.cs                 |    21 +-
 .../Datastream/StreamTransformer.cs             |     6 +-
 .../Datastream/StreamVisitor.cs                 |     6 +-
 .../Apache.Ignite.Core/Events/JobEvent.cs       |     6 +-
 .../Events/LocalEventListener.cs                |    79 +
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |     4 +-
 .../Apache.Ignite.Core/IDataRegionMetrics.cs    |    55 +
 .../Apache.Ignite.Core/IDataStorageMetrics.cs   |    87 +
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |    38 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |   281 +-
 .../IgniteConfigurationSection.xsd              |   488 +-
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |    51 +-
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |    19 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |   137 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |    12 +-
 .../Impl/Binary/BinaryProcessor.cs              |   157 +-
 .../Impl/Binary/BinaryProcessorClient.cs        |   112 +
 .../Impl/Binary/BinaryReader.cs                 |   110 +-
 .../Impl/Binary/BinaryReaderExtensions.cs       |    76 +-
 .../Impl/Binary/BinaryReflectiveActions.cs      |    16 +
 .../Binary/BinarySurrogateTypeDescriptor.cs     |     8 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |   184 +-
 .../Impl/Binary/BinaryTypeId.cs                 |   228 +
 .../Impl/Binary/BinaryUtils.cs                  |   256 +-
 .../Impl/Binary/BinaryWriter.cs                 |   168 +-
 .../Impl/Binary/BinaryWriterExtensions.cs       |    41 +-
 .../Impl/Binary/IBinaryProcessor.cs             |    73 +
 .../Impl/Binary/IBinaryRawWriteAware.cs         |    42 +
 .../Impl/Binary/IBinaryTypeDescriptor.cs        |     6 +-
 .../Impl/Binary/Io/BinaryHeapStream.cs          |  1018 +-
 .../Impl/Binary/Io/BinaryStreamBase.cs          |  1249 --
 .../Impl/Binary/Marshaller.cs                   |   107 +-
 .../Impl/Binary/Metadata/BinaryType.cs          |    86 +-
 .../Impl/Binary/MultidimensionalArrayHolder.cs  |   132 +
 .../Binary/MultidimensionalArraySerializer.cs   |    48 +
 .../Impl/Binary/ReflectionUtils.cs              |    27 +
 .../Impl/Binary/SerializableSerializer.cs       |    29 +-
 .../Impl/Binary/Structure/BinaryStructure.cs    |   147 +-
 .../Binary/Structure/BinaryStructureTracker.cs  |    16 +-
 .../Impl/Binary/TypeResolver.cs                 |     2 +-
 .../Impl/Cache/CacheAffinityImpl.cs             |     2 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheEntry.cs |    31 +-
 .../Impl/Cache/CacheEntryFilterHolder.cs        |     2 +-
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |     4 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |    83 +-
 .../Impl/Cache/ICacheInternal.cs                |     6 +
 .../Impl/Cache/MemoryMetrics.cs                 |     2 +
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |   264 -
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |    31 +-
 .../Impl/Cache/Query/PlatformQueryQursorBase.cs |    88 +
 .../Impl/Cache/Query/QueryCursor.cs             |    18 +-
 .../Impl/Cache/Query/QueryCursorBase.cs         |   309 +
 .../Impl/Cache/Store/CacheStoreInternal.cs      |    14 +-
 .../Impl/Client/Cache/CacheClient.cs            |   622 +
 .../Impl/Client/Cache/CacheFlags.cs             |    38 +
 .../Cache/ClientCacheConfigurationSerializer.cs |   261 +
 .../Cache/Query/ClientFieldsQueryCursor.cs      |    79 +
 .../Client/Cache/Query/ClientQueryCursor.cs     |    46 +
 .../Client/Cache/Query/ClientQueryCursorBase.cs |    89 +
 .../Impl/Client/Cache/Query/StatementType.cs    |    42 +
 .../Apache.Ignite.Core/Impl/Client/ClientOp.cs  |    72 +
 .../Impl/Client/ClientProtocolVersion.cs        |   119 +
 .../Impl/Client/ClientSocket.cs                 |   286 +
 .../Impl/Client/IgniteClient.cs                 |   249 +
 .../Impl/Cluster/ClusterGroupImpl.cs            |    57 +-
 .../Apache.Ignite.Core/Impl/Common/Classpath.cs |    52 +-
 .../Common/IgniteConfigurationXmlSerializer.cs  |    22 +-
 .../Impl/Common/IgniteHome.cs                   |     8 +-
 .../Impl/Compute/Closure/ComputeActionJob.cs    |     4 +-
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |     6 +-
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |     4 +-
 .../Closure/ComputeReducingClosureTask.cs       |     2 +-
 .../Compute/Closure/IComputeResourceInjector.cs |     2 +-
 .../Impl/Compute/ComputeFunc.cs                 |     2 +-
 .../Impl/Compute/ComputeImpl.cs                 |     2 +-
 .../Impl/Compute/ComputeJob.cs                  |     2 +-
 .../Impl/Compute/ComputeJobHolder.cs            |    21 +-
 .../Impl/Compute/ComputeOutFunc.cs              |     2 +-
 .../Impl/DataRegionMetrics.cs                   |    61 +
 .../Impl/DataStorageMetrics.cs                  |    87 +
 .../Impl/Datastream/DataStreamerBatch.cs        |    10 +-
 .../Impl/Datastream/DataStreamerImpl.cs         |    45 +-
 .../Impl/Deployment/PeerAssemblyResolver.cs     |    17 +-
 .../Impl/Deployment/PeerLoadingExtensions.cs    |     4 +-
 .../Impl/Deployment/PeerLoadingObjectHolder.cs  |     4 +-
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |    33 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |    43 +-
 .../Apache.Ignite.Core/Impl/IIgniteInternal.cs  |    80 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |    47 +-
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |    91 +-
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |   302 +-
 .../Apache.Ignite.Core/Impl/MemoryInfo.cs       |   110 +
 .../Impl/Messaging/MessageListenerHolder.cs     |     4 +-
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |    93 -
 .../PersistentStore/PersistentStoreMetrics.cs   |     2 +
 .../Impl/PlatformJniTarget.cs                   |   254 +-
 .../Impl/PlatformTargetAdapter.cs               |    13 +-
 .../Impl/Resource/ResourceProcessor.cs          |     2 +-
 .../Impl/Resource/ResourceTypeDescriptor.cs     |     2 +-
 .../Impl/Services/ServiceProxySerializer.cs     |    62 +-
 .../Impl/Services/Services.cs                   |   112 +-
 .../dotnet/Apache.Ignite.Core/Impl/Shell.cs     |    65 +
 .../Impl/Unmanaged/IUnmanagedTarget.cs          |    42 -
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |    96 -
 .../Impl/Unmanaged/Jni/AppDomains.cs            |   135 +
 .../Impl/Unmanaged/Jni/CallbackDelegates.cs     |    52 +
 .../Impl/Unmanaged/Jni/Callbacks.cs             |   292 +
 .../Impl/Unmanaged/Jni/ConsoleWriter.cs         |    47 +
 .../Impl/Unmanaged/Jni/DllLoader.cs             |   210 +
 .../Impl/Unmanaged/Jni/Env.cs                   |   505 +
 .../Impl/Unmanaged/Jni/EnvDelegates.cs          |   109 +
 .../Impl/Unmanaged/Jni/EnvInterface.cs          |   263 +
 .../Impl/Unmanaged/Jni/GlobalRef.cs             |    87 +
 .../Impl/Unmanaged/Jni/JniResult.cs             |    35 +
 .../Impl/Unmanaged/Jni/Jvm.cs                   |   299 +
 .../Impl/Unmanaged/Jni/JvmDelegates.cs          |    31 +
 .../Impl/Unmanaged/Jni/JvmDll.cs                |   414 +
 .../Impl/Unmanaged/Jni/JvmInitArgs.cs           |    33 +
 .../Impl/Unmanaged/Jni/JvmInterface.cs          |    40 +
 .../Impl/Unmanaged/Jni/JvmOption.cs             |    34 +
 .../Impl/Unmanaged/Jni/MethodId.cs              |   157 +
 .../Impl/Unmanaged/Jni/NativeMethod.cs          |    48 +
 .../Apache.Ignite.Core/Impl/Unmanaged/Os.cs     |    72 +
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |    38 -
 .../Impl/Unmanaged/UnmanagedCallbackOp.cs       |     3 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |   307 +-
 .../Impl/Unmanaged/UnmanagedContext.cs          |    53 -
 .../Unmanaged/UnmanagedNonReleaseableTarget.cs  |    70 -
 .../Impl/Unmanaged/UnmanagedTarget.cs           |    77 -
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   252 +-
 .../dotnet/Apache.Ignite.Core/NuGet/Install.ps1 |     2 +-
 .../Apache.Ignite.Core/NuGet/PostBuild.ps1      |     6 +-
 .../Apache.Ignite.Core/NuGet/Uninstall.ps1      |     2 +-
 .../PersistentStore/CheckpointWriteOrder.cs     |    40 +
 .../PersistentStore/IPersistentStoreMetrics.cs  |     2 +
 .../PersistentStoreConfiguration.cs             |    35 +-
 .../PersistentStore/WalMode.cs                  |     3 +
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../Resource/InstanceResourceAttribute.cs       |     3 +-
 .../Apache.Ignite.Core/Services/IServices.cs    |    32 +
 .../Services/ServiceConfiguration.cs            |    70 +-
 .../Services/ServiceDeploymentException.cs      |   143 +
 .../Services/ServiceInvocationException.cs      |     9 +-
 .../dotnet/Apache.Ignite.Core/build-common.ps1  |    77 -
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../Apache.Ignite.EntityFramework.csproj        |     3 +-
 .../Apache.Ignite.EntityFramework.nuspec        |     4 +
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../Apache.Ignite.Linq.csproj                   |     3 +-
 .../Apache.Ignite.Linq.nuspec                   |     5 +
 .../CacheClientLinqExtensions.cs                |   131 +
 .../Apache.Ignite.Linq/CacheExtensions.cs       |   195 -
 .../Apache.Ignite.Linq/CacheLinqExtensions.cs   |   197 +
 .../Apache.Ignite.Linq/ICacheQueryable.cs       |     1 +
 .../Impl/CacheFieldsQueryExecutor.cs            |     3 +-
 .../Impl/CacheFieldsQueryProvider.cs            |     2 +-
 .../Impl/CacheQueryExpressionVisitor.cs         |     5 +
 .../Impl/CacheQueryModelVisitor.cs              |    28 +-
 .../Apache.Ignite.Linq/Impl/CacheQueryable.cs   |     8 +-
 .../Impl/CacheQueryableBase.cs                  |     1 +
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |    59 +-
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../dotnet/Apache.Ignite.Linq/QueryOptions.cs   |    13 +
 .../Apache.Ignite.Log4Net.csproj                |     4 +-
 .../Apache.Ignite.Log4Net.nuspec                |     4 +
 .../Properties/AssemblyInfo.cs                  |     6 +-
 .../Apache.Ignite.NLog.nuspec                   |     4 +
 .../Properties/AssemblyInfo.cs                  |     6 +-
 modules/platforms/dotnet/Apache.Ignite.ndproj   | 11141 +++++++++++++++++
 modules/platforms/dotnet/Apache.Ignite.sln      |   143 -
 .../dotnet/Apache.Ignite.sln.DotSettings        |     2 +
 .../dotnet/Apache.Ignite/IgniteRunner.cs        |     9 +-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |     6 +-
 modules/platforms/dotnet/DEVNOTES.txt           |    15 +-
 modules/platforms/dotnet/README.md              |   241 +-
 modules/platforms/dotnet/build.ps1              |   134 +-
 modules/platforms/dotnet/docfx/.gitignore       |     9 +
 .../dotnet/docfx/Apache.Ignite.docfx.json       |    88 +
 modules/platforms/dotnet/docfx/README.txt       |     2 +
 modules/platforms/dotnet/docfx/api/index.md     |     5 +
 modules/platforms/dotnet/docfx/filterConfig.yml |    19 +
 .../platforms/dotnet/docfx/generate-docs.cmd    |     1 +
 .../platforms/dotnet/docfx/images/favicon.ico   |   Bin 0 -> 1150 bytes
 .../dotnet/docfx/images/logo_ignite_32_32.png   |   Bin 0 -> 1676 bytes
 modules/platforms/dotnet/docfx/index.md         |    10 +
 modules/platforms/dotnet/docfx/toc.yml          |    19 +
 .../Apache.Ignite.Examples.csproj               |     2 +
 .../Datagrid/BinaryModeExample.cs               |    11 +-
 .../Datagrid/EntryProcessorExample.cs           |     4 +
 .../Datagrid/MultiTieredCacheExample.cs         |     2 -
 .../Datagrid/QueryDmlExample.cs                 |     5 +-
 .../Misc/LifecycleExample.cs                    |     1 -
 .../ThinClient/ThinClientPutGetExample.cs       |    92 +
 .../ThinClient/ThinClientQueryExample.cs        |   147 +
 modules/platforms/dotnet/examples/README.txt    |     2 +
 .../dotnetcore/Apache.Ignite.Examples.csproj    |    29 +
 .../dotnet/examples/dotnetcore/App.config       |    41 +
 .../dotnet/examples/dotnetcore/Employee.cs      |    67 +
 .../dotnet/examples/dotnetcore/LinqExample.cs   |   239 +
 .../dotnet/examples/dotnetcore/Organization.cs  |    58 +
 .../dotnet/examples/dotnetcore/Program.cs       |   109 +
 .../dotnet/examples/dotnetcore/PutGetExample.cs |   122 +
 .../dotnet/examples/dotnetcore/README.txt       |     8 +
 .../dotnet/examples/dotnetcore/SqlExample.cs    |   191 +
 .../dotnet/run-dotnetcore-examples.bat          |    27 +
 modules/rest-http/pom.xml                       |     2 +-
 .../http/jetty/GridJettyObjectMapper.java       |     3 +
 .../http/jetty/GridJettyRestHandler.java        |    49 +-
 .../rest/protocols/http/jetty/favicon.ico       |   Bin 1406 -> 1150 bytes
 modules/rocketmq/pom.xml                        |     2 +-
 modules/scalar-2.10/pom.xml                     |     2 +-
 modules/scalar/pom.xml                          |     2 +-
 modules/schedule/pom.xml                        |     2 +-
 modules/slf4j/pom.xml                           |     2 +-
 .../apache/ignite/logger/slf4j/Slf4jLogger.java |     6 +
 modules/spark-2.10/pom.xml                      |     2 +-
 modules/spark/pom.xml                           |     2 +-
 .../org/apache/ignite/spark/IgniteContext.scala |     6 +-
 .../ignite/testsuites/IgniteRDDTestSuite.java   |     2 +-
 .../org/apache/ignite/spark/IgniteRDDSpec.scala |    11 +-
 modules/spring-data/pom.xml                     |     9 +-
 modules/spring/pom.xml                          |     2 +-
 .../org/apache/ignite/IgniteSpringBean.java     |    59 +-
 .../GridSpringBeanSerializationSelfTest.java    |     4 +-
 .../ignite/internal/IgniteSpringBeanTest.java   |     2 +-
 .../GridServiceInjectionSpringResourceTest.java |    10 +-
 ...teSpringBeanSpringResourceInjectionTest.java |   215 +
 .../ignite/spring/injection/spring-bean.xml     |    84 +
 .../testsuites/IgniteSpringTestSuite.java       |     2 +
 modules/sqlline/bin/sqlline.bat                 |   112 +
 modules/sqlline/bin/sqlline.sh                  |    54 +
 modules/sqlline/licenses/apache-2.0.txt         |   202 +
 modules/sqlline/pom.xml                         |    81 +
 modules/ssh/pom.xml                             |     2 +-
 modules/storm/pom.xml                           |     2 +-
 modules/tools/pom.xml                           |     2 +-
 .../ant/beautifier/GridJavadocAntTask.java      |     8 +-
 modules/twitter/pom.xml                         |     2 +-
 modules/urideploy/pom.xml                       |     2 +-
 modules/visor-console-2.10/pom.xml              |     2 +-
 modules/visor-console/pom.xml                   |     2 +-
 .../ignite/visor/commands/VisorConsole.scala    |    28 +-
 .../commands/alert/VisorAlertCommand.scala      |    29 +-
 .../commands/cache/VisorCacheCommand.scala      |   192 +-
 .../cache/VisorCacheRebalanceCommand.scala      |   134 +
 .../commands/common/VisorConsoleCommand.scala   |     5 +-
 .../config/VisorConfigurationCommand.scala      |    31 +-
 .../visor/commands/open/VisorOpenCommand.scala  |    34 +-
 .../commands/tasks/VisorTasksCommand.scala      |    33 +-
 .../commands/top/VisorTopologyCommand.scala     |   111 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   184 +-
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |     9 +-
 .../commands/cache/VisorCacheCommandSpec.scala  |    44 +-
 .../top/VisorActivationCommandSpec.scala        |    70 +
 .../testsuites/VisorConsoleSelfTestSuite.scala  |     3 +-
 modules/visor-plugins/pom.xml                   |     2 +-
 modules/web-console/.gitignore                  |    11 +-
 modules/web-console/DEVNOTES.txt                |    39 +-
 modules/web-console/assembly/README.txt         |    60 +
 modules/web-console/assembly/direct-install.xml |    91 +
 modules/web-console/backend/.eslintrc           |   185 -
 modules/web-console/backend/.gitignore          |     6 -
 modules/web-console/backend/app/agentSocket.js  |    16 +-
 .../web-console/backend/app/agentsHandler.js    |    86 +-
 modules/web-console/backend/app/apiServer.js    |    81 +-
 .../web-console/backend/app/browsersHandler.js  |   453 +-
 modules/web-console/backend/app/configure.js    |    14 +-
 modules/web-console/backend/app/mongo.js        |   156 +-
 modules/web-console/backend/app/mongoose.js     |     8 +-
 modules/web-console/backend/app/nconf.js        |    38 +-
 modules/web-console/backend/app/routes.js       |    26 +-
 modules/web-console/backend/app/settings.js     |    86 +-
 .../backend/config/settings.json.sample         |     8 +-
 .../ignite_modules/migrations/README.txt        |     4 +
 modules/web-console/backend/index.js            |    74 +-
 modules/web-console/backend/injector.js         |     4 +-
 modules/web-console/backend/middlewares/api.js  |     5 +
 .../1508395969410-init-registered-date.js       |    33 +
 .../web-console/backend/migrations/README.txt   |     4 +
 modules/web-console/backend/package.json        |    32 +-
 .../web-console/backend/routes/activities.js    |    37 +-
 modules/web-console/backend/routes/admin.js     |    13 +-
 modules/web-console/backend/routes/caches.js    |     6 +-
 modules/web-console/backend/routes/clusters.js  |     6 +-
 .../web-console/backend/routes/configuration.js |     6 +-
 modules/web-console/backend/routes/demo.js      |     7 +-
 .../backend/routes/demo/domains.json            |     5 +
 modules/web-console/backend/routes/domains.js   |     6 +-
 modules/web-console/backend/routes/downloads.js |     6 +-
 modules/web-console/backend/routes/igfss.js     |     6 +-
 modules/web-console/backend/routes/notebooks.js |     7 +-
 modules/web-console/backend/routes/profile.js   |    10 +-
 modules/web-console/backend/routes/public.js    |     9 +-
 modules/web-console/backend/services/Utils.js   |    51 +
 .../web-console/backend/services/activities.js  |     7 +-
 modules/web-console/backend/services/auth.js    |    27 +-
 modules/web-console/backend/services/caches.js  |    11 +-
 .../web-console/backend/services/clusters.js    |    11 +-
 .../backend/services/configurations.js          |     5 +-
 modules/web-console/backend/services/domains.js |    11 +-
 .../web-console/backend/services/downloads.js   |    13 +-
 modules/web-console/backend/services/igfss.js   |    11 +-
 modules/web-console/backend/services/mails.js   |    89 +-
 .../web-console/backend/services/notebooks.js   |    11 +-
 .../backend/services/notifications.js           |     5 +-
 .../web-console/backend/services/sessions.js    |     5 +-
 modules/web-console/backend/services/users.js   |    31 +-
 .../compose/frontend/nginx/web-console.conf     |     1 -
 .../docker/standalone/nginx/web-console.conf    |     1 -
 modules/web-console/frontend/.eslintrc          |     2 +-
 modules/web-console/frontend/.gitignore         |     8 +-
 modules/web-console/frontend/app/app.js         |    46 +-
 .../frontend/app/browserUpdate/index.js         |    34 +
 .../frontend/app/browserUpdate/style.scss       |    36 +
 .../app/components/bs-select-menu/style.scss    |    14 +-
 .../app/components/bs-select-menu/template.pug  |    15 +-
 .../cluster-select/cluster-select.controller.js |    64 -
 .../cluster-select/cluster-select.pug           |    47 -
 .../cluster-select/cluster-select.scss          |    30 -
 .../app/components/cluster-select/index.js      |    29 -
 .../components/cluster-selector/component.js    |    25 +
 .../components/cluster-selector/controller.js   |    62 +
 .../app/components/cluster-selector/index.js    |    23 +
 .../app/components/cluster-selector/style.scss  |    66 +
 .../components/cluster-selector/template.pug    |    75 +
 .../components/connected-clusters/controller.js |    36 +
 .../app/components/connected-clusters/index.js  |    29 +
 .../components/connected-clusters/style.scss    |    40 +
 .../components/connected-clusters/template.pug  |    18 +
 .../app/components/grid-export/component.js     |    52 +
 .../app/components/grid-export/index.js         |    24 +
 .../app/components/grid-export/template.pug     |    18 +
 .../components/grid-item-selected/controller.js |     3 +
 .../app/components/grid-no-data/component.js    |    33 +
 .../app/components/grid-no-data/controller.js   |    50 +
 .../app/components/grid-no-data/index.js        |    24 +
 .../app/components/grid-no-data/style.scss      |    31 +
 .../app/components/ignite-icon/directive.js     |    10 +-
 .../app/components/ignite-icon/index.js         |     2 +
 .../app/components/ignite-icon/service.js       |    32 +
 .../app/components/list-editable/component.js   |    36 +
 .../list-editable-cols/cols.directive.js        |    79 +
 .../list-editable-cols/cols.style.scss          |    51 +
 .../list-editable-cols/cols.template.pug        |    28 +
 .../components/list-editable-cols/index.js      |    28 +
 .../list-editable-cols/row.directive.js         |    40 +
 .../app/components/list-editable/controller.js  |    79 +
 .../app/components/list-editable/index.js       |    27 +
 .../app/components/list-editable/style.scss     |   132 +
 .../app/components/list-editable/template.pug   |    51 +
 .../list-of-registered-users.column-defs.js     |    49 +-
 .../list-of-registered-users.controller.js      |    23 +-
 .../list-of-registered-users.tpl.pug            |     2 +-
 .../components/pcbScaleNumber.js                |     1 +
 .../page-configure-basic/controller.js          |    10 +-
 .../mixins/pcb-form-field-size.pug              |     1 +
 .../components/page-queries/Notebook.data.js    |   168 +
 .../components/page-queries/Notebook.service.js |    74 +
 .../app/components/page-queries/controller.js   |  1935 +++
 .../app/components/page-queries/index.js        |    62 +
 .../page-queries/notebook.controller.js         |    62 +
 .../app/components/page-queries/style.scss      |    91 +
 .../components/page-queries/template.tpl.pug    |   410 +
 .../app/components/ui-grid-filters/directive.js |    62 +
 .../app/components/ui-grid-filters/index.js     |    43 +
 .../app/components/ui-grid-filters/style.scss   |    36 +
 .../app/components/ui-grid-filters/template.pug |    47 +
 .../app/components/version-picker/index.js      |     2 +-
 .../web-console-footer-links/component.js       |    23 +
 .../web-console-footer-links/style.scss         |    31 +
 .../web-console-footer-links/template.pug       |    17 +
 .../app/components/web-console-footer/index.js  |     4 +-
 .../components/web-console-footer/style.scss    |     8 +-
 .../components/web-console-footer/template.pug  |     1 +
 .../components/web-console-header/component.js  |    11 +
 .../web-console-header-extension/component.js   |    22 +
 .../web-console-header-extension/template.pug   |    15 +
 .../app/components/web-console-header/index.js  |     4 +-
 .../components/web-console-header/style.scss    |    57 +-
 .../components/web-console-header/template.pug  |    19 +-
 .../frontend/app/core/admin/Admin.data.js       |    12 +-
 .../frontend/app/data/getting-started.json      |    13 +
 .../app/directives/information/information.pug  |     2 +-
 .../app/directives/information/information.scss |     5 +-
 .../ui-ace-docker/ui-ace-docker.controller.js   |    26 +-
 .../ui-ace-java/ui-ace-java.controller.js       |   143 +-
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |   148 +-
 .../ui-ace-pom/ui-ace-pom.controller.js         |    26 +-
 .../ui-ace-sharp/ui-ace-sharp.controller.js     |    12 +-
 .../ui-ace-spring/ui-ace-spring.controller.js   |   139 +-
 .../frontend/app/filters/duration.filter.js     |     2 +-
 .../helpers/jade/form/form-field-dropdown.pug   |     2 +-
 .../frontend/app/helpers/jade/mixins.pug        |    11 +-
 .../app/modules/agent/AgentManager.service.js   |   147 +-
 .../app/modules/agent/decompress.worker.js      |    34 +
 .../app/modules/branding/branding.module.js     |     4 +-
 .../app/modules/branding/branding.provider.js   |   111 -
 .../app/modules/branding/branding.service.js    |    47 +
 .../modules/configuration/Version.service.js    |   169 -
 .../configuration/configuration.module.js       |     4 -
 .../generator/AbstractTransformer.js            |    24 +-
 .../generator/ConfigurationGenerator.js         |   298 +-
 .../generator/JavaTransformer.service.js        |    10 +-
 .../configuration/generator/Maven.service.js    |     2 +-
 .../generator/PlatformGenerator.js              |     6 +-
 .../generator/SpringTransformer.service.js      |     2 +-
 .../generator/defaults/Cache.service.js         |    14 +-
 .../generator/defaults/Cluster.service.js       |    41 +
 .../app/modules/dialog/dialog.controller.js     |    26 +-
 .../app/modules/form/field/down.directive.js    |    16 +-
 .../app/modules/form/field/input/text.scss      |     1 +
 .../app/modules/form/field/up.directive.js      |    14 +-
 .../app/modules/form/group/add.directive.js     |    24 +-
 .../app/modules/form/group/tooltip.directive.js |    24 +-
 .../app/modules/navbar/userbar.directive.js     |    32 +-
 .../frontend/app/modules/sql/Notebook.data.js   |   168 -
 .../app/modules/sql/Notebook.service.js         |    74 -
 .../app/modules/sql/notebook.controller.js      |    62 -
 .../frontend/app/modules/sql/sql.controller.js  |  1786 ---
 .../frontend/app/modules/sql/sql.module.js      |    61 -
 .../app/modules/states/configuration.state.js   |     1 +
 .../states/configuration/caches/general.pug     |    23 +-
 .../states/configuration/caches/memory.pug      |    18 +-
 .../states/configuration/caches/store.pug       |     6 +-
 .../configuration/clusters/attributes.pug       |     4 +-
 .../configuration/clusters/client-connector.pug |    59 +
 .../clusters/collision/job-stealing.pug         |     4 +-
 .../states/configuration/clusters/connector.pug |     2 +-
 .../configuration/clusters/data-storage.pug     |   261 +
 .../states/configuration/clusters/memory.pug    |     6 +-
 .../configuration/clusters/persistence.pug      |     4 +-
 .../configuration/clusters/sql-connector.pug    |     4 +-
 .../states/configuration/clusters/swap.pug      |     2 +-
 .../states/configuration/domains/general.pug    |     4 +-
 .../states/configuration/domains/query.pug      |    15 +-
 .../states/configuration/domains/store.pug      |     2 +-
 .../frontend/app/modules/user/permissions.js    |     4 +-
 .../frontend/app/modules/user/user.module.js    |     7 +-
 .../frontend/app/primitives/btn/index.scss      |    17 +
 .../app/primitives/form-field/index.scss        |    17 +-
 .../frontend/app/primitives/switcher/index.pug  |     2 +-
 .../frontend/app/primitives/switcher/index.scss |    69 +-
 .../app/primitives/ui-grid-settings/index.scss  |    12 +
 .../frontend/app/primitives/ui-grid/index.scss  |     7 +
 .../frontend/app/services/Clusters.js           |     6 +
 .../frontend/app/services/JavaTypes.service.js  |    15 +
 .../app/services/LegacyTable.service.js         |     6 +-
 .../app/services/LegacyUtils.service.js         |    34 +-
 .../frontend/app/services/Messages.service.js   |    14 +-
 .../frontend/app/services/Version.service.js    |   179 +
 .../frontend/app/services/Version.spec.js       |   106 +
 .../web-console/frontend/app/services/index.js  |    23 +
 .../frontend/app/utils/SimpleWorkerPool.js      |   120 +
 .../frontend/controllers/clusters-controller.js |    69 +-
 .../frontend/controllers/domains-controller.js  |    31 +-
 modules/web-console/frontend/package.json       |    47 +-
 .../frontend/public/images/checkbox-active.svg  |    25 +
 .../frontend/public/images/checkbox.svg         |    22 +
 .../frontend/public/images/icons/alert.svg      |     1 +
 .../frontend/public/images/icons/attention.svg  |     3 +
 .../frontend/public/images/icons/checkmark.svg  |     3 +
 .../public/images/icons/connectedClusters.svg   |     1 +
 .../frontend/public/images/icons/index.js       |     7 +
 .../frontend/public/images/icons/info.svg       |     3 +
 .../frontend/public/images/icons/refresh.svg    |     1 +
 .../frontend/public/images/icons/sort.svg       |     1 +
 .../frontend/public/images/multicluster.png     |   Bin 0 -> 21921 bytes
 .../frontend/public/stylesheets/style.scss      |    22 +
 .../frontend/test/unit/Version.test.js          |   106 -
 modules/web-console/frontend/tsconfig.json      |    12 +
 .../views/configuration/clusters.tpl.pug        |    14 +-
 .../views/configuration/domains-import.tpl.pug  |     6 +
 .../frontend/views/includes/header-left.pug     |     8 +-
 .../frontend/views/includes/header-right.pug    |     8 +-
 modules/web-console/frontend/views/index.pug    |    10 +-
 .../frontend/views/settings/profile.tpl.pug     |     9 +-
 .../web-console/frontend/views/signin.tpl.pug   |     6 +
 .../web-console/frontend/views/sql/sql.tpl.pug  |   288 -
 .../frontend/views/templates/message.tpl.pug    |     1 +
 .../frontend/webpack/webpack.common.js          |     4 +-
 .../frontend/webpack/webpack.dev.babel.js       |     5 +-
 modules/web-console/pom.xml                     |   204 +-
 .../web-agent/bin/ignite-web-agent.bat          |    19 +
 .../web-agent/bin/ignite-web-agent.sh           |     5 +
 modules/web-console/web-agent/pom.xml           |     2 +-
 .../agent/db/dialect/OracleMetadataDialect.java |    13 +-
 .../agent/handlers/AbstractListener.java        |    27 +
 .../console/agent/handlers/ClusterListener.java |   184 +-
 .../ignite/console/agent/rest/RestExecutor.java |   264 +-
 .../ignite/console/agent/rest/RestResult.java   |    25 +-
 .../ignite/console/demo/AgentClusterDemo.java   |    41 +-
 .../ignite/console/demo/AgentMetadataDemo.java  |     7 +
 .../demo/service/DemoCachesLoadService.java     |     3 +-
 modules/web/ignite-appserver-test/pom.xml       |     2 +-
 modules/web/ignite-websphere-test/pom.xml       |     2 +-
 modules/web/pom.xml                             |     2 +-
 .../cache/websession/WebSessionFilter.java      |    12 +-
 .../internal/websession/WebSessionSelfTest.java |     2 +
 modules/yardstick/DEVNOTES-standalone.txt       |    11 +-
 modules/yardstick/DEVNOTES.txt                  |    11 +
 .../config/benchmark-bin-identity.properties    |     1 +
 .../config/benchmark-cache-load.properties      |     1 +
 .../config/benchmark-client-mode.properties     |     1 +
 .../config/benchmark-failover.properties        |     1 +
 .../yardstick/config/benchmark-full.properties  |     1 +
 .../yardstick/config/benchmark-h2.properties    |     1 +
 .../yardstick/config/benchmark-ml.properties    |    90 +
 .../config/benchmark-multicast.properties       |     1 +
 .../yardstick/config/benchmark-mysql.properties |     1 +
 .../yardstick/config/benchmark-pgsql.properties |     1 +
 .../config/benchmark-put-indexed-val.properties |     1 +
 .../benchmark-query-put-separated.properties    |     1 +
 .../yardstick/config/benchmark-query.properties |     1 +
 .../config/benchmark-remote.properties          |     1 +
 .../config/benchmark-sql-dml.properties         |     1 +
 .../yardstick/config/benchmark-store.properties |     1 +
 modules/yardstick/config/benchmark.properties   |     1 +
 modules/yardstick/pom-standalone.xml            |    58 +-
 modules/yardstick/pom.xml                       |    58 +-
 .../yardstick/IgniteBenchmarkArguments.java     |     8 +-
 .../org/apache/ignite/yardstick/IgniteNode.java |    21 +-
 .../cache/WaitMapExchangeFinishCallable.java    |     4 +-
 .../cache/jdbc/JdbcAbstractBenchmark.java       |     4 +-
 .../yardstick/cache/jdbc/RdbmsBenchmark.java    |     6 +-
 .../apache/ignite/yardstick/ml/DataChanger.java |    65 +
 ...niteKMeansDistributedClustererBenchmark.java |    75 +
 .../IgniteKMeansLocalClustererBenchmark.java    |    50 +
 .../yardstick/ml/clustering/package-info.java   |    22 +
 .../IgniteCholeskyDecompositionBenchmark.java   |    69 +
 .../IgniteEigenDecompositionBenchmark.java      |    69 +
 .../IgniteLUDecompositionBenchmark.java         |    75 +
 ...niteSingularValueDecompositionBenchmark.java |    59 +
 .../ml/decomposition/package-info.java          |    22 +
 .../math/IgniteAbstractMatrixMulBenchmark.java  |   107 +
 ...niteDenseLocalOffHeapMatrixMulBenchmark.java |    32 +
 ...gniteDenseLocalOnHeapMatrixMulBenchmark.java |    32 +
 ...parseBlockDistributedMatrixMulBenchmark.java |    32 +
 ...niteSparseDistributedMatrixMulBenchmark.java |    35 +
 .../IgniteSparseLocalMatrixMulBenchmark.java    |    32 +
 .../ignite/yardstick/ml/math/package-info.java  |    22 +
 .../ignite/yardstick/ml/package-info.java       |    22 +
 ...iteOLSMultipleLinearRegressionBenchmark.java |    92 +
 .../yardstick/ml/regression/package-info.java   |    22 +
 .../IgniteColumnDecisionTreeGiniBenchmark.java  |    70 +
 ...niteColumnDecisionTreeVarianceBenchmark.java |    71 +
 .../yardstick/ml/trees/SplitDataGenerator.java  |   426 +
 .../ignite/yardstick/ml/trees/package-info.java |    22 +
 modules/yarn/pom.xml                            |     2 +-
 .../apache/ignite/yarn/ApplicationMaster.java   |     9 +-
 modules/zeromq/pom.xml                          |     2 +-
 modules/zookeeper/pom.xml                       |     2 +-
 parent/pom.xml                                  |    17 +
 pom.xml                                         |   182 +-
 2486 files changed, 182749 insertions(+), 46853 deletions(-)
----------------------------------------------------------------------



[14/31] ignite git commit: Test fix

Posted by vo...@apache.org.
Test fix


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

Branch: refs/heads/ignite-6022-proto
Commit: 790cd266cfe77ee84a4887697179e946397792aa
Parents: b30729e
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Dec 15 19:10:45 2017 +0300
Committer: Alexander Paschenko <al...@gmail.com>
Committed: Fri Dec 15 19:10:45 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/790cd266/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java
index 5e2e39e..8609615 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java
@@ -260,7 +260,8 @@ public class JdbcThinBatchSelfTest extends JdbcThinAbstractDmlStatementSelfTest
             pstmt.executeBatch();
 
             fail("BatchUpdateException must be thrown");
-        } catch(BatchUpdateException e) {
+        }
+        catch(BatchUpdateException e) {
             int [] updCnts = e.getUpdateCounts();
 
             assertEquals("Invalid update counts size", BATCH_SIZE, updCnts.length);
@@ -268,7 +269,7 @@ public class JdbcThinBatchSelfTest extends JdbcThinAbstractDmlStatementSelfTest
             for (int i = 0; i < BATCH_SIZE; ++i)
                 assertEquals("Invalid update count",1, updCnts[i]);
 
-            if (!e.getMessage().contains("Failed to execute SQL query.")) {
+            if (!e.getMessage().contains("Value conversion failed")) {
                 log.error("Invalid exception: ", e);
 
                 fail();


[21/31] ignite git commit: Minors.

Posted by vo...@apache.org.
Minors.


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

Branch: refs/heads/ignite-6022-proto
Commit: 5d99bebc95b276b2f0a5d11d757d372a2d1c6453
Parents: 850cd7e
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 18 12:10:23 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 18 12:10:23 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/query/h2/dml/UpdatePlanBuilder.java     | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5d99bebc/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index 563456b..5ffd264 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -264,11 +264,6 @@ public final class UpdatePlanBuilder {
             null,
             distributed
         );
-
-        /*
-        if (sel == null)
-                elRows = merge.rows();
-         */
     }
 
     /**