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/03/22 11:12:47 UTC

[01/23] ignite git commit: IGNITE-4671 - FairAffinityFunction fails on node restart with backupFilter set and no backups

Repository: ignite
Updated Branches:
  refs/heads/ignite-4565-ddl 41a7beac9 -> 815f0cbdc


IGNITE-4671 - FairAffinityFunction fails on node restart with backupFilter set and no backups


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

Branch: refs/heads/ignite-4565-ddl
Commit: 9df5e94d5cf14ddd55e29b81989177a7798f7e1a
Parents: caa3acb
Author: dkarachentsev <dk...@gridgain.com>
Authored: Tue Feb 21 15:34:59 2017 +0300
Committer: dkarachentsev <dk...@gridgain.com>
Committed: Tue Feb 21 15:34:59 2017 +0300

----------------------------------------------------------------------
 .../affinity/fair/FairAffinityFunction.java     |  33 ++---
 .../affinity/fair/FairAffinityNodesRestart.java | 130 +++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 3 files changed, 149 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9df5e94d/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
index cf1cb02..7acb5b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
@@ -911,37 +911,37 @@ public class FairAffinityFunction implements AffinityFunction {
             if (exclNeighbors)
                 return allowNeighbors || !neighborsContainPartition(node, part);
             else if (affinityBackupFilter != null) {
-                List<ClusterNode> assigment = assignments.get(part);
+                List<ClusterNode> assignment = assignments.get(part);
 
-                assert assigment.size() > 0;
+                if (assignment.isEmpty())
+                    return true;
 
                 List<ClusterNode> newAssignment;
 
                 if (tier == 0) {
-                    for (int t = 1; t < assigment.size(); t++) {
-                        newAssignment = new ArrayList<>(assigment.size() - 1);
+                    for (int t = 1; t < assignment.size(); t++) {
+                        newAssignment = new ArrayList<>(assignment.size() - 1);
 
                         newAssignment.add(node);
 
                         if (t != 1)
-                            newAssignment.addAll(assigment.subList(1, t));
+                            newAssignment.addAll(assignment.subList(1, t));
 
-                        if (t + 1 < assigment.size())
-                            newAssignment.addAll(assigment.subList(t + 1, assigment.size()));
+                        if (t + 1 < assignment.size())
+                            newAssignment.addAll(assignment.subList(t + 1, assignment.size()));
 
-                        if (!affinityBackupFilter.apply(assigment.get(t), newAssignment))
+                        if (!affinityBackupFilter.apply(assignment.get(t), newAssignment))
                             return false;
-
                     }
 
                     return true;
                 }
-                else if (tier < assigment.size()) {
-                    newAssignment = new ArrayList<>(assigment.size() - 1);
+                else if (tier < assignment.size()) {
+                    newAssignment = new ArrayList<>(assignment.size() - 1);
 
                     int i = 0;
 
-                    for (ClusterNode assignmentNode: assigment) {
+                    for (ClusterNode assignmentNode: assignment) {
                         if (i != tier)
                             newAssignment.add(assignmentNode);
 
@@ -949,17 +949,18 @@ public class FairAffinityFunction implements AffinityFunction {
                     }
                 }
                 else
-                    newAssignment = assigment;
+                    newAssignment = assignment;
 
                 return affinityBackupFilter.apply(node, newAssignment);
             }
             else if (backupFilter != null) {
                 if (tier == 0) {
-                    List<ClusterNode> assigment = assignments.get(part);
+                    List<ClusterNode> assignment = assignments.get(part);
 
-                    assert assigment.size() > 0;
+                    if (assignment.isEmpty())
+                        return true;
 
-                    List<ClusterNode> backups = assigment.subList(1, assigment.size());
+                    List<ClusterNode> backups = assignment.subList(1, assignment.size());
 
                     return !F.exist(backups, new IgnitePredicate<ClusterNode>() {
                         @Override public boolean apply(ClusterNode n) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9df5e94d/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityNodesRestart.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityNodesRestart.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityNodesRestart.java
new file mode 100644
index 0000000..37f1bfb
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityNodesRestart.java
@@ -0,0 +1,130 @@
+/*
+ * 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.cache.affinity.fair;
+
+import java.util.List;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.typedef.P2;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Tests that FairAffinityFunction doesn't throw exception on nodes restart,
+ * with backup filter set and 0 cache backups.
+ */
+public class FairAffinityNodesRestart extends GridCommonAbstractTest {
+    /** */
+    private final static P2<ClusterNode, ClusterNode> BACKUP_FILTER = new P2<ClusterNode, ClusterNode>() {
+        @Override public boolean apply(ClusterNode node, ClusterNode node2) {
+            return true;
+        }
+    };
+
+    /** */
+    private final static P2<ClusterNode, List<ClusterNode>> AFF_BACKUP_FILTER = new P2<ClusterNode, List<ClusterNode>>() {
+        @Override public boolean apply(ClusterNode node, List<ClusterNode> nodes) {
+            return true;
+        }
+    };
+
+    /** */
+    private boolean affBackup;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = new CacheConfiguration("fair-cache");
+
+        FairAffinityFunction aff = new FairAffinityFunction(32);
+
+        if (!affBackup)
+            aff.setBackupFilter(BACKUP_FILTER);
+        else
+            aff.setAffinityBackupFilter(AFF_BACKUP_FILTER);
+
+        ccfg.setAffinity(aff);
+        ccfg.setBackups(0);
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @param idx Node index.
+     * @return Future.
+     */
+    private IgniteInternalFuture<IgniteEx> startAsyncGrid(final int idx) {
+        return GridTestUtils.runAsync(new Callable<IgniteEx>() {
+            @Override public IgniteEx call() throws Exception {
+                return startGrid(idx);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBackupFilter() throws Exception {
+        affBackup = false;
+
+        check();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityBackupFilter() throws Exception {
+        affBackup = true;
+
+        check();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void check() throws Exception {
+        for (int i = 0; i < 2; i++) {
+            IgniteInternalFuture<IgniteEx> fut0 = startAsyncGrid(0);
+            IgniteInternalFuture<IgniteEx> fut1 = startAsyncGrid(1);
+            IgniteInternalFuture<IgniteEx> fut2 = startAsyncGrid(2);
+
+            IgniteEx ignite = fut0.get();
+            fut1.get();
+            fut2.get();
+
+            IgniteCache<Integer, String> cache = ignite.cache("fair-cache");
+
+            for (int j = 0; j < 100; j++)
+                cache.put(i, String.valueOf(i));
+
+            stopGrid(0);
+            stopGrid(1);
+            stopGrid(2);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9df5e94d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 092d95e..a24f020 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -27,6 +27,7 @@ import org.apache.ignite.cache.affinity.AffinityHistoryCleanupTest;
 import org.apache.ignite.cache.affinity.fair.FairAffinityDynamicCacheSelfTest;
 import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionNodesSelfTest;
 import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionSelfTest;
+import org.apache.ignite.cache.affinity.fair.FairAffinityNodesRestart;
 import org.apache.ignite.cache.affinity.local.LocalAffinityFunctionTest;
 import org.apache.ignite.cache.store.GridCacheBalancingStoreSelfTest;
 import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest;
@@ -231,6 +232,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(FairAffinityFunctionNodesSelfTest.class);
         suite.addTestSuite(FairAffinityFunctionSelfTest.class);
         suite.addTestSuite(FairAffinityDynamicCacheSelfTest.class);
+        suite.addTestSuite(FairAffinityNodesRestart.class);
         suite.addTestSuite(GridCacheAffinityBackupsSelfTest.class);
         suite.addTestSuite(IgniteCacheAffinitySelfTest.class);
         suite.addTestSuite(AffinityClientNodeSelfTest.class);


[02/23] ignite git commit: Implemented support for enforce join order flag. (cherry picked from commit a7f77d4)

Posted by vo...@apache.org.
Implemented support for enforce join order flag.
(cherry picked from commit a7f77d4)


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

Branch: refs/heads/ignite-4565-ddl
Commit: 5f5cce4f36e6bc0a3468ba1d80fa5f3d158e927f
Parents: 9df5e94
Author: Alexey Kuznetsov <ak...@gridgain.com>
Authored: Wed Mar 1 22:09:40 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Mar 2 10:49:48 2017 +0700

----------------------------------------------------------------------
 .../internal/visor/query/VisorQueryArgV3.java   | 51 ++++++++++++++++++++
 .../internal/visor/query/VisorQueryJob.java     |  6 +--
 .../resources/META-INF/classnames.properties    |  1 +
 3 files changed, 55 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5cce4f/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java
new file mode 100644
index 0000000..f32c00a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.query;
+
+/**
+ * Arguments for {@link VisorQueryTask}.
+ */
+public class VisorQueryArgV3 extends VisorQueryArgV2 {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Enforce join order flag. */
+    private final boolean enforceJoinOrder;
+
+    /**
+     * @param cacheName Cache name for query.
+     * @param qryTxt Query text.
+     * @param distributedJoins If {@code true} then distributed joins enabled.
+     * @param enforceJoinOrder If {@code true} then enforce join order.
+     * @param loc Flag whether to execute query locally.
+     * @param pageSize Result batch size.
+     */
+    public VisorQueryArgV3(String cacheName, String qryTxt,
+        boolean distributedJoins, boolean enforceJoinOrder, boolean loc, int pageSize) {
+        super(cacheName, qryTxt, distributedJoins, loc, pageSize);
+
+        this.enforceJoinOrder = enforceJoinOrder;
+    }
+
+    /**
+     * @return Enforce join order flag.
+     */
+    public boolean enforceJoinOrder() {
+        return enforceJoinOrder;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5cce4f/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
index c66b2dd..1ac90ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
@@ -131,9 +131,8 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
                 if (scanWithFilter) {
                     boolean caseSensitive = qryTxt.startsWith(SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE);
 
-                    String ptrn = caseSensitive
-                        ? qryTxt.substring(SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE.length())
-                        : qryTxt.substring(SCAN_CACHE_WITH_FILTER.length());
+                    String ptrn = qryTxt.substring(
+                        caseSensitive ? SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE.length() : SCAN_CACHE_WITH_FILTER.length());
 
                     filter = new VisorQueryScanSubstringFilter(caseSensitive, ptrn);
                 }
@@ -162,6 +161,7 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
                 qry.setPageSize(arg.pageSize());
                 qry.setLocal(arg.local());
                 qry.setDistributedJoins(arg instanceof VisorQueryArgV2 && ((VisorQueryArgV2)arg).distributedJoins());
+                qry.setEnforceJoinOrder(arg instanceof VisorQueryArgV3 && ((VisorQueryArgV3)arg).enforceJoinOrder());
 
                 long start = U.currentTimeMillis();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5cce4f/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index df605da..7bebe24 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1819,6 +1819,7 @@ org.apache.ignite.internal.visor.node.VisorSpisConfiguration
 org.apache.ignite.internal.visor.node.VisorTransactionConfiguration
 org.apache.ignite.internal.visor.query.VisorQueryArg
 org.apache.ignite.internal.visor.query.VisorQueryArgV2
+org.apache.ignite.internal.visor.query.VisorQueryArgV3
 org.apache.ignite.internal.visor.query.VisorQueryCleanupTask
 org.apache.ignite.internal.visor.query.VisorQueryCleanupTask$VisorQueryCleanupJob
 org.apache.ignite.internal.visor.query.VisorQueryField


[19/23] ignite git commit: Wired things up.

Posted by vo...@apache.org.
Wired things up.


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

Branch: refs/heads/ignite-4565-ddl
Commit: edaf29af95e1503d95001d1bede5a3d52d120ca5
Parents: 013faff
Author: devozerov <vo...@gridgain.com>
Authored: Wed Mar 22 11:30:55 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Mar 22 11:30:55 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 223 +++++++++++++------
 .../query/ddl/IndexOperationState.java          |  12 +-
 2 files changed, 153 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/edaf29af/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 92bc7c1..2e5ee50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -47,9 +47,12 @@ import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
 import org.apache.ignite.internal.processors.query.ddl.AbstractIndexOperation;
 import org.apache.ignite.internal.processors.query.ddl.CreateIndexOperation;
+import org.apache.ignite.internal.processors.query.ddl.DropIndexOperation;
 import org.apache.ignite.internal.processors.query.ddl.IndexAcceptDiscoveryMessage;
 import org.apache.ignite.internal.processors.query.ddl.IndexFinishDiscoveryMessage;
 import org.apache.ignite.internal.processors.query.ddl.IndexOperationCancellationToken;
+import org.apache.ignite.internal.processors.query.ddl.IndexOperationHandler;
+import org.apache.ignite.internal.processors.query.ddl.IndexOperationState;
 import org.apache.ignite.internal.processors.query.ddl.IndexOperationStatusRequest;
 import org.apache.ignite.internal.processors.query.ddl.IndexOperationStatusResponse;
 import org.apache.ignite.internal.processors.query.ddl.IndexProposeDiscoveryMessage;
@@ -89,6 +92,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 import static org.apache.ignite.internal.GridTopic.TOPIC_DYNAMIC_SCHEMA;
 import static org.apache.ignite.internal.IgniteComponentType.INDEXING;
+import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL;
 
 /**
  * Indexing processor.
@@ -127,6 +131,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /** Index create/drop client futures. */
     private final ConcurrentMap<UUID, QueryIndexClientFuture> idxCliFuts = new ConcurrentHashMap<>();
 
+    /** Index operation states. */
+    private final ConcurrentHashMap<UUID, IndexOperationState> idxOpStates = new ConcurrentHashMap<>();
+
     /** IO message listener. */
     private final GridMessageListener ioLsnr;
 
@@ -425,73 +432,90 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     public void onIndexAcceptMessage(IndexAcceptDiscoveryMessage msg) {
         idxLock.writeLock().lock();
 
-        // TODO
-
         try {
-//            // Validate.
-//            if (op instanceof CreateIndexOperation) {
-//                CreateIndexOperation op0 = (CreateIndexOperation)op;
-//
-//                QueryIndex idx = op0.index();
-//
-//                // Check conflict with other indexes.
-//                String idxName = op0.index().getName();
-//
-//                QueryIndexKey idxKey = new QueryIndexKey(space, idxName);
-//
-//                QueryIndexDescriptorImpl oldIdx = idxs.get(idxKey);
-//
-//                if (oldIdx != null) {
-//                    if (!op0.ifNotExists())
-//                        msg.onError(ctx.localNodeId(), "Index already exists [space=" + space + ", index=" + idxName);
-//
-//                    return;
-//                }
-//
-//                // Make sure table exists.
-//                String tblName = op0.tableName();
-//
-//                QueryTypeDescriptorImpl typeDesc = null;
-//
-//                for (QueryTypeDescriptorImpl type : types.values()) {
-//                    if (F.eq(tblName, type.tableName())) {
-//                        typeDesc = type;
-//
-//                        break;
-//                    }
-//                }
-//
-//                if (typeDesc == null) {
-//                    msg.onError(ctx.localNodeId(), "Table doesn't exist: " + tblName);
-//
-//                    return;
-//                }
-//
-//                // Make sure that index can be applied to the given table.
-//                for (String idxField : idx.getFieldNames()) {
-//                    if (!typeDesc.fields().containsKey(idxField)) {
-//                        msg.onError(ctx.localNodeId(), "Field doesn't exist: " + idxField);
-//
-//                        return;
-//                    }
-//                }
-//            }
-//            else if (op instanceof DropIndexOperation) {
-//                DropIndexOperation op0 = (DropIndexOperation)op;
-//
-//                String idxName = op0.indexName();
-//
-//                QueryIndexKey idxKey = new QueryIndexKey(space, idxName);
-//
-//                QueryIndexDescriptorImpl oldIdx = idxs.get(idxKey);
-//
-//                if (oldIdx == null) {
-//                    if (!op0.ifExists())
-//                        msg.onError(ctx.localNodeId(), "Index doesn't exist: " + idxName);
-//                }
-//            }
-//            else
-//                msg.onError(ctx.localNodeId(), "Unsupported operation: " + op);
+            AbstractIndexOperation op = msg.operation();
+            String space = op.space();
+
+            boolean completed = false;
+            String errMsg = null;
+
+            // Validate.
+            if (op instanceof CreateIndexOperation) {
+                CreateIndexOperation op0 = (CreateIndexOperation)op;
+
+                QueryIndex idx = op0.index();
+
+                // Make sure table exists.
+                String tblName = op0.tableName();
+
+                QueryTypeDescriptorImpl type0 = null;
+
+                for (QueryTypeDescriptorImpl type : types.values()) {
+                    if (F.eq(tblName, type.tableName())) {
+                        type0 = type;
+
+                        break;
+                    }
+                }
+
+                if (type0 == null) {
+                    completed = true;
+                    errMsg = "Table doesn't exist: " + tblName;
+                }
+                else {
+                    // Make sure that index can be applied to the given table.
+                    for (String idxField : idx.getFieldNames()) {
+                        if (!type0.fields().containsKey(idxField)) {
+                            completed = true;
+                            errMsg = "Field doesn't exist: " + idxField;
+                        }
+                    }
+                }
+
+                // Check conflict with other indexes.
+                if (errMsg != null) {
+                    String idxName = op0.index().getName();
+
+                    QueryIndexKey idxKey = new QueryIndexKey(space, idxName);
+
+                    if (idxs.get(idxKey) != null) {
+                        completed = true;
+
+                        if (!op0.ifNotExists())
+                            errMsg = "Index already exists [space=" + space + ", index=" + idxName + ']';
+                    }
+                }
+            }
+            else if (op instanceof DropIndexOperation) {
+                DropIndexOperation op0 = (DropIndexOperation)op;
+
+                String idxName = op0.indexName();
+
+                QueryIndexDescriptorImpl oldIdx = idxs.get(new QueryIndexKey(space, idxName));
+
+                if (oldIdx == null) {
+                    completed = true;
+
+                    if (!op0.ifExists())
+                        errMsg = "Index doesn't exist: " + idxName;
+                }
+                else {
+                    // Make sure that index relate to expected table.
+                    if (F.eq(oldIdx.typeDescriptor().tableName(), op.tableName())) {
+                        completed = true;
+                        errMsg = "Index doesn't exist: " + idxName;
+                    }
+                }
+            }
+            else {
+                completed = true;
+                errMsg = "Unsupported operation: " + op;
+            }
+
+            // Start async operation.
+            Exception err = errMsg != null ? new IgniteException(errMsg) : null;
+
+            startIndexOperation(op, completed, err);
         }
         finally {
             idxLock.writeLock().unlock();
@@ -499,12 +523,26 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Handle index accept message.
+     * Handle index finish message.
      *
      * @param msg Message.
      */
     public void onIndexFinishMessage(IndexFinishDiscoveryMessage msg) {
+        UUID opId = msg.operation().operationId();
+
+        idxOpStates.remove(opId);
+
+        QueryIndexClientFuture cliFut = idxCliFuts.remove(opId);
+
+        if (cliFut != null) {
+            if (msg.hasError()) {
+                IgniteException err = new IgniteException(msg.errorMessage());
 
+                cliFut.onDone(err); // TODO: Better message and code handling.
+            }
+            else
+                cliFut.onDone();
+        }
     }
 
     /**
@@ -513,7 +551,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param node Node.
      */
     public void onNodeLeave(ClusterNode node) {
-        // TODO.
+        for (IndexOperationState idxOpState : idxOpStates.values())
+            idxOpState.onNodeLeave(node.id());
     }
 
     /**
@@ -1317,7 +1356,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param err Error.
      */
     private void startIndexOperation(AbstractIndexOperation op, boolean completed, Exception err) {
-        // TODO
+        IndexOperationHandler hnd = new IndexOperationHandler(ctx, this, op, completed, err);
+
+        hnd.init();
+
+        IndexOperationState state = new IndexOperationState(ctx, this, hnd);
+
+        idxOpStates.put(op.operationId(), state);
+
+        state.tryMap();
     }
 
     /**
@@ -1326,7 +1373,35 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param req Status request.
      */
     private void processStatusRequest(IndexOperationStatusRequest req) {
-        // TODO
+        UUID opId = req.operationId();
+
+        IndexOperationState idxOpState = idxOpStates.get(opId);
+
+        if (idxOpState != null)
+            idxOpState.onStatusRequest(req.senderNodeId());
+        else
+            // Operation completed successfully.
+            sendStatusResponse(req.senderNodeId(), opId, null);
+    }
+
+    /**
+     * Send status response.
+     *
+     * @param destNodeId Destination node ID.
+     * @param opId Operation ID.
+     * @param errMsg Error message.
+     */
+    public void sendStatusResponse(UUID destNodeId, UUID opId, String errMsg) {
+        try {
+            IndexOperationStatusResponse resp = new IndexOperationStatusResponse(ctx.localNodeId(), opId, errMsg);
+
+            // TODO: Proper pool!
+            ctx.io().sendToGridTopic(destNodeId, TOPIC_DYNAMIC_SCHEMA, resp, PUBLIC_POOL);
+        }
+        catch (IgniteCheckedException e) {
+            // Node left, ignore.
+            // TODO: Better logging all over the state and handler to simplify debug!
+        }
     }
 
     /**
@@ -1335,7 +1410,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param resp Status response.
      */
     private void processStatusResponse(IndexOperationStatusResponse resp) {
-        // TODO
+        IndexOperationState idxOpState = idxOpStates.get(resp.operationId());
+
+        if (idxOpState != null)
+            idxOpState.onNodeFinished(resp.senderNodeId(), resp.errorMessage());
+        else {
+            // TODO: Log!
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/edaf29af/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexOperationState.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexOperationState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexOperationState.java
index 75ab2e7..88ba5cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexOperationState.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexOperationState.java
@@ -183,17 +183,7 @@ public class IndexOperationState {
                     errMsg = e.getMessage();
                 }
 
-                try {
-                    IndexOperationStatusResponse resp =
-                        new IndexOperationStatusResponse(ctx.localNodeId(), hnd.operation().operationId(), errMsg);
-
-                    // TODO: Proper pool!
-                    ctx.io().sendToGridTopic(nodeId, TOPIC_DYNAMIC_SCHEMA, resp, PUBLIC_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    // Node left, ignore.
-                    // TODO: Better logging all over the state and handler to simplify debug!
-                }
+                qryProc.sendStatusResponse(nodeId, hnd.operation().operationId(), errMsg);
             }
         });
     }


[06/23] ignite git commit: IGNITE-3386 - Reentrant lock is lost when owner leaves topology

Posted by vo...@apache.org.
IGNITE-3386 - Reentrant lock is lost when owner leaves topology


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

Branch: refs/heads/ignite-4565-ddl
Commit: c2b201ed5bf4bea5dd5aa013f685fdfa02544527
Parents: 840ab99
Author: Evgenii Zhuravlev <ez...@gridgain.com>
Authored: Thu Mar 2 10:27:13 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Mar 2 10:32:50 2017 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java |  1 +
 .../datastructures/DataStructuresProcessor.java |  3 +-
 .../datastructures/GridCacheLockImpl.java       | 17 +++++-
 .../internal/GridCacheRecreateLockTest.java     | 62 ++++++++++++++++++++
 .../testsuites/IgniteComputeGridTestSuite.java  |  2 +
 5 files changed, 82 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c2b201ed/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 0de08d5..7f8974c 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -527,6 +527,7 @@ public interface Ignite extends AutoCloseable {
      *      all threads on other nodes waiting to acquire lock are interrupted.
      * @param fair If {@code True}, fair lock will be created.
      * @param create Boolean flag indicating whether data structure should be created if does not exist.
+     *      Will re-create lock if the node that stored the lock left topology and there are no backups left.
      * @return ReentrantLock for the given name.
      * @throws IgniteException If reentrant lock could not be fetched or created.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2b201ed/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 1cad22f..698efd9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -1396,7 +1396,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                         name,
                         key,
                         reentrantLockView,
-                        dsCacheCtx);
+                        dsCacheCtx,
+                        create);
 
                     dsMap.put(key, reentrantLock0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2b201ed/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
index 3ab7289..1cf78fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
@@ -105,6 +105,9 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
     /** Flag indicating that every operation on this lock should be interrupted. */
     private volatile boolean interruptAll;
 
+    /** Re-create flag. */
+    private volatile boolean reCreate;
+
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -522,7 +525,14 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
                                 GridCacheLockState val = lockView.get(key);
 
                                 if (val == null)
-                                    throw new IgniteCheckedException("Failed to find reentrant lock with given name: " + name);
+                                    if (reCreate) {
+                                        val = new GridCacheLockState(0, ctx.nodeId(), 0, failoverSafe, fair);
+
+                                        lockView.put(key, val);
+                                    }
+                                    else
+                                        throw new IgniteCheckedException("Failed to find reentrant lock with " +
+                                            "the given name: " + name);
 
                                 final long newThreadID = newThread.getId();
 
@@ -1048,12 +1058,14 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
      * @param key Reentrant lock key.
      * @param lockView Reentrant lock projection.
      * @param ctx Cache context.
+     * @param reCreate If {@code true} reentrant lock will be re-created in case it is not in cache.
      */
     @SuppressWarnings("unchecked")
     public GridCacheLockImpl(String name,
         GridCacheInternalKey key,
         IgniteInternalCache<GridCacheInternalKey, GridCacheLockState> lockView,
-        GridCacheContext ctx) {
+        GridCacheContext ctx,
+        boolean reCreate) {
         assert name != null;
         assert key != null;
         assert ctx != null;
@@ -1063,6 +1075,7 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
         this.key = key;
         this.lockView = lockView;
         this.ctx = ctx;
+        this.reCreate = reCreate;
 
         log = ctx.logger(getClass());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2b201ed/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
new file mode 100644
index 0000000..ae4ef87
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
@@ -0,0 +1,62 @@
+package org.apache.ignite.internal;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLock;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.testframework.junits.common.GridCommonTest;
+
+/**
+ * Create lock after owner node left topology test
+ */
+@GridCommonTest(group = "Kernal Self")
+public class GridCacheRecreateLockTest extends GridCommonAbstractTest {
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void test() throws Exception {
+        final Ignite ignite = startNodeAndLock("node1");
+
+        new Thread(new Runnable() {
+            @Override public void run() {
+                try {
+                    Thread.sleep(2000);
+                }
+                catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+
+                ignite.close();
+            }
+        }).start();
+
+        startNodeAndLock("node2");
+    }
+
+    private Ignite startNodeAndLock(String name) {
+        try {
+            IgniteConfiguration cfg = new IgniteConfiguration();
+            cfg.setGridName(name);
+
+            Ignite ignite = Ignition.start(cfg);
+
+            IgniteLock lock = ignite.reentrantLock("lock", true, true, true);
+
+            System.out.println("acquiring lock");
+
+            lock.lock();
+
+            System.out.println("acquired lock");
+
+            return ignite;
+        }
+        catch (Exception e) {
+            assertTrue(false);
+        }
+
+        return null;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2b201ed/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
index 8a501fd..7c8b6a9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.ClusterNodeMetricsSelfTest;
 import org.apache.ignite.internal.GridAffinityNoCacheSelfTest;
 import org.apache.ignite.internal.GridAffinitySelfTest;
 import org.apache.ignite.internal.GridAlwaysFailoverSpiFailSelfTest;
+import org.apache.ignite.internal.GridCacheRecreateLockTest;
 import org.apache.ignite.internal.GridCancelOnGridStopSelfTest;
 import org.apache.ignite.internal.GridCancelUnusedJobSelfTest;
 import org.apache.ignite.internal.GridCancelledJobsMetricsSelfTest;
@@ -152,6 +153,7 @@ public class IgniteComputeGridTestSuite {
         suite.addTestSuite(TaskNodeRestartTest.class);
         suite.addTestSuite(IgniteRoundRobinErrorAfterClientReconnectTest.class);
         suite.addTestSuite(PublicThreadpoolStarvationTest.class);
+        suite.addTestSuite(GridCacheRecreateLockTest.class);
 
         return suite;
     }


[23/23] ignite git commit: Removed table name from "drop index" message.

Posted by vo...@apache.org.
Removed table name from "drop index" message.


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

Branch: refs/heads/ignite-4565-ddl
Commit: 815f0cbdcc84433ff4dde5d61a6bdd3e25979889
Parents: f39de03
Author: devozerov <vo...@gridgain.com>
Authored: Wed Mar 22 13:43:17 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Mar 22 13:43:17 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 137 +++++++++++++++----
 .../processors/query/QueryIndexState.java       |   2 +-
 .../processors/query/QueryIndexStates.java      |  10 +-
 .../query/ddl/AbstractIndexOperation.java       |  14 +-
 .../query/ddl/CreateIndexOperation.java         |  13 +-
 .../query/ddl/DropIndexOperation.java           |   5 +-
 .../query/ddl/IndexOperationHandler.java        |   2 +-
 7 files changed, 133 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/815f0cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index c27b2f9..4eda6fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -78,6 +78,8 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -322,36 +324,119 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             }
         }
 
-        // Apply dynamic changes to candidates.
-        Collection<AbstractIndexOperation> ops = new ArrayList<>();
+        // Ensure that candidates has unique index names. Otherwise we will not be able to apply pending operations.
+        Map<String, QueryTypeDescriptorImpl> idxTypMap = new HashMap<>();
+
+        for (QueryTypeCandidate cand : cands) {
+            QueryTypeDescriptorImpl desc = cand.descriptor();
+
+            for (String idxName : desc.indexes().keySet()) {
+                QueryTypeDescriptorImpl oldDesc = idxTypMap.put(idxName, desc);
+
+                if (oldDesc != null)
+                    throw new IgniteException("Duplicate index name [idxName=" + idxName +
+                        ", type1=" + desc.name() + ", type2=" + oldDesc.name() + ']');
+            }
+        }
+
+        IdentityHashMap<AbstractIndexOperation, String> activeOps = new IdentityHashMap<>();
 
         if (initIdxStates != null) {
-            Map<String, QueryIndexState> readyIdxStates = initIdxStates.readyOperations();
-            Map<String, QueryIndexActiveOperation> acceptedOps = initIdxStates.acceptedActiveOperations();
+            // Apply ready operations.
+            for (Map.Entry<String, QueryIndexState> entry : initIdxStates.readyOperations().entrySet()) {
+                String idxName = entry.getKey();
+                QueryIndexState idxState = entry.getValue();
+
+                if (idxState.removed()) {
+                    // Handle remove. If relevant index is not found, this is not a problem as consistency between
+                    // nodes are not compromised
+                    QueryTypeDescriptorImpl desc = idxTypMap.remove(idxState.indexName());
+
+                    if (desc != null)
+                        QueryUtils.processDynamicIndexChange(idxName, null, desc);
+                }
+                else {
+                    // Handle create.
+                    QueryTypeDescriptorImpl desc = null;
 
-            for (QueryTypeCandidate cand : cands) {
-                QueryTypeDescriptorImpl desc = cand.descriptor();
+                    for (QueryTypeCandidate cand : cands) {
+                        if (F.eq(cand.descriptor().tableName(), idxState.tableName())) {
+                            desc = cand.descriptor();
 
-                for (Map.Entry<String, QueryIndexState> entry : readyIdxStates.entrySet()) {
-                    String idxName = entry.getKey();
-                    QueryIndexState idxState = entry.getValue();
+                            break;
+                        }
+                    }
 
-                    if (F.eq(desc.tableName(), idxState.tableName()))
-                        QueryUtils.processDynamicIndexChange(idxName, idxState.index(), desc);
+                    if (desc == null)
+                        throw new IgniteException("Table not found for index remove [idxName=" + idxName +
+                            ", tblName=" + idxState.tableName() + ']');
+
+                    QueryTypeDescriptorImpl oldDesc = idxTypMap.put(idxName, desc);
+
+                    if (oldDesc != null)
+                        throw new IgniteException("Duplicate index name [idxName=" + idxName +
+                            ", type1=" + desc.name() + ", type2=" + oldDesc.name() + ']');
+
+                    QueryUtils.processDynamicIndexChange(idxName, idxState.index(), desc);
                 }
+            }
+
+            // Apply pending operations.
+            for (Map.Entry<String, QueryIndexActiveOperation> acceptedOpEntry :
+                initIdxStates.acceptedActiveOperations().entrySet()) {
+                String errMsg = null;
+
+                String idxName = acceptedOpEntry.getKey();
+                AbstractIndexOperation op = acceptedOpEntry.getValue().operation();
+
+                if (op instanceof CreateIndexOperation) {
+                    // Handle create.
+                    CreateIndexOperation op0 = (CreateIndexOperation)op;
+
+                    QueryTypeDescriptorImpl desc = null;
+
+                    for (QueryTypeCandidate cand : cands) {
+                        if (F.eq(cand.descriptor().tableName(), op0.tableName())) {
+                            desc = cand.descriptor();
+
+                            break;
+                        }
+                    }
 
-                for (Map.Entry<String, QueryIndexActiveOperation> acceptedOpEntry : acceptedOps.entrySet()) {
-                    String idxName = acceptedOpEntry.getKey();
-                    AbstractIndexOperation op = acceptedOpEntry.getValue().operation();
+                    if (desc == null)
+                        errMsg = "Table not found: " + op0.tableName();
+                    else {
+                        QueryTypeDescriptorImpl oldDesc = idxTypMap.get(idxName);
 
-                    if (F.eq(desc.tableName(), op.tableName())) {
-                        QueryIndex idx = op instanceof CreateIndexOperation ? ((CreateIndexOperation)op).index() : null;
+                        if (oldDesc != null) {
+                            if (!op0.ifNotExists())
+                                errMsg = "Index already exists: " + idxName;
+                        }
+                        else {
+                            idxTypMap.put(idxName, desc);
 
-                        QueryUtils.processDynamicIndexChange(idxName, idx, desc);
+                            QueryUtils.processDynamicIndexChange(idxName, op0.index(), desc);
+                        }
                     }
+                }
+                else {
+                    // Handle drop.
+                    DropIndexOperation op0 = (DropIndexOperation)op;
+
+                    QueryTypeDescriptorImpl desc = idxTypMap.get(op0.indexName());
 
-                    ops.add(op);
+                    if (desc == null) {
+                        if (!op0.ifExists())
+                            errMsg = "Index doesn't exist: " + idxName;
+                    }
+                    else {
+                        idxTypMap.remove(idxName);
+
+                        QueryUtils.processDynamicIndexChange(idxName, null, desc);
+                    }
                 }
+
+                activeOps.put(op, errMsg);
             }
         }
 
@@ -359,8 +444,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         registerCache0(space, cctx, cands);
 
         // If cache was registered successfully, start pending operations.
-        for (AbstractIndexOperation op : ops)
-            startIndexOperation(op, true, null);
+        for (Map.Entry<AbstractIndexOperation, String> activeOp : activeOps.entrySet()) {
+            String errMsg = activeOp.getValue();
+
+            Exception err = errMsg != null ? new IgniteException(errMsg) : null;
+
+            startIndexOperation(activeOp.getKey(), true, err);
+        }
 
         // Warn about possible implicit deserialization.
         if (!mustDeserializeClss.isEmpty()) {
@@ -501,13 +591,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     if (!op0.ifExists())
                         errMsg = "Index doesn't exist: " + idxName;
                 }
-                else {
-                    // Make sure that index relate to expected table.
-                    if (F.eq(oldIdx.typeDescriptor().tableName(), op.tableName())) {
-                        completed = true;
-                        errMsg = "Index doesn't exist: " + idxName;
-                    }
-                }
             }
             else {
                 completed = true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/815f0cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexState.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexState.java
index 71fd3d8..83ded3a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexState.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexState.java
@@ -65,7 +65,7 @@ public class QueryIndexState implements Serializable {
     /**
      * @return Table name.
      */
-    public String tableName() {
+    @Nullable public String tableName() {
         return tblName;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/815f0cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexStates.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexStates.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexStates.java
index a222203..cb1724d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexStates.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexStates.java
@@ -127,7 +127,6 @@ public class QueryIndexStates implements Serializable {
         synchronized (mux) {
             AbstractIndexOperation op = msg.operation();
 
-            String tblName = op.tableName();
             String idxName = op.indexName();
 
             QueryIndexActiveOperation curOp = activeOps.remove(idxName);
@@ -137,12 +136,15 @@ public class QueryIndexStates implements Serializable {
                     if (!msg.hasError()) {
                         QueryIndexState state;
 
-                        if (op instanceof CreateIndexOperation)
-                            state = new QueryIndexState(tblName, idxName, ((CreateIndexOperation) op).index());
+                        if (op instanceof CreateIndexOperation) {
+                            CreateIndexOperation op0 = (CreateIndexOperation)op;
+
+                            state = new QueryIndexState(op0.tableName(), idxName, ((CreateIndexOperation)op).index());
+                        }
                         else {
                             assert op instanceof DropIndexOperation;
 
-                            state = new QueryIndexState(tblName, idxName, null);
+                            state = new QueryIndexState(null, idxName, null);
                         }
 
                         readyOps.put(idxName, state);

http://git-wip-us.apache.org/repos/asf/ignite/blob/815f0cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/AbstractIndexOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/AbstractIndexOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/AbstractIndexOperation.java
index 6f5e35e..11e7966 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/AbstractIndexOperation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/AbstractIndexOperation.java
@@ -36,22 +36,17 @@ public abstract class AbstractIndexOperation implements Serializable {
     /** Space. */
     private final String space;
 
-    /** Table name. */
-    private final String tblName;
-
     /**
      * Constructor.
      *
      * @param cliNodeId Client node ID.
      * @param opId Operation ID.
      * @param space Space.
-     * @param tblName Table name.
      */
-    public AbstractIndexOperation(UUID cliNodeId, UUID opId, String space, String tblName) {
+    public AbstractIndexOperation(UUID cliNodeId, UUID opId, String space) {
         this.cliNodeId = cliNodeId;
         this.opId = opId;
         this.space = space;
-        this.tblName = tblName;
     }
 
     /**
@@ -76,13 +71,6 @@ public abstract class AbstractIndexOperation implements Serializable {
     }
 
     /**
-     * @return Table name.
-     */
-    public String tableName() {
-        return tblName;
-    }
-
-    /**
      * @return Index name.
      */
     public abstract String indexName();

http://git-wip-us.apache.org/repos/asf/ignite/blob/815f0cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/CreateIndexOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/CreateIndexOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/CreateIndexOperation.java
index efbbb25..56f76e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/CreateIndexOperation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/CreateIndexOperation.java
@@ -30,6 +30,9 @@ public class CreateIndexOperation extends AbstractIndexOperation {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Table name. */
+    private final String tblName;
+
     /** Index. */
     @GridToStringInclude
     private final QueryIndex idx;
@@ -49,8 +52,9 @@ public class CreateIndexOperation extends AbstractIndexOperation {
      */
     public CreateIndexOperation(UUID cliNodeId, UUID opId, String space, String tblName, QueryIndex idx,
         boolean ifNotExists) {
-        super(cliNodeId, opId, space, tblName);
+        super(cliNodeId, opId, space);
 
+        this.tblName = tblName;
         this.idx = idx;
         this.ifNotExists = ifNotExists;
     }
@@ -61,6 +65,13 @@ public class CreateIndexOperation extends AbstractIndexOperation {
     }
 
     /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
      * @return Index params.
      */
     public QueryIndex index() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/815f0cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DropIndexOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DropIndexOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DropIndexOperation.java
index a8d646d..89db235 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DropIndexOperation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DropIndexOperation.java
@@ -40,12 +40,11 @@ public class DropIndexOperation extends AbstractIndexOperation {
      * @param cliNodeId Client node ID.
      * @param opId Operation id.
      * @param space Space.
-     * @param tblName Table name.
      * @param idxName Index name.
      * @param ifExists Ignore operation if index doesn't exist.
      */
-    DropIndexOperation(UUID cliNodeId, UUID opId, String space, String tblName, String idxName, boolean ifExists) {
-        super(cliNodeId, opId, space, tblName);
+    public DropIndexOperation(UUID cliNodeId, UUID opId, String space, String idxName, boolean ifExists) {
+        super(cliNodeId, opId, space);
 
         this.idxName = idxName;
         this.ifExists = ifExists;

http://git-wip-us.apache.org/repos/asf/ignite/blob/815f0cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexOperationHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexOperationHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexOperationHandler.java
index 7825877..f3c5ea3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexOperationHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexOperationHandler.java
@@ -140,7 +140,7 @@ public class IndexOperationHandler {
      * @return Worker name.
      */
     private String workerName() {
-        return "index-op-worker" + op.space() + "-" + op.tableName() + "-" + op.indexName();
+        return "index-op-worker" + op.space() + "-" + op.indexName();
     }
 
     /**


[18/23] ignite git commit: Merge branch 'master' into ignite-4565-ddl

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


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

Branch: refs/heads/ignite-4565-ddl
Commit: 013faff85bc256758b80f917711abfac8900a30c
Parents: 41a7bea 827befb
Author: devozerov <vo...@gridgain.com>
Authored: Wed Mar 22 11:02:25 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Mar 22 11:02:25 2017 +0300

----------------------------------------------------------------------
 .../affinity/fair/FairAffinityFunction.java     |  33 +-
 .../affinity/fair/FairAffinityNodesRestart.java | 130 +++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../include/ignite/binary/binary_raw_reader.h   |  19 +
 .../include/ignite/binary/binary_raw_writer.h   |  15 +
 .../include/ignite/binary/binary_reader.h       |  21 +
 .../include/ignite/binary/binary_writer.h       |  17 +
 .../include/ignite/impl/binary/binary_common.h  |   6 +
 .../ignite/impl/binary/binary_reader_impl.h     | 128 ++-
 .../include/ignite/impl/binary/binary_utils.h   |  25 +-
 .../ignite/impl/binary/binary_writer_impl.h     |  46 +-
 .../cpp/binary/src/binary/binary_raw_reader.cpp |  10 +
 .../cpp/binary/src/binary/binary_raw_writer.cpp |  10 +
 .../cpp/binary/src/binary/binary_reader.cpp     |  12 +-
 .../cpp/binary/src/binary/binary_writer.cpp     |  10 +
 .../src/impl/binary/binary_reader_impl.cpp      | 177 +++-
 .../cpp/binary/src/impl/binary/binary_utils.cpp |  12 +
 .../src/impl/binary/binary_writer_impl.cpp      | 105 ++-
 modules/platforms/cpp/common/Makefile.am        |   3 +-
 .../platforms/cpp/common/include/Makefile.am    |   1 +
 .../cpp/common/include/ignite/common/utils.h    |  53 +-
 .../platforms/cpp/common/include/ignite/time.h  | 138 +++
 .../cpp/common/project/vs/common.vcxproj        |   2 +
 .../common/project/vs/common.vcxproj.filters    |   6 +
 .../platforms/cpp/common/src/common/utils.cpp   |  30 +
 modules/platforms/cpp/common/src/time.cpp       |  83 ++
 modules/platforms/cpp/core-test/Makefile.am     |   1 +
 .../cpp/core-test/config/cache-query.xml        |  23 +
 .../include/ignite/binary_test_utils.h          |  53 ++
 .../cpp/core-test/include/ignite/test_type.h    |  11 +-
 .../cpp/core-test/project/vs/core-test.vcxproj  |   1 +
 .../project/vs/core-test.vcxproj.filters        |   3 +
 .../cpp/core-test/src/binary_object_test.cpp    | 110 +--
 .../src/binary_reader_writer_raw_test.cpp       | 712 +++------------
 .../core-test/src/binary_reader_writer_test.cpp | 913 +++----------------
 .../cpp/core-test/src/binary_session_test.cpp   |  18 +
 .../cpp/core-test/src/cache_query_test.cpp      |  65 +-
 .../platforms/cpp/core-test/src/cache_test.cpp  |  12 +
 .../cpp/core-test/src/date_time_test.cpp        | 265 ++++++
 .../IgniteConfigurationSerializerTest.cs        |  44 +-
 .../Common/IgniteConfigurationXmlSerializer.cs  | 250 +++--
 modules/web-console/frontend/app/app.js         |   1 +
 .../form-field-datepicker.pug                   |  12 +-
 .../list-of-registered-users/index.js           |   2 +
 .../list-of-registered-users.column-defs.js     |  48 +-
 .../list-of-registered-users.controller.js      | 193 +++-
 .../list-of-registered-users.scss               |  28 +
 .../list-of-registered-users.tpl.pug            |  50 +-
 .../ui-grid-header/ui-grid-header.scss          |   6 +
 .../ui-grid-header/ui-grid-header.tpl.pug       |   4 +-
 .../ui-grid-settings/ui-grid-settings.scss      |  10 +
 .../frontend/app/primitives/badge/index.scss    |  36 +
 .../frontend/app/primitives/index.js            |  19 +
 .../frontend/app/primitives/tabs/index.scss     |  73 ++
 .../frontend/public/stylesheets/style.scss      |   2 +
 .../config/benchmark-atomic.properties          |  18 +-
 .../config/benchmark-bin-identity.properties    |  10 +-
 .../config/benchmark-cache-load.properties      |  10 +-
 .../config/benchmark-client-mode.properties     |  10 +-
 .../config/benchmark-compute.properties         |  18 +-
 .../config/benchmark-failover.properties        |  18 +-
 .../yardstick/config/benchmark-full.properties  |  18 +-
 .../config/benchmark-multicast.properties       |  10 +-
 .../config/benchmark-put-indexed-val.properties |  18 +-
 .../benchmark-query-put-separated.properties    |  10 +-
 .../yardstick/config/benchmark-query.properties |  18 +-
 .../config/benchmark-remote-sample.properties   |  18 +-
 .../config/benchmark-remote.properties          |  18 +-
 .../config/benchmark-sample.properties          |  18 +-
 .../config/benchmark-sql-dml.properties         |  18 +-
 .../yardstick/config/benchmark-store.properties |  10 +-
 .../yardstick/config/benchmark-tx.properties    |  18 +-
 modules/yardstick/config/benchmark.properties   |  18 +-
 .../test-max-int-values-offheap.properties      |  23 +-
 .../test-max-int-values-onheap.properties       |  23 +-
 .../config/test-max-int-values-swap.properties  |  23 +-
 76 files changed, 2367 insertions(+), 2037 deletions(-)
----------------------------------------------------------------------



[13/23] ignite git commit: IGNITE-4690: CPP: Added ignite::Time type

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/src/binary_reader_writer_raw_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/binary_reader_writer_raw_test.cpp b/modules/platforms/cpp/core-test/src/binary_reader_writer_raw_test.cpp
index f2b3d33..9e610e6 100644
--- a/modules/platforms/cpp/core-test/src/binary_reader_writer_raw_test.cpp
+++ b/modules/platforms/cpp/core-test/src/binary_reader_writer_raw_test.cpp
@@ -171,216 +171,54 @@ void CheckRawPrimitiveArray(T dflt, T val1, T val2)
 
 void CheckRawWritesRestricted(BinaryRawWriter& writer)
 {
-    try
-    {
-        writer.WriteInt8(1);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        int8_t arr[1];
-
-        writer.WriteInt8Array(arr, 1);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        Guid val(1, 1);
-
-        writer.WriteGuid(val);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        Date val(1);
-
-        writer.WriteDate(val);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        Timestamp val(1);
-
-        writer.WriteTimestamp(val);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        writer.WriteString("test");
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try 
-    {
-        writer.WriteArray<int8_t>();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try 
-    {
-        writer.WriteCollection<int8_t>();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try 
-    {
-        writer.WriteMap<int8_t, int8_t>();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(writer.WriteInt8(1), IgniteError, IsBinaryError);
+
+    int8_t arr[1];
+    BOOST_CHECK_EXCEPTION(writer.WriteInt8Array(arr, 1), IgniteError, IsBinaryError);
+
+    Guid guid(1, 1);
+    BOOST_CHECK_EXCEPTION(writer.WriteGuid(guid), IgniteError, IsBinaryError);
+
+    Date date(1);
+    BOOST_CHECK_EXCEPTION(writer.WriteDate(date), IgniteError, IsBinaryError);
+
+    Time time(1);
+    BOOST_CHECK_EXCEPTION(writer.WriteTime(time), IgniteError, IsBinaryError);
+
+    Timestamp ts(1);
+    BOOST_CHECK_EXCEPTION(writer.WriteTimestamp(ts), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION(writer.WriteString("test"), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION(writer.WriteArray<int8_t>(), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION(writer.WriteCollection<int8_t>(), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION((writer.WriteMap<int8_t, int8_t>()), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION(writer.WriteInt8(1), IgniteError, IsBinaryError);
 }
 
 void CheckRawReadsRestricted(BinaryRawReader& reader)
 {
-    try
-    {
-        reader.ReadInt8();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        int8_t arr[1];
-
-        reader.ReadInt8Array(arr, 1);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadGuid();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadDate();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadTimestamp();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadString();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadArray<int8_t>();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadCollection<int8_t>();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadMap<int8_t, int8_t>();
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(reader.ReadInt8(), IgniteError, IsBinaryError);
+
+    int8_t arr[1];
+    BOOST_CHECK_EXCEPTION(reader.ReadInt8Array(arr, 1), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION(reader.ReadGuid(), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION(reader.ReadDate(), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION(reader.ReadTimestamp(), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION(reader.ReadString(), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION(reader.ReadArray<int8_t>(), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION(reader.ReadCollection<int8_t>(), IgniteError, IsBinaryError);
+
+    BOOST_CHECK_EXCEPTION((reader.ReadMap<int8_t, int8_t>()), IgniteError, IsBinaryError);
 }
 
 void CheckRawCollectionEmpty(CollectionType* colType)
@@ -400,27 +238,9 @@ void CheckRawCollectionEmpty(CollectionType* colType)
 
     rawWriter.WriteInt8(1);
 
-    try
-    {
-        colWriter.Write(1);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colWriter.Write(1), IgniteError, IsBinaryError);
 
-    try
-    {
-        colWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colWriter.Close(), IgniteError, IsBinaryError);
 
     out.Synchronize();
 
@@ -439,16 +259,7 @@ void CheckRawCollectionEmpty(CollectionType* colType)
     BOOST_REQUIRE(!colReader.HasNext());
     BOOST_REQUIRE(!colReader.IsNull());
 
-    try
-    {
-        colReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -478,27 +289,9 @@ void CheckRawCollection(CollectionType* colType)
 
     rawWriter.WriteInt8(1);
 
-    try
-    {
-        colWriter.Write(1);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colWriter.Write(1), IgniteError, IsBinaryError);
 
-    try
-    {
-        colWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colWriter.Close(), IgniteError, IsBinaryError);
 
     out.Synchronize();
 
@@ -529,16 +322,7 @@ void CheckRawCollection(CollectionType* colType)
 
     BOOST_REQUIRE(!colReader.HasNext());
 
-    try
-    {
-        colReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -609,27 +393,9 @@ void CheckRawMapEmpty(MapType* mapType)
 
     rawWriter.WriteInt8(1);
 
-    try
-    {
-        mapWriter.Write(1, BinaryInner(1));
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(mapWriter.Write(1, BinaryInner(1)), IgniteError, IsBinaryError);
 
-    try
-    {
-        mapWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(mapWriter.Close(), IgniteError, IsBinaryError);
 
     out.Synchronize();
 
@@ -648,19 +414,9 @@ void CheckRawMapEmpty(MapType* mapType)
     BOOST_REQUIRE(!mapReader.HasNext());
     BOOST_REQUIRE(!mapReader.IsNull());
 
-    try
-    {
-        int8_t key;
-        BinaryInner val;
-
-        mapReader.GetNext(&key, &val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    int8_t key;
+    BinaryInner val;
+    BOOST_CHECK_EXCEPTION(mapReader.GetNext(&key, &val), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -690,27 +446,9 @@ void CheckRawMap(MapType* mapType)
 
     rawWriter.WriteInt8(1);
 
-    try
-    {
-        mapWriter.Write(4, BinaryInner(4));
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        mapWriter.Close();
+    BOOST_CHECK_EXCEPTION(mapWriter.Write(4, BinaryInner(4)), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(mapWriter.Close(), IgniteError, IsBinaryError);
 
     out.Synchronize();
 
@@ -749,16 +487,7 @@ void CheckRawMap(MapType* mapType)
 
     BOOST_REQUIRE(!mapReader.HasNext());
 
-    try
-    {
-        mapReader.GetNext(&key, &val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(mapReader.GetNext(&key, &val), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -819,6 +548,13 @@ BOOST_AUTO_TEST_CASE(TestPrimitiveDate)
     CheckRawPrimitive<Date>(val);
 }
 
+BOOST_AUTO_TEST_CASE(TestPrimitiveTime)
+{
+    Time val(time(NULL) * 1000);
+
+    CheckRawPrimitive<Time>(val);
+}
+
 BOOST_AUTO_TEST_CASE(TestPrimitiveTimestamp)
 {
     Timestamp val(time(NULL), 0);
@@ -884,6 +620,15 @@ BOOST_AUTO_TEST_CASE(TestPrimitiveArrayDate)
     CheckRawPrimitiveArray<Date>(dflt, val1, val2);
 }
 
+BOOST_AUTO_TEST_CASE(TestPrimitiveArrayTime)
+{
+    Time dflt(1);
+    Time val1(2);
+    Time val2(3);
+
+    CheckRawPrimitiveArray<Time>(dflt, val1, val2);
+}
+
 BOOST_AUTO_TEST_CASE(TestPrimitiveArrayTimestamp)
 {
     Timestamp dflt(1);
@@ -937,6 +682,28 @@ BOOST_AUTO_TEST_CASE(TestDateNull)
     BOOST_REQUIRE(actualVal == expVal);
 }
 
+BOOST_AUTO_TEST_CASE(TestTimeNull)
+{
+    InteropUnpooledMemory mem(1024);
+
+    InteropOutputStream out(&mem);
+    BinaryWriterImpl writer(&out, NULL);
+    BinaryRawWriter rawWriter(&writer);
+
+    rawWriter.WriteNull();
+
+    out.Synchronize();
+
+    InteropInputStream in(&mem);
+    BinaryReaderImpl reader(&in);
+    BinaryRawReader rawReader(&reader);
+
+    Time expVal;
+    Time actualVal = rawReader.ReadTime();
+
+    BOOST_REQUIRE(actualVal == expVal);
+}
+
 BOOST_AUTO_TEST_CASE(TestTimestampNull)
 {
     InteropUnpooledMemory mem(1024);
@@ -1030,29 +797,10 @@ BOOST_AUTO_TEST_CASE(TestStringArrayNull)
     BOOST_REQUIRE(!arrReader.HasNext());
     BOOST_REQUIRE(arrReader.IsNull());
 
-    try
-    {
-        char res[100];
-
-        arrReader.GetNext(res, 100);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrReader.GetNext();
+    char res[100];
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(res, 100), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -1073,55 +821,14 @@ BOOST_AUTO_TEST_CASE(TestStringArrayEmpty)
 
     rawWriter.WriteInt8(1);
 
-    try
-    {
-        const char* val = "test";
+    const char* val1 = "test";
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val1, 4), IgniteError, IsBinaryError);
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val1), IgniteError, IsBinaryError);
 
-        arrWriter.Write(val, 4);
+    std::string val2 = "test";
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val2), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        const char* val = "test";
-
-        arrWriter.Write(val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        std::string val = "test";
-
-        arrWriter.Write(val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrWriter.Close(), IgniteError, IsBinaryError);
 
     out.Synchronize();
 
@@ -1135,29 +842,9 @@ BOOST_AUTO_TEST_CASE(TestStringArrayEmpty)
     BOOST_REQUIRE(!arrReader.HasNext());
     BOOST_REQUIRE(!arrReader.IsNull());
 
-    try
-    {
-        char res[100];
-
-        arrReader.GetNext(res, 100);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    char res[100];
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(res, 100), IgniteError, IsBinaryError);
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -1188,55 +875,14 @@ BOOST_AUTO_TEST_CASE(TestStringArray)
 
     rawWriter.WriteInt8(1);
 
-    try
-    {
-        const char* val = "test";
+    const char* val1 = "test";
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val1, 4), IgniteError, IsBinaryError);
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val1), IgniteError, IsBinaryError);
 
-        arrWriter.Write(val, 4);
+    std::string val2 = "test";
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val2), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        const char* val = "test";
-
-        arrWriter.Write(val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        std::string val = "test";
-
-        arrWriter.Write(val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrWriter.Close(), IgniteError, IsBinaryError);
 
     out.Synchronize();
 
@@ -1298,29 +944,9 @@ BOOST_AUTO_TEST_CASE(TestStringArray)
 
     BOOST_REQUIRE(!arrReader.HasNext());
 
-    try
-    {
-        char res[100];
-
-        arrReader.GetNext(res, 100);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    char res[100];
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(res, 100), IgniteError, IsBinaryError);
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -1413,16 +1039,7 @@ BOOST_AUTO_TEST_CASE(TestArrayNull)
     BOOST_REQUIRE(!arrReader.HasNext());
     BOOST_REQUIRE(arrReader.IsNull());
 
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -1443,27 +1060,9 @@ BOOST_AUTO_TEST_CASE(TestArrayEmpty)
 
     rawWriter.WriteInt8(1);
 
-    try
-    {
-        arrWriter.Write(1);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrWriter.Close();
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(1), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrWriter.Close(), IgniteError, IsBinaryError);
 
     out.Synchronize();
 
@@ -1477,16 +1076,7 @@ BOOST_AUTO_TEST_CASE(TestArrayEmpty)
     BOOST_REQUIRE(!arrReader.HasNext());
     BOOST_REQUIRE(!arrReader.IsNull());
 
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -1515,27 +1105,9 @@ BOOST_AUTO_TEST_CASE(TestArray)
 
     rawWriter.WriteInt8(1);
 
-    try
-    {
-        arrWriter.Write(1);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrWriter.Close();
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(1), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrWriter.Close(), IgniteError, IsBinaryError);
 
     out.Synchronize();
 
@@ -1561,16 +1133,7 @@ BOOST_AUTO_TEST_CASE(TestArray)
 
     BOOST_REQUIRE(!arrReader.HasNext());
 
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -1599,16 +1162,7 @@ BOOST_AUTO_TEST_CASE(TestCollectionNull)
     BOOST_REQUIRE(!colReader.HasNext());
     BOOST_REQUIRE(colReader.IsNull()); 
 
-    try
-    {
-        colReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }
@@ -1673,19 +1227,9 @@ BOOST_AUTO_TEST_CASE(TestMapNull)
     BOOST_REQUIRE(!mapReader.HasNext());
     BOOST_REQUIRE(mapReader.IsNull());
 
-    try
-    {
-        int8_t key;
-        BinaryInner val;
-
-        mapReader.GetNext(&key, &val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    int8_t key;
+    BinaryInner val;
+    BOOST_CHECK_EXCEPTION(mapReader.GetNext(&key, &val), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp b/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
index dd5743a..3115764 100644
--- a/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
+++ b/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
@@ -46,16 +46,7 @@ void CheckPrimitive(T val)
     BinaryWriterImpl writerImpl(&out, &idRslvr, NULL, NULL, 0);
     BinaryWriter writer(&writerImpl);
 
-    try
-    {
-        Write<T>(writer, NULL, val);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(Write<T>(writer, NULL, val), IgniteError, IsBinaryError);
 
     Write<T>(writer, "test", val);
 
@@ -73,27 +64,11 @@ void CheckPrimitive(T val)
     BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
-    try
-    {
-        Read<T>(reader, NULL);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(Read<T>(reader, NULL), IgniteError, IsBinaryError);
 
-    try
-    {
-        T readVal = Read<T>(reader, "test"); 
+    T readVal = Read<T>(reader, "test"); 
 
-        BOOST_REQUIRE(readVal == val);
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_FAIL(err.GetText());
-    }
+    BOOST_REQUIRE(readVal == val);
 }
 
 template<typename T>
@@ -111,21 +86,12 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
 
     out.Position(IGNITE_DFLT_HDR_LEN);
 
-    try
-    {
-        T nullFieldArr[2];
+    T nullFieldArr[2];
 
-        nullFieldArr[0] = val1;
-        nullFieldArr[1] = val2;
+    nullFieldArr[0] = val1;
+    nullFieldArr[1] = val2;
 
-        WriteArray<T>(writer, NULL, nullFieldArr, 2);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(WriteArray<T>(writer, NULL, nullFieldArr, 2), IgniteError, IsBinaryError);
 
     T arr1[2];
     arr1[0] = dflt;
@@ -269,16 +235,7 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
         in.Position(IGNITE_DFLT_HDR_LEN);
         BOOST_REQUIRE(ReadArray<T>(reader, fieldName, NULL, 2) == 2);
 
-        try
-        {
-            ReadArray<T>(reader, NULL, arr1, 2);
-
-            BOOST_FAIL("Not restricted.");
-        }
-        catch (IgniteError& err)
-        {
-            BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-        }
+        BOOST_CHECK_EXCEPTION(ReadArray<T>(reader, NULL, arr1, 2), IgniteError, IsBinaryError);
 
         BOOST_REQUIRE(ReadArray<T>(reader, fieldName, arr1, 0) == 2);
         BOOST_REQUIRE(arr1[0] == dflt);
@@ -296,216 +253,52 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
 
 void CheckWritesRestricted(BinaryWriter& writer)
 {
-    try
-    {
-        writer.WriteInt8("field", 1);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        int8_t arr[1];
+    BOOST_CHECK_EXCEPTION(writer.WriteInt8("field", 1), IgniteError, IsBinaryError);
 
-        writer.WriteInt8Array("field", arr, 1);
+    int8_t arr[1];
+    BOOST_CHECK_EXCEPTION(writer.WriteInt8Array("field", arr, 1), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        Guid val(1, 1);
-
-        writer.WriteGuid("field", val);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        Date val(1);
-
-        writer.WriteDate("field", val);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        Timestamp val(1);
-
-        writer.WriteTimestamp("field", val);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        writer.WriteString("field", "test");
+    Guid guid(1, 1);
+    BOOST_CHECK_EXCEPTION(writer.WriteGuid("field", guid), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    Date date(1);
+    BOOST_CHECK_EXCEPTION(writer.WriteDate("field", date), IgniteError, IsBinaryError);
 
-    try 
-    {
-        writer.WriteArray<int8_t>("field");
+    Time time(1);
+    BOOST_CHECK_EXCEPTION(writer.WriteTime("field", time), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    Timestamp ts(1);
+    BOOST_CHECK_EXCEPTION(writer.WriteTimestamp("field", ts), IgniteError, IsBinaryError);
 
-    try 
-    {
-        writer.WriteCollection<int8_t>("field");
+    BOOST_CHECK_EXCEPTION(writer.WriteString("field", "test"), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(writer.WriteArray<int8_t>("field"), IgniteError, IsBinaryError);
 
-    try 
-    {
-        writer.WriteMap<int8_t, int8_t>("field");
+    BOOST_CHECK_EXCEPTION(writer.WriteCollection<int8_t>("field"), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION((writer.WriteMap<int8_t, int8_t>("field")), IgniteError, IsBinaryError);
 }
 
 void CheckReadsRestricted(BinaryReader& reader)
 {
-    try
-    {
-        reader.ReadInt8("field");
+    BOOST_CHECK_EXCEPTION(reader.ReadInt8("field"), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    int8_t arr[1];
+    BOOST_CHECK_EXCEPTION(reader.ReadInt8Array("field", arr, 1), IgniteError, IsBinaryError);
 
-    try
-    {
-        int8_t arr[1];
+    BOOST_CHECK_EXCEPTION(reader.ReadGuid("field"), IgniteError, IsBinaryError);
 
-        reader.ReadInt8Array("field", arr, 1);
+    BOOST_CHECK_EXCEPTION(reader.ReadDate("field"), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(reader.ReadTimestamp("field"), IgniteError, IsBinaryError);
 
-    try
-    {
-        reader.ReadGuid("field");
+    BOOST_CHECK_EXCEPTION(reader.ReadString("field"), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(reader.ReadArray<int8_t>("field"), IgniteError, IsBinaryError);
 
-    try
-    {
-        reader.ReadDate("field");
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadTimestamp("field");
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadString("field");
+    BOOST_CHECK_EXCEPTION(reader.ReadCollection<int8_t>("field"), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadArray<int8_t>("field");
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadCollection<int8_t>("field");
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadMap<int8_t, int8_t>("field");
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION((reader.ReadMap<int8_t, int8_t>("field")), IgniteError, IsBinaryError);
 }
 
 void CheckCollectionEmpty(CollectionType* colType)
@@ -529,27 +322,9 @@ void CheckCollectionEmpty(CollectionType* colType)
 
     writer.WriteInt8("field2", 1);
 
-    try
-    {
-        colWriter.Write(1);
+    BOOST_CHECK_EXCEPTION(colWriter.Write(1), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        colWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colWriter.Close(), IgniteError, IsBinaryError);
 
     writerImpl.PostWrite();
 
@@ -575,17 +350,8 @@ void CheckCollectionEmpty(CollectionType* colType)
     BOOST_REQUIRE(colReader.GetSize() == 0);
     BOOST_REQUIRE(!colReader.HasNext());
     BOOST_REQUIRE(!colReader.IsNull());
-
-    try
-    {
-        colReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    
+    BOOST_CHECK_EXCEPTION(colReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -619,27 +385,9 @@ void CheckCollection(CollectionType* colType)
 
     writer.WriteInt8("field2", 1);
 
-    try
-    {
-        colWriter.Write(1);
+    BOOST_CHECK_EXCEPTION(colWriter.Write(1), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        colWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colWriter.Close(), IgniteError, IsBinaryError);
 
     writerImpl.PostWrite();
 
@@ -678,16 +426,7 @@ void CheckCollection(CollectionType* colType)
 
     BOOST_REQUIRE(!colReader.HasNext());
 
-    try
-    {
-        colReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -772,27 +511,9 @@ void CheckMapEmpty(MapType* mapType)
 
     writer.WriteInt8("field2", 1);
 
-    try
-    {
-        mapWriter.Write(1, BinaryInner(1));
+    BOOST_CHECK_EXCEPTION(mapWriter.Write(1, BinaryInner(1)), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        mapWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(mapWriter.Close(), IgniteError, IsBinaryError);
 
     writerImpl.PostWrite();
 
@@ -819,19 +540,9 @@ void CheckMapEmpty(MapType* mapType)
     BOOST_REQUIRE(!mapReader.HasNext());
     BOOST_REQUIRE(!mapReader.IsNull());
 
-    try
-    {
-        int8_t key;
-        BinaryInner val;
-
-        mapReader.GetNext(&key, &val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    int8_t key;
+    BinaryInner val;
+    BOOST_CHECK_EXCEPTION(mapReader.GetNext(&key, &val), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -865,27 +576,9 @@ void CheckMap(MapType* mapType)
 
     writer.WriteInt8("field2", 1);
 
-    try
-    {
-        mapWriter.Write(4, BinaryInner(4));
+    BOOST_CHECK_EXCEPTION(mapWriter.Write(4, BinaryInner(4)), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        mapWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(mapWriter.Close(), IgniteError, IsBinaryError);
 
     writerImpl.PostWrite();
 
@@ -932,16 +625,7 @@ void CheckMap(MapType* mapType)
 
     BOOST_REQUIRE(!mapReader.HasNext());
 
-    try
-    {
-        mapReader.GetNext(&key, &val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(mapReader.GetNext(&key, &val), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -1002,6 +686,13 @@ BOOST_AUTO_TEST_CASE(TestPrimitiveDate)
     CheckPrimitive<Date>(val);
 }
 
+BOOST_AUTO_TEST_CASE(TestPrimitiveTime)
+{
+    Time val(time(NULL) * 1000);
+
+    CheckPrimitive<Time>(val);
+}
+
 BOOST_AUTO_TEST_CASE(TestPrimitiveTimestamp)
 {
     Timestamp val(time(NULL), 0);
@@ -1067,6 +758,15 @@ BOOST_AUTO_TEST_CASE(TestPrimitiveArrayDate)
     CheckPrimitiveArray<Date>(dflt, val1, val2);
 }
 
+BOOST_AUTO_TEST_CASE(TestPrimitiveArrayTime)
+{
+    Time dflt(1);
+    Time val1(2);
+    Time val2(3);
+
+    CheckPrimitiveArray<Time>(dflt, val1, val2);
+}
+
 BOOST_AUTO_TEST_CASE(TestPrimitiveArrayTimestamp)
 {
     Timestamp dflt(1);
@@ -1088,16 +788,7 @@ BOOST_AUTO_TEST_CASE(TestGuidNull)
 
     out.Position(IGNITE_DFLT_HDR_LEN);
 
-    try
-    {
-        writer.WriteNull(NULL);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(writer.WriteNull(NULL), IgniteError, IsBinaryError);
 
     writer.WriteNull("test");
 
@@ -1115,16 +806,7 @@ BOOST_AUTO_TEST_CASE(TestGuidNull)
     
     in.Position(IGNITE_DFLT_HDR_LEN);
 
-    try
-    {
-        reader.ReadGuid(NULL);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(reader.ReadGuid(NULL), IgniteError, IsBinaryError);
 
     Guid expVal;
     Guid actualVal = reader.ReadGuid("test");
@@ -1144,16 +826,7 @@ BOOST_AUTO_TEST_CASE(TestDateNull)
 
     out.Position(IGNITE_DFLT_HDR_LEN);
 
-    try
-    {
-        writer.WriteNull(NULL);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(writer.WriteNull(NULL), IgniteError, IsBinaryError);
 
     writer.WriteNull("test");
 
@@ -1171,16 +844,7 @@ BOOST_AUTO_TEST_CASE(TestDateNull)
     
     in.Position(IGNITE_DFLT_HDR_LEN);
 
-    try
-    {
-        reader.ReadDate(NULL);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(reader.ReadDate(NULL), IgniteError, IsBinaryError);
 
     Date expVal;
     Date actualVal = reader.ReadDate("test");
@@ -1188,7 +852,7 @@ BOOST_AUTO_TEST_CASE(TestDateNull)
     BOOST_REQUIRE(actualVal == expVal);
 }
 
-BOOST_AUTO_TEST_CASE(TestTimestampNull)
+BOOST_AUTO_TEST_CASE(TestTimeNull)
 {
     TemplatedBinaryIdResolver<BinaryDummy> idRslvr;
 
@@ -1200,16 +864,7 @@ BOOST_AUTO_TEST_CASE(TestTimestampNull)
 
     out.Position(IGNITE_DFLT_HDR_LEN);
 
-    try
-    {
-        writer.WriteNull(NULL);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(writer.WriteNull(NULL), IgniteError, IsBinaryError);
 
     writer.WriteNull("test");
 
@@ -1227,16 +882,45 @@ BOOST_AUTO_TEST_CASE(TestTimestampNull)
 
     in.Position(IGNITE_DFLT_HDR_LEN);
 
-    try
-    {
-        reader.ReadTimestamp(NULL);
+    BOOST_CHECK_EXCEPTION(reader.ReadTime(NULL), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    Time expVal;
+    Time actualVal = reader.ReadTime("test");
+
+    BOOST_REQUIRE(actualVal == expVal);
+}
+
+BOOST_AUTO_TEST_CASE(TestTimestampNull)
+{
+    TemplatedBinaryIdResolver<BinaryDummy> idRslvr;
+
+    InteropUnpooledMemory mem(1024);
+
+    InteropOutputStream out(&mem);
+    BinaryWriterImpl writerImpl(&out, &idRslvr, NULL, NULL, 0);
+    BinaryWriter writer(&writerImpl);
+
+    out.Position(IGNITE_DFLT_HDR_LEN);
+
+    BOOST_CHECK_EXCEPTION(writer.WriteNull(NULL), IgniteError, IsBinaryError);
+
+    writer.WriteNull("test");
+
+    writerImpl.PostWrite();
+
+    out.Synchronize();
+
+    InteropInputStream in(&mem);
+
+    int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
+    int32_t footerEnd = footerBegin + 5;
+
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
+    BinaryReader reader(&readerImpl);
+
+    in.Position(IGNITE_DFLT_HDR_LEN);
+
+    BOOST_CHECK_EXCEPTION(reader.ReadTimestamp(NULL), IgniteError, IsBinaryError);
 
     Timestamp expVal;
     Timestamp actualVal = reader.ReadTimestamp("test");
@@ -1259,38 +943,11 @@ BOOST_AUTO_TEST_CASE(TestString) {
     const char* writeVal2 = "test";
     std::string writeVal3 = writeVal1;
 
-    try
-    {
-        writer.WriteString(NULL, writeVal1);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(writer.WriteString(NULL, writeVal1), IgniteError, IsBinaryError);
 
-    try
-    {
-        writer.WriteString(NULL, writeVal1, 4);
+    BOOST_CHECK_EXCEPTION(writer.WriteString(NULL, writeVal1, 4), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        writer.WriteString(NULL, writeVal3);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(writer.WriteString(NULL, writeVal3), IgniteError, IsBinaryError);
 
     writer.WriteString("field1", writeVal1);
     writer.WriteString("field2", writeVal1, 4);
@@ -1312,29 +969,10 @@ BOOST_AUTO_TEST_CASE(TestString) {
 
     in.Position(IGNITE_DFLT_HDR_LEN);
 
-    try
-    {
-        char nullCheckRes[9];
-
-        reader.ReadString(NULL, nullCheckRes, 9);
-
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        reader.ReadString(NULL);
+    char nullCheckRes[9];
+    BOOST_CHECK_EXCEPTION(reader.ReadString(NULL, nullCheckRes, 9), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Not restricted.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(reader.ReadString(NULL), IgniteError, IsBinaryError);
 
     char readVal1[9];
     char readVal2[5];
@@ -1396,29 +1034,9 @@ BOOST_AUTO_TEST_CASE(TestStringArrayNull)
     BOOST_REQUIRE(!arrReader.HasNext());
     BOOST_REQUIRE(arrReader.IsNull());
 
-    try
-    {
-        char res[100];
-
-        arrReader.GetNext(res, 100);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    char res[100];
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(res, 100), IgniteError, IsBinaryError);
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -1443,55 +1061,14 @@ BOOST_AUTO_TEST_CASE(TestStringArrayEmpty)
 
     writer.WriteInt8("field2", 1);
 
-    try
-    {
-        const char* val = "test";
+    const char* val1 = "test";
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val1, 4), IgniteError, IsBinaryError);
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val1), IgniteError, IsBinaryError);
 
-        arrWriter.Write(val, 4);
+    std::string val2 = "test";
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val2), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        const char* val = "test";
-
-        arrWriter.Write(val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        std::string val = "test";
-
-        arrWriter.Write(val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrWriter.Close(), IgniteError, IsBinaryError);
 
     writerImpl.PostWrite();
 
@@ -1513,29 +1090,10 @@ BOOST_AUTO_TEST_CASE(TestStringArrayEmpty)
     BOOST_REQUIRE(!arrReader.HasNext());
     BOOST_REQUIRE(!arrReader.IsNull());
 
-    try
-    {
-        char res[100];
-
-        arrReader.GetNext(res, 100);
+    char res[100];
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(res, 100), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -1570,55 +1128,14 @@ BOOST_AUTO_TEST_CASE(TestStringArray)
 
     writer.WriteInt8("field2", 1);
 
-    try
-    {
-        const char* val = "test";
-
-        arrWriter.Write(val, 4);
+    const char* val1 = "test";
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val1, 4), IgniteError, IsBinaryError);
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val1), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    std::string val2 = "test";
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(val2), IgniteError, IsBinaryError);
 
-    try
-    {
-        const char* val = "test";
-
-        arrWriter.Write(val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        std::string val = "test";
-
-        arrWriter.Write(val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrWriter.Close(), IgniteError, IsBinaryError);
 
     writerImpl.PostWrite();
 
@@ -1688,29 +1205,10 @@ BOOST_AUTO_TEST_CASE(TestStringArray)
 
     BOOST_REQUIRE(!arrReader.HasNext());
 
-    try
-    {
-        char res[100];
+    char res[100];
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(res, 100), IgniteError, IsBinaryError);
 
-        arrReader.GetNext(res, 100);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -1840,16 +1338,7 @@ BOOST_AUTO_TEST_CASE(TestArrayNull)
     BOOST_REQUIRE(!arrReader.HasNext());
     BOOST_REQUIRE(arrReader.IsNull());
 
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -1874,27 +1363,9 @@ BOOST_AUTO_TEST_CASE(TestArrayEmpty)
 
     writer.WriteInt8("field2", 1);
 
-    try
-    {
-        arrWriter.Write(1);
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(1), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrWriter.Close();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrWriter.Close(), IgniteError, IsBinaryError);
 
     writerImpl.PostWrite();
 
@@ -1916,16 +1387,7 @@ BOOST_AUTO_TEST_CASE(TestArrayEmpty)
     BOOST_REQUIRE(!arrReader.HasNext());
     BOOST_REQUIRE(!arrReader.IsNull());
 
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -1958,27 +1420,9 @@ BOOST_AUTO_TEST_CASE(TestArray)
 
     writer.WriteInt8("field2", 1);
 
-    try
-    {
-        arrWriter.Write(1);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
-
-    try
-    {
-        arrWriter.Close();
+    BOOST_CHECK_EXCEPTION(arrWriter.Write(1), IgniteError, IsBinaryError);
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrWriter.Close(), IgniteError, IsBinaryError);
 
     writerImpl.PostWrite();
 
@@ -2012,16 +1456,7 @@ BOOST_AUTO_TEST_CASE(TestArray)
 
     BOOST_REQUIRE(!arrReader.HasNext());
 
-    try
-    {
-        arrReader.GetNext();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(arrReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -2060,18 +1495,9 @@ BOOST_AUTO_TEST_CASE(TestCollectionNull)
     BOOST_REQUIRE(colReader.GetType() == IGNITE_COLLECTION_UNDEFINED);
     BOOST_REQUIRE(colReader.GetSize() == -1);
     BOOST_REQUIRE(!colReader.HasNext());
-    BOOST_REQUIRE(colReader.IsNull()); 
-
-    try
-    {
-        colReader.GetNext();
+    BOOST_REQUIRE(colReader.IsNull());
 
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(colReader.GetNext(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -2148,19 +1574,9 @@ BOOST_AUTO_TEST_CASE(TestMapNull)
     BOOST_REQUIRE(!mapReader.HasNext());
     BOOST_REQUIRE(mapReader.IsNull());
 
-    try
-    {
-        int8_t key;
-        BinaryInner val;
-
-        mapReader.GetNext(&key, &val);
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    int8_t key;
+    BinaryInner val;
+    BOOST_CHECK_EXCEPTION(mapReader.GetNext(&key, &val), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt8("field2") == 1);
 }
@@ -2203,16 +1619,7 @@ BOOST_AUTO_TEST_CASE(TestRawMode)
 
     BinaryRawWriter rawWriter = writer.RawWriter();
 
-    try
-    {
-        writer.RawWriter();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(writer.RawWriter(), IgniteError, IsBinaryError);
 
     rawWriter.WriteInt8(1);
 
@@ -2234,16 +1641,7 @@ BOOST_AUTO_TEST_CASE(TestRawMode)
 
     BinaryRawReader rawReader = reader.RawReader();
 
-    try
-    {
-        reader.RawReader();
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(reader.RawReader(), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(rawReader.ReadInt8() == 1);
 
@@ -2345,16 +1743,7 @@ BOOST_AUTO_TEST_CASE(TestFieldSeek)
     in.Position(IGNITE_DFLT_HDR_LEN);
     BOOST_REQUIRE(reader.ReadInt32("val1") == 1);
 
-    try
-    {
-        reader.ReadInt64("val2");
-
-        BOOST_FAIL("Error expected.");
-    }
-    catch (IgniteError& err)
-    {
-        BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_ERR_BINARY);
-    }
+    BOOST_CHECK_EXCEPTION(reader.ReadInt64("val2"), IgniteError, IsBinaryError);
 
     BOOST_REQUIRE(reader.ReadInt32("val2") == 2);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/src/binary_session_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/binary_session_test.cpp b/modules/platforms/cpp/core-test/src/binary_session_test.cpp
index 0b9b34d..08bc1eb 100644
--- a/modules/platforms/cpp/core-test/src/binary_session_test.cpp
+++ b/modules/platforms/cpp/core-test/src/binary_session_test.cpp
@@ -155,6 +155,24 @@ BOOST_AUTO_TEST_CASE(TestDate)
     BOOST_REQUIRE(readVal == writeVal);
 }
 
+BOOST_AUTO_TEST_CASE(TestTime)
+{
+    Time writeVal = Time(42);
+
+    InteropUnpooledMemory mem(1024);
+
+    InteropOutputStream out(&mem);
+    BinaryWriterImpl writeSes(&out, NULL);
+    writeSes.WriteTopObject<Time>(writeVal);
+    out.Synchronize();
+
+    InteropInputStream in(&mem);
+    BinaryReaderImpl reader(&in);
+    Time readVal = reader.ReadTopObject<Time>();
+
+    BOOST_REQUIRE(readVal == writeVal);
+}
+
 BOOST_AUTO_TEST_CASE(TestTimestamp)
 {
     Timestamp writeVal = Timestamp(77);

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/src/cache_query_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cache_query_test.cpp b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
index 28eb3fd..abb374a 100644
--- a/modules/platforms/cpp/core-test/src/cache_query_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
@@ -1130,30 +1130,23 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryExceptions)
 
     cursor = cache.Query(qry);
 
-    try
-    {
-        BOOST_REQUIRE(cursor.HasNext());
+    BOOST_REQUIRE(cursor.HasNext());
 
-        QueryFieldsRow row = cursor.GetNext();
+    QueryFieldsRow row = cursor.GetNext();
 
-        BOOST_REQUIRE(row.HasNext());
+    BOOST_REQUIRE(row.HasNext());
 
-        int age = row.GetNext<int>();
+    int age = row.GetNext<int>();
 
-        BOOST_REQUIRE(age == 10);
+    BOOST_REQUIRE(age == 10);
 
-        std::string name = row.GetNext<std::string>();
+    std::string name = row.GetNext<std::string>();
 
-        BOOST_REQUIRE(name == "A1");
+    BOOST_REQUIRE(name == "A1");
 
-        BOOST_REQUIRE(!row.HasNext());
+    BOOST_REQUIRE(!row.HasNext());
 
-        CheckEmpty(cursor);
-    }
-    catch (IgniteError& error)
-    {
-        BOOST_FAIL(error.GetText());
-    }
+    CheckEmpty(cursor);
 }
 
 /**
@@ -1639,6 +1632,46 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryTimestampEqual)
 }
 
 /**
+ * Test query for Time type.
+ */
+BOOST_AUTO_TEST_CASE(TestFieldsQueryTimeEqual)
+{
+    // Test simple query.
+    Cache<int, Time> cache = grid.GetOrCreateCache<int, Time>("TimeCache");
+
+    // Test query with field of type 'Timestamp'.
+    SqlFieldsQuery qry("select _key from Time where _val='04:11:02'");
+
+    QueryFieldsCursor cursor = cache.Query(qry);
+    CheckEmpty(cursor);
+
+    int32_t entryCnt = 1000; // Number of entries.
+
+    for (int i = 0; i < entryCnt; i++)
+    {
+        int secs = i % 60;
+        int mins = i / 60;
+        cache.Put(i, MakeTimeGmt(4, mins, secs));
+    }
+
+    cursor = cache.Query(qry);
+
+    BOOST_REQUIRE(cursor.HasNext());
+
+    QueryFieldsRow row = cursor.GetNext();
+
+    BOOST_REQUIRE(row.HasNext());
+
+    int key = row.GetNext<int>();
+
+    BOOST_CHECK(key == 662);
+
+    BOOST_REQUIRE(!row.HasNext());
+
+    CheckEmpty(cursor);
+}
+
+/**
  * Test fields query with several pages.
  */
 BOOST_AUTO_TEST_CASE(TestFieldsQueryPagesSeveral)

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/src/cache_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cache_test.cpp b/modules/platforms/cpp/core-test/src/cache_test.cpp
index f38eecc..437ed234 100644
--- a/modules/platforms/cpp/core-test/src/cache_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_test.cpp
@@ -452,6 +452,18 @@ BOOST_AUTO_TEST_CASE(TestPutGetDate)
     BOOST_REQUIRE(now == cache.Get(5));
 }
 
+BOOST_AUTO_TEST_CASE(TestPutGetTime)
+{
+    // Get existing cache
+    cache::Cache<int, Time> cache = grid0.GetOrCreateCache<int, Time>("partitioned");
+
+    Time now = Time(time(NULL) * 1000);
+
+    cache.Put(5, now);
+
+    BOOST_REQUIRE(now == cache.Get(5));
+}
+
 BOOST_AUTO_TEST_CASE(TestPutGetTimestamp)
 {
     // Get existing cache

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/src/date_time_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/date_time_test.cpp b/modules/platforms/cpp/core-test/src/date_time_test.cpp
new file mode 100644
index 0000000..0c93a17
--- /dev/null
+++ b/modules/platforms/cpp/core-test/src/date_time_test.cpp
@@ -0,0 +1,265 @@
+/*
+ * 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.
+ */
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include <ignite/time.h>
+#include <ignite/date.h>
+#include <ignite/timestamp.h>
+
+#include <ignite/common/utils.h>
+
+#include <ignite/test_utils.h>
+
+using namespace ignite;
+using namespace cache;
+using namespace boost::unit_test;
+
+/**
+ * Check operators for type.
+ * All args should refer to a different instances.
+ * Also, val1 should be less then val2 and equeal val3.
+ */
+template<typename T>
+void CheckOperators(const T& val1, const T& val2, const T& val3)
+{
+    BOOST_CHECK(&val1 != &val2);
+    BOOST_CHECK(&val1 != &val3);
+    BOOST_CHECK(&val2 != &val3);
+
+    BOOST_CHECK(val1 == val1);
+    BOOST_CHECK(val2 == val2);
+
+    BOOST_CHECK(val1 == val3);
+    BOOST_CHECK(val3 == val1);
+
+    BOOST_CHECK(val1 != val2);
+    BOOST_CHECK(val2 != val1);
+    BOOST_CHECK(val3 != val2);
+    BOOST_CHECK(val2 != val3);
+
+    BOOST_CHECK(val1 < val2);
+    BOOST_CHECK(val3 < val2);
+
+    BOOST_CHECK(val1 <= val2);
+    BOOST_CHECK(val3 <= val2);
+    BOOST_CHECK(val1 <= val1);
+    BOOST_CHECK(val2 <= val2);
+    BOOST_CHECK(val1 <= val3);
+    BOOST_CHECK(val3 <= val1);
+
+    BOOST_CHECK(val2 > val1);
+    BOOST_CHECK(val2 > val3);
+
+    BOOST_CHECK(val2 >= val1);
+    BOOST_CHECK(val2 >= val3);
+    BOOST_CHECK(val1 >= val1);
+    BOOST_CHECK(val2 >= val2);
+    BOOST_CHECK(val1 >= val3);
+    BOOST_CHECK(val3 >= val1);
+}
+
+void CheckDate(int year, int mon, int day)
+{
+    Date date = common::MakeDateGmt(year, mon, day);
+    tm res;
+
+    common::DateToCTm(date, res);
+
+    BOOST_CHECK_EQUAL(res.tm_year + 1900, year);
+    BOOST_CHECK_EQUAL(res.tm_mon + 1, mon);
+    BOOST_CHECK_EQUAL(res.tm_mday, day);
+}
+
+void CheckTimestamp(int year, int mon, int day, int hour, int mins, int sec, int ns)
+{
+    Timestamp ts = common::MakeTimestampGmt(year, mon, day, hour, mins, sec, ns);
+    tm res;
+
+    common::TimestampToCTm(ts, res);
+
+    BOOST_CHECK_EQUAL(res.tm_year + 1900, year);
+    BOOST_CHECK_EQUAL(res.tm_mon + 1, mon);
+    BOOST_CHECK_EQUAL(res.tm_mday, day);
+    BOOST_CHECK_EQUAL(res.tm_hour, hour);
+    BOOST_CHECK_EQUAL(res.tm_min, mins);
+    BOOST_CHECK_EQUAL(res.tm_sec, sec);
+}
+
+BOOST_AUTO_TEST_SUITE(DateTimeTestSuite)
+
+BOOST_AUTO_TEST_CASE(TimeOperators1)
+{
+    Time val1(1);
+    Time val2(2);
+    Time val3(1);
+
+    CheckOperators(val1, val2, val3);
+}
+
+BOOST_AUTO_TEST_CASE(TimeOperators2)
+{
+    Time val1(154362);
+    Time val2(val1.GetMilliseconds() + 42363);
+    Time val3(val1);
+
+    CheckOperators(val1, val2, val3);
+}
+
+BOOST_AUTO_TEST_CASE(DateOperators1)
+{
+    Date val1(1);
+    Date val2(2);
+    Date val3(1);
+
+    CheckOperators(val1, val2, val3);
+}
+
+BOOST_AUTO_TEST_CASE(DateOperators2)
+{
+    Date val1(154362);
+    Date val2(val1.GetMilliseconds() + 42363);
+    Date val3(val1);
+
+    CheckOperators(val1, val2, val3);
+}
+
+BOOST_AUTO_TEST_CASE(TimestampOperators1)
+{
+    Timestamp val1(1);
+    Timestamp val2(2);
+    Timestamp val3(1);
+
+    CheckOperators(val1, val2, val3);
+}
+
+BOOST_AUTO_TEST_CASE(TimestampOperators2)
+{
+    Timestamp val1(154362);
+    Timestamp val2(val1.GetMilliseconds() + 42363);
+    Timestamp val3(val1);
+
+    CheckOperators(val1, val2, val3);
+}
+
+BOOST_AUTO_TEST_CASE(TimestampOperators3)
+{
+    Timestamp val1(42, 1);
+    Timestamp val2(42, 2);
+    Timestamp val3(42, 1);
+
+    CheckOperators(val1, val2, val3);
+}
+
+BOOST_AUTO_TEST_CASE(TimestampOperators4)
+{
+    Timestamp val1(42, 154362);
+    Timestamp val2(42, val1.GetSecondFraction() + 42363);
+    Timestamp val3(42, val1.GetSecondFraction());
+
+    CheckOperators(val1, val2, val3);
+}
+
+BOOST_AUTO_TEST_CASE(TimestampOperators5)
+{
+    Timestamp val1(154362, 154362);
+    Timestamp val2(val1.GetMilliseconds() + 42363, val1.GetSecondFraction() + 42363);
+    Timestamp val3(val1);
+
+    CheckOperators(val1, val2, val3);
+}
+
+BOOST_AUTO_TEST_CASE(MakeTime)
+{
+    using namespace common;
+
+    BOOST_CHECK_EQUAL(MakeTimeGmt(0, 0, 0).GetMilliseconds(), 0);
+    BOOST_CHECK_EQUAL(MakeTimeGmt(23, 59, 59).GetMilliseconds(), 86399000);
+    BOOST_CHECK_EQUAL(MakeTimeGmt(14, 23, 47).GetMilliseconds(), 51827000);
+    BOOST_CHECK_EQUAL(MakeTimeGmt(0, 24, 12).GetMilliseconds(), 1452000);
+    BOOST_CHECK_EQUAL(MakeTimeGmt(8, 0, 0).GetMilliseconds(), 28800000);
+
+    BOOST_CHECK_EQUAL(MakeTimeGmt(0, 0, 0).GetSeconds(), 0);
+    BOOST_CHECK_EQUAL(MakeTimeGmt(23, 59, 59).GetSeconds(), 86399);
+    BOOST_CHECK_EQUAL(MakeTimeGmt(14, 23, 47).GetSeconds(), 51827);
+    BOOST_CHECK_EQUAL(MakeTimeGmt(0, 24, 12).GetSeconds(), 1452);
+    BOOST_CHECK_EQUAL(MakeTimeGmt(8, 0, 0).GetSeconds(), 28800);
+}
+
+BOOST_AUTO_TEST_CASE(MakeDate)
+{
+    using namespace common;
+
+    BOOST_CHECK_EQUAL(MakeDateGmt(1970, 1, 1).GetMilliseconds(), 0);
+    BOOST_CHECK_EQUAL(MakeDateGmt(2000, 12, 31).GetMilliseconds(), 978220800000);
+    BOOST_CHECK_EQUAL(MakeDateGmt(2017, 3, 20).GetMilliseconds(), 1489968000000);
+
+    BOOST_CHECK_EQUAL(MakeDateGmt(1970, 1, 1).GetSeconds(), 0);
+    BOOST_CHECK_EQUAL(MakeDateGmt(2000, 12, 31).GetSeconds(), 978220800);
+    BOOST_CHECK_EQUAL(MakeDateGmt(2017, 3, 20).GetSeconds(), 1489968000);
+}
+
+BOOST_AUTO_TEST_CASE(MakeTimestamp)
+{
+    using namespace common;
+
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(1970, 1, 1, 0, 0, 0, 0).GetMilliseconds(), 0);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2000, 12, 31, 23, 59, 59, 999999999).GetMilliseconds(), 978307199999);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2001, 9, 9, 1, 46, 39, 999999999).GetMilliseconds(), 999999999999);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2017, 3, 20, 18, 43, 19, 170038645).GetMilliseconds(), 1490035399170);
+
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(1970, 1, 1, 0, 0, 0, 0).GetSeconds(), 0);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2000, 12, 31, 23, 59, 59, 999999999).GetSeconds(), 978307199);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2001, 9, 9, 1, 46, 39, 999999999).GetSeconds(), 999999999);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2017, 3, 20, 18, 43, 19, 170038645).GetSeconds(), 1490035399);
+
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(1970, 1, 1, 0, 0, 0, 0).GetSecondFraction(), 0);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2000, 12, 31, 23, 59, 59, 999999999).GetSecondFraction(), 999999999);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2001, 9, 9, 1, 46, 39, 999999999).GetSecondFraction(), 999999999);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2017, 3, 20, 18, 43, 19, 170038645).GetSecondFraction(), 170038645);
+
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(1970, 1, 1, 0, 0, 0, 0).GetDate().GetMilliseconds(), 0);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2000, 12, 31, 23, 59, 59, 999999999).GetDate().GetMilliseconds(), 978307199999);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2001, 9, 9, 1, 46, 39, 999999999).GetDate().GetMilliseconds(), 999999999999);
+    BOOST_CHECK_EQUAL(MakeTimestampGmt(2017, 3, 20, 18, 43, 19, 170038645).GetDate().GetMilliseconds(), 1490035399170);
+}
+
+BOOST_AUTO_TEST_CASE(CastDateToTm)
+{
+    CheckDate(2024, 8, 5);
+    CheckDate(1987, 1, 1);
+    CheckDate(1999, 2, 18);
+    CheckDate(1997, 12, 9);
+    CheckDate(2007, 12, 31);
+    CheckDate(2001, 6, 21);
+}
+
+BOOST_AUTO_TEST_CASE(CastTimestampToTm)
+{
+    CheckTimestamp(1970, 1, 1, 0, 0, 0, 0);
+    CheckTimestamp(2000, 12, 31, 23, 59, 59, 999999999);
+    CheckTimestamp(2001, 9, 9, 1, 46, 39, 999999999);
+    CheckTimestamp(2017, 3, 20, 18, 43, 19, 170038645);
+    CheckTimestamp(2007, 12, 31, 19, 24, 44, 894375963);
+    CheckTimestamp(2001, 6, 21, 13, 53, 2, 25346547);
+}
+
+BOOST_AUTO_TEST_SUITE_END()


[12/23] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-4565-ddl
Commit: 19ba8f477b07074439718d4e75cddefe4bf6c71a
Parents: a140a83 92beda2
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Mar 21 14:33:23 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Mar 21 14:33:23 2017 +0300

----------------------------------------------------------------------
 .gitignore                                      |    2 +-
 assembly/release-fabric-base.xml                |    6 +
 config/hadoop/default-config.xml                |   29 -
 examples/config/filesystem/example-igfs.xml     |   27 -
 .../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  |    6 +-
 .../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 +-
 .../apache/ignite/p2p/P2PGridifySelfTest.java   |    8 +-
 .../singlesplit/SingleSplitsLoadTest.java       |    2 +-
 .../gridify/GridifySingleSplitLoadTest.java     |    6 +-
 .../org/test/gridify/ExternalAopTarget.java     |   16 +-
 .../gridify/ExternalNonSpringAopSelfTest.java   |    6 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |    4 +-
 .../s3/S3CheckpointManagerSelfTest.java         |    6 +-
 .../jmh/cache/JmhCacheAbstractBenchmark.java    |    6 +-
 .../internal/TaskEventSubjectIdSelfTest.java    |    4 +-
 .../ClientAbstractMultiThreadedSelfTest.java    |    4 +-
 .../client/ClientDefaultCacheSelfTest.java      |    4 +-
 .../internal/client/ClientStartNodeTask.java    |   14 +-
 .../internal/client/ClientStopNodeTask.java     |    2 +-
 .../ClientTcpSslAuthenticationSelfTest.java     |    4 +-
 ...skExecutionAfterTopologyRestartSelfTest.java |    4 +-
 .../internal/client/ClientTestRestServer.java   |    6 +-
 .../ClientAbstractMultiNodeSelfTest.java        |   17 +-
 .../integration/ClientAbstractSelfTest.java     |    4 +-
 .../integration/ClientPreferDirectSelfTest.java |    4 +-
 .../client/router/ClientFailedInitSelfTest.java |    4 +-
 .../client/router/RouterFactorySelfTest.java    |    4 +-
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |   16 +-
 ...BinaryMarshallerInsertStatementSelfTest.java |    4 +-
 ...cBinaryMarshallerMergeStatementSelfTest.java |    4 +-
 .../jdbc2/JdbcComplexQuerySelfTest.java         |    4 +-
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |    4 +-
 .../jdbc2/JdbcDistributedJoinsQueryTest.java    |    4 +-
 .../internal/jdbc2/JdbcEmptyCacheSelfTest.java  |    4 +-
 .../internal/jdbc2/JdbcLocalCachesSelfTest.java |    4 +-
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |    4 +-
 .../internal/jdbc2/JdbcNoDefaultCacheTest.java  |    4 +-
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |    4 +-
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |    4 +-
 .../internal/jdbc2/JdbcStatementSelfTest.java   |    4 +-
 .../rest/AbstractRestProcessorSelfTest.java     |    4 +-
 .../rest/ClientMemcachedProtocolSelfTest.java   |    4 +-
 .../JettyRestProcessorAbstractSelfTest.java     |   29 +-
 .../rest/JettyRestProcessorSignedSelfTest.java  |    4 +-
 .../rest/RestBinaryProtocolSelfTest.java        |    4 +-
 .../rest/RestMemcacheProtocolSelfTest.java      |    4 +-
 .../rest/RestProcessorMultiStartSelfTest.java   |    4 +-
 .../rest/RestProcessorStartSelfTest.java        |    8 +-
 .../rest/TaskCommandHandlerSelfTest.java        |    4 +-
 .../rest/protocols/tcp/MockNioSession.java      |    5 +-
 .../tcp/redis/RedisProtocolSelfTest.java        |    4 +-
 .../ignite/jdbc/JdbcComplexQuerySelfTest.java   |    4 +-
 .../ignite/jdbc/JdbcConnectionSelfTest.java     |    6 +-
 .../ignite/jdbc/JdbcEmptyCacheSelfTest.java     |    4 +-
 .../ignite/jdbc/JdbcLocalCachesSelfTest.java    |    4 +-
 .../ignite/jdbc/JdbcMetadataSelfTest.java       |    4 +-
 .../ignite/jdbc/JdbcNoDefaultCacheTest.java     |    4 +-
 .../jdbc/JdbcPreparedStatementSelfTest.java     |    4 +-
 .../ignite/jdbc/JdbcResultSetSelfTest.java      |    4 +-
 .../ignite/jdbc/JdbcStatementSelfTest.java      |    4 +-
 .../src/test/resources/spring-server-node.xml   |    2 +-
 .../test/resources/spring-server-ssl-node.xml   |    2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   15 +-
 .../apache/ignite/IgniteSystemProperties.java   |    8 +
 .../main/java/org/apache/ignite/Ignition.java   |   43 +-
 .../org/apache/ignite/IgnitionListener.java     |    2 +-
 .../apache/ignite/binary/BinaryRawReader.java   |   13 +
 .../apache/ignite/binary/BinaryRawWriter.java   |   13 +
 .../org/apache/ignite/binary/BinaryReader.java  |   15 +
 .../org/apache/ignite/binary/BinaryWriter.java  |   15 +
 .../org/apache/ignite/cache/CacheManager.java   |    2 +-
 .../apache/ignite/cache/CacheTypeMetadata.java  |    4 +-
 .../org/apache/ignite/cache/QueryEntity.java    |    4 +-
 .../org/apache/ignite/cache/QueryIndex.java     |   25 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |    4 +-
 .../org/apache/ignite/cache/query/SqlQuery.java |    6 +-
 .../apache/ignite/cache/query/TextQuery.java    |    4 +-
 .../apache/ignite/compute/gridify/Gridify.java  |   10 +
 .../ignite/compute/gridify/GridifySetToSet.java |   10 +
 .../compute/gridify/GridifySetToValue.java      |   10 +
 .../configuration/CacheConfiguration.java       |   65 +-
 .../configuration/FileSystemConfiguration.java  |   75 +-
 .../configuration/HadoopConfiguration.java      |    2 +-
 .../configuration/IgniteConfiguration.java      |  154 +-
 .../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     |  132 +-
 .../apache/ignite/internal/GridComponent.java   |   48 +-
 .../apache/ignite/internal/GridDiagnostic.java  |   16 +-
 .../ignite/internal/GridJobSiblingImpl.java     |    4 +-
 .../ignite/internal/GridKernalContext.java      |   35 +-
 .../ignite/internal/GridKernalContextImpl.java  |   64 +-
 .../ignite/internal/GridKernalGatewayImpl.java  |   14 +-
 .../apache/ignite/internal/GridLoggerProxy.java |   25 +-
 .../ignite/internal/GridPluginComponent.java    |   21 +-
 .../org/apache/ignite/internal/GridTopic.java   |    3 +
 .../ignite/internal/IgniteComputeImpl.java      |   15 +-
 .../apache/ignite/internal/IgniteKernal.java    |   85 +-
 .../ignite/internal/IgniteNodeAttributes.java   |   10 +-
 .../ignite/internal/IgniteTransactionsEx.java   |    8 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  304 +-
 .../ignite/internal/MappingStoreTask.java       |   59 +
 .../internal/MarshallerContextAdapter.java      |  211 --
 .../ignite/internal/MarshallerContextImpl.java  |  694 +++--
 .../internal/MarshallerMappingFileStore.java    |  174 ++
 .../ignite/internal/MarshallerPlatformIds.java  |   30 +
 .../internal/binary/BinaryClassDescriptor.java  |  103 +-
 .../ignite/internal/binary/BinaryContext.java   |   21 +-
 .../internal/binary/BinaryFieldAccessor.java    |   79 +-
 .../binary/BinaryMetadataCollector.java         |   11 +
 .../internal/binary/BinaryObjectImpl.java       |   50 +-
 .../binary/BinaryObjectOffheapImpl.java         |    9 +
 .../internal/binary/BinaryReaderExImpl.java     |  344 ++-
 .../binary/BinarySerializedFieldComparator.java |    3 +
 .../ignite/internal/binary/BinaryUtils.java     |  147 +-
 .../ignite/internal/binary/BinaryWriteMode.java |    6 +
 .../internal/binary/BinaryWriterExImpl.java     |   82 +-
 .../internal/binary/GridBinaryMarshaller.java   |    8 +-
 .../binary/builder/BinaryBuilderReader.java     |   37 +
 .../GridClientConnectionManagerAdapter.java     |    2 +-
 .../connection/GridClientNioTcpConnection.java  |    2 +-
 .../GridClientOptimizedMarshaller.java          |   16 +-
 .../client/router/impl/GridTcpRouterImpl.java   |    4 +-
 .../internal/cluster/ClusterGroupAdapter.java   |   10 +-
 .../cluster/ClusterNodeLocalMapImpl.java        |    2 +-
 .../internal/cluster/IgniteClusterImpl.java     |    2 +-
 .../igfs/common/IgfsHandshakeRequest.java       |   17 -
 .../internal/igfs/common/IgfsMarshaller.java    |    6 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |    4 +-
 .../apache/ignite/internal/jdbc2/JdbcUtils.java |    4 +-
 .../internal/managers/GridManagerAdapter.java   |   23 +-
 .../checkpoint/GridCheckpointManager.java       |    4 +-
 .../managers/communication/GridIoManager.java   |  116 +-
 .../communication/GridIoMessageFactory.java     |   38 +-
 .../managers/communication/GridIoPolicy.java    |   13 +-
 .../deployment/GridDeploymentCommunication.java |    6 +-
 .../discovery/GridDiscoveryManager.java         |   70 +-
 .../eventstorage/GridEventStorageManager.java   |    8 +-
 .../failover/GridFailoverContextImpl.java       |   11 -
 .../managers/failover/GridFailoverManager.java  |    3 -
 .../processors/GridProcessorAdapter.java        |   21 +-
 .../affinity/GridAffinityProcessor.java         |    2 +-
 .../processors/cache/CacheObjectContext.java    |    3 +-
 .../cache/CachePartitionExchangeWorkerTask.java |   29 +
 .../internal/processors/cache/CacheType.java    |    8 +-
 .../processors/cache/GridCacheAdapter.java      |  102 +-
 .../processors/cache/GridCacheAtomicFuture.java |    5 +-
 .../cache/GridCacheClearAllRunnable.java        |    4 +-
 .../processors/cache/GridCacheContext.java      |   11 +-
 .../cache/GridCacheDeploymentManager.java       |    8 +-
 .../processors/cache/GridCacheEventManager.java |    4 +-
 .../cache/GridCacheEvictionManager.java         |    9 +-
 .../processors/cache/GridCacheGateway.java      |    3 +-
 .../processors/cache/GridCacheIoManager.java    |   95 +-
 .../processors/cache/GridCacheMapEntry.java     |   12 +-
 .../processors/cache/GridCacheMessage.java      |   17 +-
 .../processors/cache/GridCacheMvccManager.java  |   50 +-
 .../GridCachePartitionExchangeManager.java      |  179 +-
 .../processors/cache/GridCacheProcessor.java    |  320 +-
 .../processors/cache/GridCacheProxyImpl.java    |    6 +-
 .../processors/cache/GridCacheReturn.java       |    6 +-
 .../cache/GridCacheSharedContext.java           |   17 +-
 .../cache/GridCacheSharedTtlCleanupManager.java |    2 +-
 .../processors/cache/GridCacheSwapManager.java  |    4 +-
 .../processors/cache/GridCacheTtlManager.java   |    4 +-
 .../processors/cache/GridCacheUtils.java        |   19 +-
 .../cache/GridDeferredAckMessageSender.java     |   17 +-
 .../processors/cache/IgniteCacheProxy.java      |    4 +-
 .../processors/cache/IgniteInternalCache.java   |    5 +-
 .../processors/cache/KeyCacheObjectImpl.java    |   74 +-
 .../distributed/GridCacheCommittedTxInfo.java   |  117 -
 .../distributed/GridDistributedBaseMessage.java |    2 +-
 .../GridDistributedCacheAdapter.java            |    2 +-
 .../distributed/GridDistributedLockRequest.java |   26 +-
 .../GridDistributedTxFinishRequest.java         |  217 +-
 .../GridDistributedTxFinishResponse.java        |   75 +-
 .../distributed/GridDistributedTxMapping.java   |   45 +-
 .../GridDistributedTxPrepareRequest.java        |  205 +-
 .../GridDistributedTxPrepareResponse.java       |   76 +-
 .../GridDistributedTxRemoteAdapter.java         |   59 +-
 .../GridDistributedUnlockRequest.java           |   18 +-
 .../dht/GridClientPartitionTopology.java        |   15 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   10 +-
 .../distributed/dht/GridDhtLockRequest.java     |   16 +-
 .../dht/GridDhtPartitionTopology.java           |    9 +
 .../dht/GridDhtPartitionTopologyImpl.java       |   36 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   57 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   47 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |  251 +-
 .../dht/GridDhtTxFinishResponse.java            |   70 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |  160 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   28 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  195 +-
 .../dht/GridDhtTxPrepareRequest.java            |  138 +-
 .../dht/GridDhtTxPrepareResponse.java           |  117 +-
 .../distributed/dht/GridDhtUnlockRequest.java   |   15 +-
 .../GridDhtAtomicAbstractUpdateFuture.java      |  298 +-
 .../GridDhtAtomicAbstractUpdateRequest.java     |  392 ++-
 .../dht/atomic/GridDhtAtomicCache.java          |  896 +++---
 .../GridDhtAtomicDeferredUpdateResponse.java    |   68 +-
 .../dht/atomic/GridDhtAtomicNearResponse.java   |  314 ++
 .../atomic/GridDhtAtomicSingleUpdateFuture.java |  101 +-
 .../GridDhtAtomicSingleUpdateRequest.java       |  277 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   89 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  325 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |  124 +-
 ...idNearAtomicAbstractSingleUpdateRequest.java |  481 +--
 .../GridNearAtomicAbstractUpdateFuture.java     |  468 ++-
 .../GridNearAtomicAbstractUpdateRequest.java    |  480 ++-
 .../GridNearAtomicCheckUpdateRequest.java       |  175 ++
 .../atomic/GridNearAtomicFullUpdateRequest.java |  487 +--
 ...GridNearAtomicSingleUpdateFilterRequest.java |   23 +-
 .../GridNearAtomicSingleUpdateFuture.java       |  617 ++--
 ...GridNearAtomicSingleUpdateInvokeRequest.java |   37 +-
 .../GridNearAtomicSingleUpdateRequest.java      |   65 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  962 +++---
 .../atomic/GridNearAtomicUpdateResponse.java    |  378 +--
 .../dht/atomic/NearCacheUpdates.java            |  335 +++
 .../distributed/dht/atomic/UpdateErrors.java    |  222 ++
 .../dht/colocated/GridDhtColocatedCache.java    |    8 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   25 +-
 .../dht/preloader/GridDhtPartitionDemander.java |    9 +-
 .../GridDhtPartitionsExchangeFuture.java        |    8 +-
 .../distributed/near/GridNearAtomicCache.java   |   27 +-
 .../distributed/near/GridNearGetRequest.java    |   55 +-
 .../distributed/near/GridNearLockFuture.java    |   24 +-
 .../distributed/near/GridNearLockRequest.java   |  220 +-
 .../distributed/near/GridNearLockResponse.java  |   12 +-
 ...arOptimisticSerializableTxPrepareFuture.java |   72 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   41 +-
 ...ridNearOptimisticTxPrepareFutureAdapter.java |   13 +
 .../GridNearPessimisticTxPrepareFuture.java     |   67 +-
 .../near/GridNearSingleGetRequest.java          |   46 +-
 .../near/GridNearTransactionalCache.java        |    6 +-
 .../near/GridNearTxFinishFuture.java            |  176 +-
 .../near/GridNearTxFinishRequest.java           |  174 +-
 .../near/GridNearTxFinishResponse.java          |   36 +-
 .../cache/distributed/near/GridNearTxLocal.java | 2706 ++++++++++++++++-
 .../near/GridNearTxPrepareFutureAdapter.java    |   22 +-
 .../near/GridNearTxPrepareRequest.java          |  254 +-
 .../near/GridNearTxPrepareResponse.java         |   86 +-
 .../distributed/near/GridNearTxRemote.java      |    4 +-
 .../distributed/near/GridNearUnlockRequest.java |   20 +-
 .../distributed/near/IgniteTxMappingsImpl.java  |    4 +-
 .../near/IgniteTxMappingsSingleImpl.java        |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |    4 +-
 .../cache/query/GridCacheQueryManager.java      |   64 +-
 .../continuous/CacheContinuousQueryHandler.java |    2 +-
 .../store/GridCacheStoreManagerAdapter.java     |  144 +-
 .../cache/store/GridCacheWriteBehindStore.java  |   16 +-
 .../cache/transactions/IgniteInternalTx.java    |   80 +-
 .../transactions/IgniteTransactionsImpl.java    |   12 +-
 .../cache/transactions/IgniteTxAdapter.java     |  165 +-
 .../cache/transactions/IgniteTxEntry.java       |   48 +-
 .../cache/transactions/IgniteTxHandler.java     |  157 +-
 .../IgniteTxImplicitSingleStateImpl.java        |    4 +-
 .../transactions/IgniteTxLocalAdapter.java      | 2799 ++----------------
 .../cache/transactions/IgniteTxLocalEx.java     |  145 +-
 .../cache/transactions/IgniteTxManager.java     |  299 +-
 .../cache/transactions/IgniteTxRemoteEx.java    |   11 +
 .../IgniteTxRemoteStateAdapter.java             |    2 +-
 .../cache/transactions/IgniteTxState.java       |    2 +-
 .../cache/transactions/IgniteTxStateImpl.java   |    4 +-
 .../transactions/TransactionProxyImpl.java      |   13 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |   22 +-
 .../processors/clock/GridClockServer.java       |    2 +-
 .../clock/GridClockSyncProcessor.java           |    4 +-
 .../processors/closure/AffinityTask.java        |    6 -
 .../closure/GridClosureProcessor.java           |   43 +-
 .../processors/cluster/ClusterProcessor.java    |   55 +-
 .../processors/cluster/GridUpdateNotifier.java  |   14 +-
 .../continuous/GridContinuousProcessor.java     |  148 +-
 .../datastreamer/DataStreamProcessor.java       |   63 +-
 .../datastreamer/DataStreamerImpl.java          |   39 +-
 .../datastructures/DataStructuresProcessor.java |   34 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   18 +-
 .../GridCacheAtomicReferenceImpl.java           |    6 +-
 .../GridCacheAtomicSequenceImpl.java            |    4 +-
 .../GridCacheAtomicStampedImpl.java             |    6 +-
 .../GridCacheCountDownLatchImpl.java            |    6 +-
 .../datastructures/GridCacheLockImpl.java       |   11 +-
 .../datastructures/GridCacheSemaphoreImpl.java  |   18 +-
 .../GridTransactionalCacheQueueImpl.java        |   10 +-
 .../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/HadoopMapReducePlan.java  |   80 -
 .../hadoop/HadoopMapReducePlanner.java          |   40 -
 .../processors/hadoop/HadoopTaskContext.java    |    6 +-
 .../processors/hadoop/HadoopTaskInfo.java       |    1 +
 .../hadoop/counter/HadoopCounterWriter.java     |    4 +-
 .../processors/igfs/IgfsBaseBlockKey.java       |   42 +
 .../internal/processors/igfs/IgfsBlockKey.java  |   26 +-
 .../processors/igfs/IgfsBlockLocationImpl.java  |   55 +
 .../internal/processors/igfs/IgfsContext.java   |    8 +-
 .../processors/igfs/IgfsDataManager.java        |   86 +-
 .../internal/processors/igfs/IgfsFileImpl.java  |    5 +-
 .../igfs/IgfsFragmentizerManager.java           |    4 +-
 .../internal/processors/igfs/IgfsImpl.java      |   58 +-
 .../processors/igfs/IgfsIpcHandler.java         |    6 +-
 .../processors/igfs/IgfsKernalContextAware.java |   32 -
 .../processors/igfs/IgfsMetaManager.java        |  102 +-
 .../processors/igfs/IgfsNoopProcessor.java      |    2 +-
 .../internal/processors/igfs/IgfsProcessor.java |  132 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   10 +-
 .../igfs/IgfsSecondaryFileSystemV2.java         |   40 -
 .../internal/processors/igfs/IgfsServer.java    |    6 +-
 .../processors/igfs/IgfsServerManager.java      |    2 +-
 .../processors/igfs/IgfsThreadFactory.java      |    6 +-
 .../internal/processors/igfs/IgfsUtils.java     |  320 +-
 .../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 |    6 +-
 .../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 +-
 .../local/LocalFileSystemBlockKey.java          |  103 +
 .../local/LocalFileSystemIgfsFile.java          |   10 +-
 .../LocalFileSystemPositionedReadable.java      |   65 +
 .../secondary/local/LocalFileSystemUtils.java   |   25 +
 ...fsSecondaryFileSystemPositionedReadable.java |   65 -
 .../processors/job/GridJobProcessor.java        |    6 +-
 .../internal/processors/job/GridJobWorker.java  |    4 +-
 .../jobmetrics/GridJobMetricsProcessor.java     |    3 +-
 .../marshaller/ClientRequestFuture.java         |  183 ++
 .../GridMarshallerMappingProcessor.java         |  363 +++
 .../processors/marshaller/MappedName.java       |   63 +
 .../marshaller/MappingAcceptedMessage.java      |   71 +
 .../marshaller/MappingExchangeResult.java       |   96 +
 .../marshaller/MappingProposedMessage.java      |  137 +
 .../marshaller/MappingUpdatedListener.java      |   29 +
 .../marshaller/MarshallerMappingItem.java       |   99 +
 .../marshaller/MarshallerMappingTransport.java  |  212 ++
 .../MissingMappingRequestMessage.java           |  146 +
 .../MissingMappingResponseMessage.java          |  169 ++
 .../internal/processors/odbc/OdbcProcessor.java |    6 +-
 .../processors/odbc/OdbcRequestHandler.java     |    2 +
 .../platform/PlatformAbstractTarget.java        |    8 +
 .../platform/PlatformAsyncResult.java           |   41 +
 .../platform/PlatformContextImpl.java           |   10 +-
 .../processors/platform/PlatformIgnition.java   |   34 +-
 .../platform/PlatformNoopProcessor.java         |    5 +
 .../platform/PlatformPluginExtension.java       |   39 +
 .../processors/platform/PlatformProcessor.java  |    7 +
 .../platform/PlatformProcessorImpl.java         |   73 +-
 .../processors/platform/PlatformTarget.java     |   10 +
 .../platform/PlatformTargetProxy.java           |    9 +
 .../platform/PlatformTargetProxyImpl.java       |   39 +
 .../cache/PlatformCacheEntryProcessorImpl.java  |    6 +-
 .../callback/PlatformCallbackGateway.java       |  119 +
 .../platform/callback/PlatformCallbackOp.java   |   21 +
 .../platform/compute/PlatformAbstractJob.java   |    2 +-
 .../plugin/PlatformPluginProcessor.java         |   46 +
 .../cache/PlatformCachePluginConfiguration.java |   58 +
 .../cache/PlatformCachePluginProvider.java      |  123 +
 .../utils/PlatformConfigurationUtils.java       |  154 +-
 .../platform/utils/PlatformUtils.java           |   56 +-
 .../processors/plugin/CachePluginManager.java   |    8 +-
 .../plugin/IgnitePluginProcessor.java           |   84 +-
 .../internal/processors/pool/PoolProcessor.java |   15 +-
 .../processors/port/GridPortProcessor.java      |    2 +-
 .../query/GridQueryIndexDescriptor.java         |    4 +-
 .../processors/query/GridQueryIndexType.java    |   32 -
 .../processors/query/GridQueryIndexing.java     |   16 +-
 .../processors/query/GridQueryProcessor.java    | 2270 +-------------
 .../processors/query/GridQueryProperty.java     |   21 +-
 .../query/QueryIndexDescriptorImpl.java         |  102 +
 .../processors/query/QueryTypeCandidate.java    |   66 +
 .../query/QueryTypeDescriptorImpl.java          |  337 +++
 .../processors/query/QueryTypeIdKey.java        |   94 +
 .../processors/query/QueryTypeNameKey.java      |   68 +
 .../internal/processors/query/QueryUtils.java   | 1003 +++++++
 .../query/property/QueryBinaryProperty.java     |  267 ++
 .../query/property/QueryClassProperty.java      |  130 +
 .../query/property/QueryFieldAccessor.java      |   67 +
 .../query/property/QueryMethodsAccessor.java    |   82 +
 .../query/property/QueryPropertyAccessor.java   |   53 +
 .../property/QueryReadOnlyMethodsAccessor.java  |   71 +
 .../processors/resource/GridResourceIoc.java    |    6 +-
 .../resource/GridResourceProcessor.java         |   32 +-
 .../processors/rest/GridRestProcessor.java      |    8 +-
 .../handlers/task/GridTaskCommandHandler.java   |    4 +-
 .../version/GridVersionCommandHandler.java      |    2 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |    2 +-
 .../service/GridServiceProcessor.java           |   16 +-
 .../session/GridTaskSessionProcessor.java       |    2 +-
 .../processors/task/GridTaskProcessor.java      |    4 +-
 .../processors/task/GridTaskWorker.java         |   13 +-
 .../timeout/GridTimeoutProcessor.java           |    6 +-
 .../suggestions/GridPerformanceSuggestions.java |    6 +-
 .../ignite/internal/util/IgniteUtils.java       |  103 +-
 .../ignite/internal/util/StripedExecutor.java   |   38 +-
 .../util/future/GridCompoundFuture.java         |   11 +-
 .../internal/util/ipc/IpcToNioAdapter.java      |    7 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   22 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |    7 +-
 .../util/nio/GridNioAsyncNotifyFilter.java      |   23 +-
 .../internal/util/nio/GridNioCodecFilter.java   |    9 +-
 .../util/nio/GridNioEmbeddedFuture.java         |    7 +
 .../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    |   18 +-
 .../ignite/internal/util/nio/GridNioServer.java |  131 +-
 .../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 +-
 .../util/tostring/GridToStringBuilder.java      |    2 +-
 .../ignite/internal/util/worker/GridWorker.java |   22 +-
 .../internal/visor/igfs/VisorIgfsEndpoint.java  |   26 +-
 .../visor/node/VisorBasicConfiguration.java     |   12 +-
 .../visor/node/VisorIgfsConfiguration.java      |    4 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |    8 +-
 .../node/VisorNodeDataCollectorJobResult.java   |   16 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |    2 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |   10 +-
 .../java/org/apache/ignite/lang/IgniteUuid.java |   25 +-
 .../ignite/marshaller/MarshallerContext.java    |   23 +-
 .../optimized/OptimizedMarshallerUtils.java     |   12 +-
 .../apache/ignite/mxbean/IgnitionMXBean.java    |   14 +-
 ...PlatformCachePluginConfigurationClosure.java |   31 +
 ...mCachePluginConfigurationClosureFactory.java |   37 +
 .../PlatformPluginConfigurationClosure.java     |   30 +
 ...atformPluginConfigurationClosureFactory.java |   37 +
 .../ignite/resources/FileSystemResource.java    |   62 +
 .../java/org/apache/ignite/spi/IgniteSpi.java   |    6 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   15 +-
 .../org/apache/ignite/spi/IgniteSpiThread.java  |    6 +-
 .../ignite/spi/IgniteSpiThreadFactory.java      |   10 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |    4 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |    2 +-
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |    2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   12 +-
 .../sharedfs/SharedFsTimeoutTask.java           |    6 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |    4 +-
 .../jobstealing/JobStealingCollisionSpi.java    |    4 +-
 .../spi/collision/noop/NoopCollisionSpi.java    |    2 +-
 .../PriorityQueueCollisionSpi.java              |    4 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   22 +-
 .../deployment/local/LocalDeploymentSpi.java    |    4 +-
 .../ignite/spi/discovery/DiscoveryDataBag.java  |  299 ++
 .../spi/discovery/DiscoverySpiDataExchange.java |   15 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   28 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  100 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |    4 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   79 +-
 .../tcp/internal/DiscoveryDataPacket.java       |  345 +++
 .../TcpDiscoveryJoinRequestMessage.java         |   18 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   10 +-
 .../messages/TcpDiscoveryNodeAddedMessage.java  |   78 +-
 .../memory/MemoryEventStorageSpi.java           |    4 +-
 .../ignite/spi/failover/FailoverContext.java    |   10 -
 .../spi/failover/always/AlwaysFailoverSpi.java  |    4 +-
 .../jobstealing/JobStealingFailoverSpi.java     |    4 +-
 .../spi/failover/never/NeverFailoverSpi.java    |    4 +-
 .../spi/indexing/noop/NoopIndexingSpi.java      |    2 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |    4 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |    4 +-
 .../WeightedRandomLoadBalancingSpi.java         |    4 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   10 +-
 .../spi/swapspace/noop/NoopSwapSpaceSpi.java    |    2 +-
 .../ignite/startup/BasicWarmupClosure.java      |    2 +-
 .../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  |   74 +-
 .../ignite/thread/IgniteThreadFactory.java      |   18 +-
 .../ignite/thread/IgniteThreadPoolExecutor.java |   12 +-
 .../resources/META-INF/classnames.properties    |    5 +-
 .../test/config/default-spring-url-testing.xml  |    2 +-
 modules/core/src/test/config/igfs-loopback.xml  |   27 -
 modules/core/src/test/config/igfs-shmem.xml     |   27 -
 .../core/src/test/config/load/dsi-load-base.xml |    2 +-
 .../test/config/loaders/grid-cfg-2-grids.xml    |    4 +-
 .../core/src/test/config/loaders/grid-cfg.xml   |    2 +-
 .../config/websession/example-cache-client.xml  |    2 +-
 .../test/config/websession/example-cache2.xml   |    2 +-
 .../test/config/websession/spring-cache-1.xml   |    2 +-
 .../test/config/websession/spring-cache-2.xml   |    2 +-
 .../test/config/websession/spring-cache-3.xml   |    2 +-
 .../org.apache.ignite.plugin.PluginProvider     |    1 +
 ...atformCachePluginConfigurationClosureFactory |    1 +
 ...rm.PlatformPluginConfigurationClosureFactory |    1 +
 .../GridCacheAffinityBackupsSelfTest.java       |    4 +-
 .../ignite/IgniteCacheAffinitySelfTest.java     |    6 +-
 .../cache/IgniteWarmupClosureSelfTest.java      |    4 +-
 .../affinity/AffinityClientNodeSelfTest.java    |    6 +-
 ...ityFunctionBackupFilterAbstractSelfTest.java |    4 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    4 +-
 .../affinity/AffinityHistoryCleanupTest.java    |    4 +-
 .../fair/FairAffinityDynamicCacheSelfTest.java  |    4 +-
 .../fair/FairAffinityFunctionNodesSelfTest.java |    4 +-
 .../local/LocalAffinityFunctionTest.java        |    4 +-
 ...cheStoreSessionListenerAbstractSelfTest.java |    4 +-
 ...heStoreSessionListenerLifecycleSelfTest.java |    4 +-
 .../GridCacheLoadOnlyStoreAdapterSelfTest.java  |    4 +-
 .../IgniteCacheExpiryStoreLoadSelfTest.java     |    4 +-
 .../store/StoreResourceInjectionSelfTest.java   |    4 +-
 .../CacheJdbcPojoStoreAbstractSelfTest.java     |    4 +-
 ...eJdbcStoreAbstractMultithreadedSelfTest.java |    4 +-
 ...CacheJdbcBlobStoreMultithreadedSelfTest.java |    4 +-
 .../ignite/igfs/IgfsEventsAbstractSelfTest.java |   64 +-
 .../igfs/IgfsFragmentizerAbstractSelfTest.java  |   21 +-
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |    3 +-
 .../internal/ClusterGroupAbstractTest.java      |    4 +-
 .../internal/ClusterGroupHostsSelfTest.java     |    4 +-
 .../ignite/internal/ClusterMetricsSelfTest.java |    4 +-
 .../internal/ClusterNodeMetricsSelfTest.java    |    4 +-
 .../ComputeJobCancelWithServiceSelfTest.java    |    4 +-
 .../ignite/internal/GridAffinityMappedTest.java |   11 +-
 .../internal/GridAffinityP2PSelfTest.java       |   11 +-
 .../ignite/internal/GridAffinitySelfTest.java   |   10 +-
 .../GridAlwaysFailoverSpiFailSelfTest.java      |    4 +-
 .../internal/GridCacheRecreateLockTest.java     |    2 +-
 .../internal/GridCancelUnusedJobSelfTest.java   |    6 +-
 .../GridCancelledJobsMetricsSelfTest.java       |    4 +-
 .../GridCollisionJobsContextSelfTest.java       |   10 +-
 .../internal/GridCommunicationSelfTest.java     |    2 +-
 ...omputationBinarylizableClosuresSelfTest.java |    4 +-
 .../GridContinuousJobAnnotationSelfTest.java    |    4 +-
 .../ignite/internal/GridDeploymentSelfTest.java |   16 +-
 .../internal/GridDiscoveryEventSelfTest.java    |    4 +-
 .../ignite/internal/GridDiscoverySelfTest.java  |    6 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |    2 +-
 ...ventStorageRuntimeConfigurationSelfTest.java |    4 +-
 .../internal/GridFactoryVmShutdownTest.java     |    2 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    4 +-
 .../GridFailedInputParametersSelfTest.java      |    2 +-
 .../GridFailoverCustomTopologySelfTest.java     |    4 +-
 .../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         |    4 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |    4 +-
 .../internal/GridJobStealingSelfTest.java       |    4 +-
 .../GridJobStealingZeroActiveJobsSelfTest.java  |    6 +-
 .../internal/GridLifecycleAwareSelfTest.java    |    4 +-
 .../internal/GridLifecycleBeanSelfTest.java     |   16 +-
 .../GridLocalEventListenerSelfTest.java         |    6 +-
 .../internal/GridMultipleJobsSelfTest.java      |    6 +-
 .../internal/GridMultipleSpisSelfTest.java      |    4 +-
 .../GridMultipleVersionsDeploymentSelfTest.java |    4 +-
 .../GridMultithreadedJobStealingSelfTest.java   |    4 +-
 .../ignite/internal/GridNodeLocalSelfTest.java  |    2 +-
 .../internal/GridNodeMetricsLogSelfTest.java    |    4 +-
 .../GridNodeVisorAttributesSelfTest.java        |    4 +-
 .../internal/GridNonHistoryMetricsSelfTest.java |    4 +-
 .../GridProjectionForCachesSelfTest.java        |   18 +-
 ...ectionLocalJobMultipleArgumentsSelfTest.java |    4 +-
 .../internal/GridReleaseTypeSelfTest.java       |    4 +-
 .../internal/GridRuntimeExceptionSelfTest.java  |    8 +-
 .../internal/GridSpiExceptionSelfTest.java      |    8 +-
 .../ignite/internal/GridStartStopSelfTest.java  |    6 +-
 .../internal/GridStopWithWaitSelfTest.java      |   12 +-
 .../GridTaskFailoverAffinityRunTest.java        |    6 +-
 .../GridTaskFutureImplStopGridSelfTest.java     |    8 +-
 .../GridTaskInstanceExecutionSelfTest.java      |    2 +-
 .../internal/GridTaskJobRejectSelfTest.java     |    4 +-
 .../internal/GridTaskListenerSelfTest.java      |    2 +-
 .../internal/GridTaskMapAsyncSelfTest.java      |    6 +-
 .../internal/GridTaskResultCacheSelfTest.java   |    4 +-
 .../internal/GridTaskTimeoutSelfTest.java       |    8 +-
 .../IgniteClientReconnectAbstractTest.java      |    4 +-
 .../IgniteClientReconnectApiExceptionTest.java  |    4 +-
 .../IgniteClientReconnectCacheTest.java         |   13 +-
 ...niteClientReconnectFailoverAbstractTest.java |    4 +-
 .../IgniteClientReconnectFailoverTest.java      |    4 +-
 .../IgniteClientReconnectStreamerTest.java      |    4 +-
 .../IgniteComputeEmptyClusterGroupTest.java     |    4 +-
 .../internal/IgniteExecutorServiceTest.java     |   14 +-
 ...gniteExplicitImplicitDeploymentSelfTest.java |    4 +-
 .../IgniteLocalNodeMapBeforeStartTest.java      |    2 +-
 ...RoundRobinErrorAfterClientReconnectTest.java |    6 +-
 .../IgniteSlowClientDetectionSelfTest.java      |    7 +-
 ...UpdateNotifierPerClusterSettingSelfTest.java |    4 +-
 .../MarshallerContextLockingSelfTest.java       |   67 +-
 .../ignite/internal/TaskNodeRestartTest.java    |    4 +-
 .../internal/TestRecordingCommunicationSpi.java |   33 +-
 .../BinaryArrayIdentityResolverSelfTest.java    |    4 +-
 .../BinaryConfigurationConsistencySelfTest.java |    4 +-
 .../internal/binary/BinaryEnumsSelfTest.java    |    4 +-
 .../BinaryFieldIdentityResolverSelfTest.java    |    4 +-
 ...ryIdentityResolverConfigurationSelfTest.java |    4 +-
 .../binary/BinaryMarshallerSelfTest.java        |  119 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |    4 +-
 ...naryObjectBuilderDefaultMappersSelfTest.java |    4 +-
 ...ilderSimpleNameLowerCaseMappersSelfTest.java |    4 +-
 .../binary/BinaryObjectExceptionSelfTest.java   |  209 ++
 .../binary/BinaryObjectToStringSelfTest.java    |    4 +-
 .../BinarySerialiedFieldComparatorSelfTest.java |    4 +-
 .../internal/binary/BinaryTreeSelfTest.java     |    2 +-
 .../binary/GridBinaryAffinityKeySelfTest.java   |    6 +-
 ...GridBinaryMarshallerCtxDisabledSelfTest.java |   42 +-
 ...aultBinaryMappersBinaryMetaDataSelfTest.java |    4 +-
 ...CaseBinaryMappersBinaryMetaDataSelfTest.java |    4 +-
 ...GridManagerLocalMessageListenerSelfTest.java |    6 +-
 .../managers/GridNoopManagerSelfTest.java       |    2 +-
 .../GridCheckpointManagerAbstractSelfTest.java  |   33 +-
 .../checkpoint/GridCheckpointTaskSelfTest.java  |    4 +-
 ...idCommunicationManagerListenersSelfTest.java |    6 +-
 .../GridCommunicationSendMessageSelfTest.java   |    8 +-
 .../communication/GridIoManagerSelfTest.java    |   28 +-
 .../IgniteCommunicationBalanceTest.java         |    4 +-
 .../communication/IgniteIoTestMessagesTest.java |    4 +-
 .../IgniteVariousConnectionNumberTest.java      |    6 +-
 .../GridDeploymentManagerStopSelfTest.java      |    2 +-
 .../GridDeploymentMessageCountSelfTest.java     |    6 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |    6 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |   10 +-
 .../IgniteTopologyPrintFormatSelfTest.java      |    8 +-
 .../events/GridEventStorageManagerSelfTest.java |    4 +-
 .../swapspace/GridSwapSpaceManagerSelfTest.java |    4 +-
 .../GridCacheTxLoadFromStoreOnLockSelfTest.java |    4 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    4 +-
 .../BinaryObjectOffHeapUnswapTemporaryTest.java |    4 +-
 .../cache/CacheAffinityCallSelfTest.java        |    6 +-
 .../CacheAtomicSingleMessageCountSelfTest.java  |    4 +-
 .../cache/CacheClientStoreSelfTest.java         |    6 +-
 ...cheDhtLocalPartitionAfterRemoveSelfTest.java |    4 +-
 .../cache/CacheEntryProcessorCopySelfTest.java  |    4 +-
 .../cache/CacheEnumOperationsAbstractTest.java  |    4 +-
 ...CacheExchangeMessageDuplicatedStateTest.java |    6 +-
 .../cache/CacheFutureExceptionSelfTest.java     |    6 +-
 .../cache/CacheGetEntryAbstractTest.java        |    4 +-
 ...erceptorPartitionCounterLocalSanityTest.java |    4 +-
 ...torPartitionCounterRandomOperationsTest.java |    4 +-
 .../CacheMetricsForClusterGroupSelfTest.java    |    4 +-
 .../processors/cache/CacheNamesSelfTest.java    |    4 +-
 .../CacheNamesWithSpecialCharactersTest.java    |    4 +-
 .../cache/CacheNearReaderUpdateTest.java        |    4 +-
 .../cache/CacheOffheapMapEntrySelfTest.java     |    6 +-
 .../CachePutEventListenerErrorSelfTest.java     |    4 +-
 .../processors/cache/CachePutIfAbsentTest.java  |    4 +-
 .../cache/CacheReadThroughRestartSelfTest.java  |    6 +-
 .../cache/CacheRebalancingSelfTest.java         |   20 +-
 .../CacheSerializableTransactionsTest.java      |    4 +-
 .../CacheStartupInDeploymentModesTest.java      |    4 +-
 .../cache/CacheStopAndDestroySelfTest.java      |    6 +-
 .../CacheStoreUsageMultinodeAbstractTest.java   |   10 +-
 .../cache/CacheSwapUnswapGetTest.java           |    4 +-
 .../CacheSwapUnswapGetTestSmallQueueSize.java   |    4 +-
 .../processors/cache/CacheTxFastFinishTest.java |    4 +-
 .../CacheTxNotAllowReadFromBackupTest.java      |    4 +-
 .../processors/cache/CrossCacheLockTest.java    |    6 +-
 .../cache/CrossCacheTxRandomOperationsTest.java |    6 +-
 .../EntryVersionConsistencyReadThroughTest.java |    4 +-
 ...idAbstractCacheInterceptorRebalanceTest.java |    4 +-
 ...ridCacheAbstractByteArrayValuesSelfTest.java |    4 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   16 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   20 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    |   24 +-
 .../GridCacheAbstractRemoveFailureTest.java     |    6 +-
 .../cache/GridCacheAbstractSelfTest.java        |   10 +-
 .../cache/GridCacheAbstractTxReadTest.java      |    8 +-
 ...acheAbstractUsersAffinityMapperSelfTest.java |    4 +-
 .../cache/GridCacheAffinityApiSelfTest.java     |    4 +-
 .../cache/GridCacheAffinityRoutingSelfTest.java |    6 +-
 .../GridCacheAsyncOperationsLimitSelfTest.java  |    4 +-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |    4 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |   26 +-
 .../cache/GridCacheBasicApiAbstractTest.java    |    4 +-
 .../cache/GridCacheBasicStoreAbstractTest.java  |    4 +-
 ...acheBasicStoreMultithreadedAbstractTest.java |    4 +-
 .../cache/GridCacheClearAllSelfTest.java        |    4 +-
 .../cache/GridCacheClearLocallySelfTest.java    |   10 +-
 .../cache/GridCacheClearSelfTest.java           |    4 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |    4 +-
 .../cache/GridCacheConcurrentMapTest.java       |    4 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |   18 +-
 .../GridCacheConditionalDeploymentSelfTest.java |    4 +-
 ...idCacheConfigurationConsistencySelfTest.java |    4 +-
 ...ridCacheConfigurationValidationSelfTest.java |   46 +-
 .../GridCacheDaemonNodeAbstractSelfTest.java    |    4 +-
 .../cache/GridCacheDeploymentSelfTest.java      |   22 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java |   12 +-
 .../cache/GridCacheEntryVersionSelfTest.java    |    4 +-
 .../GridCacheFinishPartitionsSelfTest.java      |    4 +-
 ...CacheFullTextQueryMultithreadedSelfTest.java |    4 +-
 ...idCacheGetAndTransformStoreAbstractTest.java |    4 +-
 .../cache/GridCacheIncrementTransformTest.java  |    4 +-
 .../GridCacheInterceptorAbstractSelfTest.java   |    8 +-
 .../cache/GridCacheIteratorPerformanceTest.java |    4 +-
 .../cache/GridCacheKeyCheckSelfTest.java        |    4 +-
 .../processors/cache/GridCacheLeakTest.java     |    4 +-
 .../cache/GridCacheLifecycleAwareSelfTest.java  |    4 +-
 .../cache/GridCacheLuceneQueryIndexTest.java    |    4 +-
 .../GridCacheMarshallerTxAbstractTest.java      |    4 +-
 .../GridCacheMarshallingNodeJoinSelfTest.java   |    4 +-
 .../cache/GridCacheMemoryModeSelfTest.java      |    4 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |    4 +-
 .../cache/GridCacheMultiUpdateLockSelfTest.java |    4 +-
 ...ridCacheMultinodeUpdateAbstractSelfTest.java |    4 +-
 ...inodeUpdateNearEnabledNoBackupsSelfTest.java |    4 +-
 .../cache/GridCacheMvccManagerSelfTest.java     |    4 +-
 .../cache/GridCacheNestedTxAbstractTest.java    |    4 +-
 .../cache/GridCacheObjectToStringSelfTest.java  |    4 +-
 .../cache/GridCacheOffHeapCleanupTest.java      |    4 +-
 ...HeapMultiThreadedUpdateAbstractSelfTest.java |    2 +-
 .../GridCacheOffHeapTieredAbstractSelfTest.java |    4 +-
 ...heOffHeapTieredEvictionAbstractSelfTest.java |    4 +-
 .../cache/GridCacheOffheapUpdateSelfTest.java   |    4 +-
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |    4 +-
 .../GridCacheOrderedPreloadingSelfTest.java     |    4 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |    4 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |    4 +-
 ...hePartitionedProjectionAffinitySelfTest.java |    4 +-
 .../cache/GridCachePartitionedWritesTest.java   |    4 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |    4 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |   10 +-
 .../GridCacheQueryIndexingDisabledSelfTest.java |    4 +-
 .../GridCacheQueryInternalKeysSelfTest.java     |    4 +-
 .../GridCacheReferenceCleanupSelfTest.java      |    4 +-
 .../cache/GridCacheReloadSelfTest.java          |    4 +-
 ...ridCacheReplicatedSynchronousCommitTest.java |    6 +-
 .../GridCacheReturnValueTransferSelfTest.java   |    4 +-
 .../cache/GridCacheSlowTxWarnTest.java          |    4 +-
 .../processors/cache/GridCacheStopSelfTest.java |    4 +-
 ...ridCacheStoreManagerDeserializationTest.java |    6 +-
 .../cache/GridCacheStorePutxSelfTest.java       |    4 +-
 .../cache/GridCacheStoreValueBytesSelfTest.java |    4 +-
 .../cache/GridCacheSwapCleanupTest.java         |    4 +-
 .../cache/GridCacheSwapPreloadSelfTest.java     |    4 +-
 .../cache/GridCacheSwapReloadSelfTest.java      |    4 +-
 ...ridCacheSwapSpaceSpiConsistencySelfTest.java |   10 +-
 ...acheTcpClientDiscoveryMultiThreadedTest.java |    4 +-
 .../processors/cache/GridCacheTestStore.java    |    2 -
 .../GridCacheTtlManagerEvictionSelfTest.java    |    4 +-
 .../GridCacheTtlManagerNotificationTest.java    |    4 +-
 .../cache/GridCacheTtlManagerSelfTest.java      |    4 +-
 .../GridCacheValueBytesPreloadingSelfTest.java  |   10 +-
 ...idCacheValueConsistencyAbstractSelfTest.java |    4 +-
 .../GridCacheVariableTopologySelfTest.java      |    4 +-
 .../cache/GridCacheVersionMultinodeTest.java    |    4 +-
 .../GridCacheVersionTopologyChangeTest.java     |    4 +-
 .../cache/GridLocalIgniteSerializationTest.java |   10 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java |    4 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |   10 +-
 ...IgniteCacheBinaryEntryProcessorSelfTest.java |    6 +-
 .../IgniteCacheBinaryObjectsScanSelfTest.java   |    6 +-
 ...teCacheConfigurationDefaultTemplateTest.java |    4 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    4 +-
 .../IgniteCacheContainsKeyAbstractSelfTest.java |    8 +-
 ...niteCacheCopyOnReadDisabledAbstractTest.java |    4 +-
 .../cache/IgniteCacheCreateRestartSelfTest.java |    4 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |    9 +-
 ...niteCacheEntryListenerExpiredEventsTest.java |    4 +-
 .../IgniteCacheEntryProcessorCallTest.java      |    8 +-
 .../IgniteCacheEntryProcessorNodeJoinTest.java  |    4 +-
 ...niteCacheExpireAndUpdateConsistencyTest.java |    4 +-
 ...IgniteCacheGetCustomCollectionsSelfTest.java |    4 +-
 .../cache/IgniteCacheIncrementTxTest.java       |    6 +-
 ...gniteCacheInvokeReadThroughAbstractTest.java |    4 +-
 ...gniteCacheLoadRebalanceEvictionSelfTest.java |    4 +-
 .../IgniteCacheManyAsyncOperationsTest.java     |    6 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    6 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |    8 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |    4 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |    6 +-
 .../IgniteCachePartitionMapUpdateTest.java      |    4 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |    8 +-
 .../cache/IgniteCachePutAllRestartTest.java     |    4 +-
 .../IgniteCachePutStackOverflowSelfTest.java    |    4 +-
 .../IgniteCacheReadThroughEvictionSelfTest.java |    2 +-
 .../IgniteCacheReadThroughStoreCallTest.java    |    4 +-
 ...iteCacheScanPredicateDeploymentSelfTest.java |    6 +-
 .../cache/IgniteCacheSerializationSelfTest.java |    6 +-
 .../cache/IgniteCacheStartStopLoadTest.java     |    4 +-
 .../cache/IgniteCacheStoreCollectionTest.java   |    4 +-
 .../IgniteCacheStoreValueAbstractTest.java      |    8 +-
 .../IgniteCacheTopologySafeGetSelfTest.java     |    4 +-
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |    4 +-
 .../cache/IgniteCachingProviderSelfTest.java    |    8 +-
 .../IgniteClientAffinityAssignmentSelfTest.java |    4 +-
 .../IgniteDaemonNodeMarshallerCacheTest.java    |    4 +-
 .../cache/IgniteDynamicCacheAndNodeStop.java    |    4 +-
 .../cache/IgniteDynamicCacheFilterTest.java     |    4 +-
 ...eDynamicCacheStartNoExchangeTimeoutTest.java |    6 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |    4 +-
 ...niteDynamicCacheStartStopConcurrentTest.java |    4 +-
 ...niteDynamicCacheWithConfigStartSelfTest.java |    4 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |    8 +-
 .../cache/IgniteInternalCacheTypesTest.java     |   17 +-
 ...iteMarshallerCacheClassNameConflictTest.java |  273 ++
 ...lerCacheClientRequestsMappingOnMissTest.java |  341 +++
 ...eMarshallerCacheConcurrentReadWriteTest.java |    4 +-
 .../cache/IgniteOnePhaseCommitNearSelfTest.java |   10 +-
 .../cache/IgnitePutAllLargeBatchSelfTest.java   |   10 +-
 ...tAllUpdateNonPreloadedPartitionSelfTest.java |   10 +-
 .../IgniteStartCacheInTransactionSelfTest.java  |    4 +-
 .../cache/IgniteStaticCacheStartSelfTest.java   |    4 +-
 .../cache/IgniteSystemCacheOnClientTest.java    |   29 +-
 ...gniteTopologyValidatorAbstractCacheTest.java |   10 +-
 ...niteTopologyValidatorGridSplitCacheTest.java |    2 +-
 .../processors/cache/IgniteTxAbstractTest.java  |    4 +-
 .../IgniteTxConcurrentGetAbstractTest.java      |    4 +-
 .../cache/IgniteTxConfigCacheSelfTest.java      |    8 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |   10 +-
 .../cache/IgniteTxMultiNodeAbstractTest.java    |    4 +-
 .../cache/IgniteTxReentryAbstractSelfTest.java  |    4 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |    8 +-
 .../MarshallerCacheJobRunNodeRestartTest.java   |    4 +-
 .../cache/OffHeapTieredTransactionSelfTest.java |    4 +-
 ...heapCacheMetricsForClusterGroupSelfTest.java |    4 +-
 .../cache/OffheapCacheOnClientsTest.java        |    4 +-
 ...sExchangeOnDiscoveryHistoryOverflowTest.java |    2 +-
 .../CacheKeepBinaryWithInterceptorTest.java     |    4 +-
 .../GridBinaryCacheEntryMemorySizeSelfTest.java |   21 +-
 ...ryDuplicateIndexObjectsAbstractSelfTest.java |    8 +-
 ...acheBinaryObjectUserClassloaderSelfTest.java |   10 +-
 ...naryObjectsAbstractDataStreamerSelfTest.java |    4 +-
 ...aryObjectsAbstractMultiThreadedSelfTest.java |    4 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |    4 +-
 .../GridCacheBinaryStoreAbstractSelfTest.java   |    4 +-
 ...yStoreBinariesSimpleNameMappersSelfTest.java |    4 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |    4 +-
 ...CacheClientNodeBinaryObjectMetadataTest.java |    6 +-
 .../DataStreamProcessorBinarySelfTest.java      |    4 +-
 .../GridDataStreamerImplSelfTest.java           |    4 +-
 .../GridCacheAffinityRoutingBinarySelfTest.java |    4 +-
 ...sNearPartitionedByteArrayValuesSelfTest.java |    4 +-
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |    4 +-
 .../dht/GridCacheMemoryModeBinarySelfTest.java  |    4 +-
 ...dCacheOffHeapTieredAtomicBinarySelfTest.java |    4 +-
 .../GridCacheOffHeapTieredBinarySelfTest.java   |    4 +-
 ...fHeapTieredEvictionAtomicBinarySelfTest.java |    4 +-
 ...acheOffHeapTieredEvictionBinarySelfTest.java |    4 +-
 ...IgniteCacheAbstractExecutionContextTest.java |    8 +-
 ...niteCacheContinuousExecutionContextTest.java |    4 +-
 ...IgniteCacheIsolatedExecutionContextTest.java |    4 +-
 ...niteCacheP2PDisableExecutionContextTest.java |    4 +-
 .../IgniteCachePrivateExecutionContextTest.java |    4 +-
 .../IgniteCacheSharedExecutionContextTest.java  |    4 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |   46 +-
 ...actQueueFailoverDataConsistencySelfTest.java |    4 +-
 .../GridCacheQueueCleanupSelfTest.java          |    4 +-
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |    6 +-
 .../GridCacheSequenceApiSelfAbstractTest.java   |    4 +-
 .../IgniteAtomicLongApiAbstractSelfTest.java    |    4 +-
 .../IgniteAtomicsAbstractTest.java              |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |    6 +-
 .../IgniteCollectionAbstractTest.java           |    4 +-
 .../IgniteDataStructureUniqueNameTest.java      |    4 +-
 .../IgniteDataStructureWithJobTest.java         |    4 +-
 .../IgniteSemaphoreAbstractSelfTest.java        |    2 +-
 ...SemaphoreFailoverSafeReleasePermitsTest.java |    4 +-
 ...achePartitionedAtomicSequenceTxSelfTest.java |    4 +-
 ...idCachePartitionedNodeRestartTxSelfTest.java |    4 +-
 ...PartitionedQueueCreateMultiNodeSelfTest.java |    4 +-
 .../CacheAbstractRestartSelfTest.java           |    6 +-
 .../distributed/CacheAffinityEarlyTest.java     |    4 +-
 ...acheAsyncOperationsFailoverAbstractTest.java |    8 +-
 .../distributed/CacheAsyncOperationsTest.java   |    6 +-
 .../CacheGetFutureHangsSelfTest.java            |    4 +-
 .../CacheGetInsideLockChangingTopologyTest.java |    4 +-
 ...ffinityAssignmentNodeJoinValidationTest.java |    4 +-
 .../CacheLateAffinityAssignmentTest.java        |   59 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java |   12 +-
 .../CacheLockReleaseNodeLeaveTest.java          |    4 +-
 .../CachePutAllFailoverAbstractTest.java        |    8 +-
 .../CacheTryLockMultithreadedTest.java          |    4 +-
 ...tractDistributedByteArrayValuesSelfTest.java |    4 +-
 .../GridCacheAbstractJobExecutionTest.java      |    8 +-
 .../GridCacheAbstractNodeRestartSelfTest.java   |    4 +-
 ...tractPartitionedByteArrayValuesSelfTest.java |    4 +-
 .../GridCacheAbstractPrimarySyncSelfTest.java   |    4 +-
 .../GridCacheBasicOpAbstractTest.java           |    4 +-
 .../GridCacheClientModesAbstractSelfTest.java   |   24 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |    4 +-
 ...acheEntrySetIterationPreloadingSelfTest.java |    4 +-
 .../distributed/GridCacheEventAbstractTest.java |    2 +-
 ...heExpiredEntriesPreloadAbstractSelfTest.java |    4 +-
 .../distributed/GridCacheLockAbstractTest.java  |    4 +-
 .../distributed/GridCacheMixedModeSelfTest.java |   12 +-
 .../GridCacheMultiNodeAbstractTest.java         |    4 +-
 .../GridCacheMultiNodeLockAbstractTest.java     |    4 +-
 ...dCacheMultithreadedFailoverAbstractTest.java |    8 +-
 .../GridCacheNodeFailureAbstractTest.java       |    4 +-
 ...ridCachePartitionNotLoadedEventSelfTest.java |    8 +-
 ...chePartitionedReloadAllAbstractSelfTest.java |    4 +-
 .../GridCachePreloadEventsAbstractSelfTest.java |    4 +-
 .../GridCachePreloadLifecycleAbstractTest.java  |    4 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |    4 +-
 .../GridCacheTransformEventSelfTest.java        |    4 +-
 ...yMetadataUpdateChangingTopologySelfTest.java |   11 +-
 ...niteBinaryMetadataUpdateNodeRestartTest.java |    4 +-
 .../distributed/IgniteCache150ClientsTest.java  |    6 +-
 ...micMessageRecoveryPairedConnectionsTest.java |    4 +-
 ...niteCacheClientNodeChangingTopologyTest.java |   22 +-
 .../IgniteCacheClientNodeConcurrentStart.java   |    6 +-
 ...teCacheClientNodePartitionsExchangeTest.java |    8 +-
 .../IgniteCacheClientReconnectTest.java         |    4 +-
 ...acheConnectionRecovery10ConnectionsTest.java |    4 +-
 .../IgniteCacheConnectionRecoveryTest.java      |    4 +-
 .../IgniteCacheCreatePutMultiNodeSelfTest.java  |    4 +-
 .../distributed/IgniteCacheCreatePutTest.java   |    4 +-
 .../distributed/IgniteCacheGetRestartTest.java  |    6 +-
 .../distributed/IgniteCacheManyClientsTest.java |    4 +-
 .../IgniteCacheMessageRecoveryAbstractTest.java |    6 +-
 ...eCacheMessageRecoveryIdleConnectionTest.java |    4 +-
 .../IgniteCacheMessageWriteTimeoutTest.java     |    4 +-
 .../IgniteCacheNearOffheapGetSelfTest.java      |   12 +-
 .../IgniteCacheNearRestartRollbackSelfTest.java |   12 +-
 .../IgniteCacheNodeJoinAbstractTest.java        |    4 +-
 .../distributed/IgniteCachePrimarySyncTest.java |    4 +-
 .../IgniteCachePutGetRestartAbstractTest.java   |   10 +-
 .../IgniteCacheReadFromBackupTest.java          |    4 +-
 .../IgniteCacheServerNodeConcurrentStart.java   |    4 +-
 .../IgniteCacheSingleGetMessageTest.java        |    4 +-
 .../IgniteCacheSizeFailoverTest.java            |    4 +-
 .../IgniteCacheSystemTransactionsSelfTest.java  |   29 +-
 .../IgniteCacheTxFairAffinityNodeJoinTest.java  |    4 +-
 .../IgniteCacheTxIteratorSelfTest.java          |    4 +-
 ...arDisabledFairAffinityPutGetRestartTest.java |    4 +-
 .../IgniteCrossCacheTxStoreSelfTest.java        |   16 +-
 .../IgniteTxCachePrimarySyncTest.java           |    9 +-
 ...teSynchronizationModesMultithreadedTest.java |    4 +-
 ...iteTxConsistencyRestartAbstractSelfTest.java |   10 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   14 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   12 +-
 .../IgniteTxPreloadAbstractTest.java            |    4 +-
 ...tPartitionedOnlyByteArrayValuesSelfTest.java |    4 +-
 ...heAbstractTransformWriteThroughSelfTest.java |    4 +-
 .../dht/GridCacheAtomicFullApiSelfTest.java     |    4 +-
 .../dht/GridCacheAtomicNearCacheSelfTest.java   |   41 +-
 .../dht/GridCacheColocatedDebugTest.java        |    4 +-
 ...eColocatedOptimisticTransactionSelfTest.java |    4 +-
 ...dCacheColocatedTxSingleThreadedSelfTest.java |    4 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |    4 +-
 .../dht/GridCacheDhtEntrySetSelfTest.java       |    4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |    4 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |    4 +-
 .../GridCacheDhtEvictionsDisabledSelfTest.java  |    4 +-
 .../dht/GridCacheDhtMappingSelfTest.java        |    4 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |    4 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |    9 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |    6 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |    4 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   10 +-
 .../dht/GridCacheDhtPreloadOffHeapSelfTest.java |    4 +-
 .../dht/GridCacheDhtPreloadPerformanceTest.java |    4 +-
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |    4 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   16 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |    8 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |    4 +-
 .../dht/GridCacheDhtTxPreloadSelfTest.java      |    4 +-
 .../dht/GridCacheGlobalLoadTest.java            |    4 +-
 ...ledFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...ePartitionedNearDisabledMetricsSelfTest.java |    8 +-
 ...idCachePartitionedPreloadEventsSelfTest.java |    4 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |    6 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |    4 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |    4 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   10 +-
 .../dht/GridNearCacheTxNodeFailureSelfTest.java |    4 +-
 ...gniteAtomicLongChangingTopologySelfTest.java |    4 +-
 .../IgniteCacheCommitDelayTxRecoveryTest.java   |    4 +-
 .../dht/IgniteCacheConcurrentPutGetRemove.java  |    4 +-
 .../IgniteCacheCrossCacheTxFailoverTest.java    |    6 +-
 .../dht/IgniteCacheLockFailoverSelfTest.java    |    8 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    4 +-
 ...artitionedBackupNodeFailureRecoveryTest.java |    4 +-
 ...woBackupsPrimaryNodeFailureRecoveryTest.java |    4 +-
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |   13 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |   41 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |    2 +-
 .../dht/IgniteCacheTxRecoveryRollbackTest.java  |  501 ++++
 .../dht/IgniteCrossCacheTxSelfTest.java         |    4 +-
 .../AtomicPutAllChangingTopologyTest.java       |    2 +-
 .../atomic/GridCacheAtomicFailoverSelfTest.java |    4 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |    6 +-
 .../atomic/GridCacheAtomicPreloadSelfTest.java  |    4 +-
 ...AtomicPrimaryWriteOrderFailoverSelfTest.java |    4 +-
 .../atomic/IgniteCacheAtomicProtocolTest.java   |  883 ++++++
 ...tionedMultiNodeLongTxTimeoutFullApiTest.java |    4 +-
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |    4 +-
 ...nlyFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...tomicClientOnlyMultiNodeFullApiSelfTest.java |    4 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 ...yOnReadDisabledMultiNodeFullApiSelfTest.java |    4 +-
 ...micFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...imaryWriteOrderMultiNodeFullApiSelfTest.java |    4 +-
 ...GridCacheAtomicMultiNodeFullApiSelfTest.java |    4 +-
 ...omicMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 ...ledFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java |    4 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCacheAtomicOffHeapFullApiSelfTest.java  |    4 +-
 ...heAtomicOffHeapMultiNodeFullApiSelfTest.java |    4 +-
 ...idCacheAtomicPartitionedMetricsSelfTest.java |    4 +-
 ...derFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...erNoStripedPoolMultiNodeFullApiSelfTest.java |   35 -
 .../near/GridCacheGetStoreErrorSelfTest.java    |    4 +-
 .../near/GridCacheNearEvictionSelfTest.java     |    4 +-
 .../near/GridCacheNearJobExecutionSelfTest.java |    4 +-
 .../near/GridCacheNearMetricsSelfTest.java      |    4 +-
 .../near/GridCacheNearMultiGetSelfTest.java     |    8 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |    4 +-
 ...nlyFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   10 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    4 +-
 .../GridCacheNearPartitionedClearSelfTest.java  |    4 +-
 .../GridCacheNearReaderPreloadSelfTest.java     |    6 +-
 .../near/GridCacheNearReadersSelfTest.java      |   21 +-
 .../near/GridCacheNearTxForceKeyTest.java       |    4 +-
 .../near/GridCacheNearTxMultiNodeSelfTest.java  |    4 +-
 .../near/GridCacheNearTxPreloadSelfTest.java    |    4 +-
 ...AffinityExcludeNeighborsPerformanceTest.java |    4 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |    4 +-
 .../GridCachePartitionedAffinitySelfTest.java   |   20 +-
 .../near/GridCachePartitionedBasicApiTest.java  |    4 +-
 .../GridCachePartitionedBasicOpSelfTest.java    |    4 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java |    4 +-
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    4 +-
 ...yOnReadDisabledMultiNodeFullApiSelfTest.java |    4 +-
 .../near/GridCachePartitionedEventSelfTest.java |    2 +-
 .../GridCachePartitionedEvictionSelfTest.java   |    4 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |    4 +-
 ...GridCachePartitionedFilteredPutSelfTest.java |    4 +-
 .../GridCachePartitionedFullApiSelfTest.java    |    8 +-
 ...idCachePartitionedHitsAndMissesSelfTest.java |    4 +-
 ...LateAffDisabledMultiNodeFullApiSelfTest.java |    4 +-
 .../GridCachePartitionedLoadCacheSelfTest.java  |    4 +-
 .../near/GridCachePartitionedLockSelfTest.java  |    4 +-
 .../GridCachePartitionedMetricsSelfTest.java    |    8 +-
 ...achePartitionedMultiNodeCounterSelfTest.java |   79 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   12 +-
 ...onedMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCachePartitionedMultiNodeSelfTest.java  |    4 +-
 ...ePartitionedMultiThreadedPutGetSelfTest.java |    4 +-
 .../near/GridCachePartitionedNestedTxTest.java  |    4 +-
 ...edNoStripedPoolMultiNodeFullApiSelfTest.java |   35 -
 ...GridCachePartitionedNodeFailureSelfTest.java |    4 +-
 .../GridCachePartitionedNodeRestartTest.java    |    4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |    4 +-
 ...achePartitionedPreloadLifecycleSelfTest.java |    4 +-
 .../GridCachePartitionedStorePutSelfTest.java   |    4 +-
 ...GridCachePartitionedTxConcurrentGetTest.java |    4 +-
 ...GridCachePartitionedTxMultiNodeSelfTest.java |    4 +-
 ...CachePartitionedTxMultiThreadedSelfTest.java |    4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   11 +-
 ...achePartitionedTxSingleThreadedSelfTest.java |    4 +-
 .../GridCachePartitionedTxTimeoutSelfTest.java  |    4 +-
 .../near/GridCachePutArrayValueSelfTest.java    |    4 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +-
 .../near/GridPartitionedBackupLoadSelfTest.java |    4 +-
 .../near/IgniteCacheNearOnlyTxTest.java         |    6 +-
 .../near/IgniteCacheNearTxRollbackTest.java     |    4 +-
 .../near/NearCacheMultithreadedUpdateTest.java  |    4 +-
 .../near/NearCachePutAllMultinodeTest.java      |    4 +-
 .../near/NearCacheSyncUpdateTest.java           |    4 +-
 .../near/NoneRebalanceModeSelfTest.java         |    4 +-
 ...cingDelayedPartitionMapExchangeSelfTest.java |    4 +-
 .../GridCacheRebalancingAsyncSelfTest.java      |    4 +-
 .../GridCacheRebalancingOrderingTest.java       |    6 +-
 .../GridCacheRebalancingSyncCheckDataTest.java  |    4 +-
 .../GridCacheRebalancingSyncSelfTest.java       |    6 +-
 ...eRebalancingUnmarshallingFailedSelfTest.java |    4 +-
 ...xcludeNeighborsMultiNodeFullApiSelfTest.java |    4 +-
 ...tedFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...xcludeNeighborsMultiNodeFullApiSelfTest.java |    4 +-
 ...dezvousAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...stractReplicatedByteArrayValuesSelfTest.java |    4 +-
 ...ridCacheAtomicReplicatedMetricsSelfTest.java |    4 +-
 ...nedFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...eplicatedAtomicMultiNodeFullApiSelfTest.java |    4 +-
 .../GridCacheReplicatedBasicApiTest.java        |    4 +-
 .../GridCacheReplicatedBasicOpSelfTest.java     |    4 +-
 .../GridCacheReplicatedEvictionSelfTest.java    |    4 +-
 .../GridCacheReplicatedFullApiSelfTest.java     |    8 +-
 .../GridCacheReplicatedJobExecutionTest.java    |    4 +-
 .../GridCacheReplicatedMarshallerTxTest.java    |    4 +-
 .../GridCacheReplicatedMetricsSelfTest.java     |    4 +-
 ...atedMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCacheReplicatedMultiNodeSelfTest.java   |    4 +-
 .../GridCacheReplicatedNodeFailureSelfTest.java |    4 +-
 .../GridCacheReplicatedTxConcurrentGetTest.java |    4 +-
 ...GridCacheReplicatedTxMultiNodeBasicTest.java |    4 +-
 ...dCacheReplicatedTxMultiThreadedSelfTest.java |    4 +-
 ...CacheReplicatedTxSingleThreadedSelfTest.java |    4 +-
 .../GridCacheReplicatedTxTimeoutSelfTest.java   |    4 +-
 ...idCacheReplicatedUnswapAdvancedSelfTest.java |    6 +-
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    4 +-
 .../replicated/GridReplicatedTxPreloadTest.java |    4 +-
 .../IgniteCacheSyncRebalanceModeSelfTest.java   |    4 +-
 ...CacheReplicatedPreloadLifecycleSelfTest.java |    4 +-
 ...idCacheReplicatedPreloadOffHeapSelfTest.java |    4 +-
 .../GridCacheReplicatedPreloadSelfTest.java     |   14 +-
 ...eplicatedPreloadStartStopEventsSelfTest.java |    4 +-
 .../cache/eviction/EvictionAbstractTest.java    |    4 +-
 .../GridCacheBatchEvictUnswapSelfTest.java      |    4 +-
 ...heConcurrentEvictionConsistencySelfTest.java |    4 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |    4 +-
 .../GridCacheDistributedEvictionsSelfTest.java  |    4 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |    4 +-
 .../GridCacheEvictionFilterSelfTest.java        |    4 +-
 .../GridCacheEvictionLockUnlockSelfTest.java    |    4 +-
 .../GridCacheEvictionTouchSelfTest.java         |    4 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |    4 +-
 .../lru/LruNearEvictionPolicySelfTest.java      |    4 +-
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |    4 +-
 .../SortedEvictionPolicyPerformanceTest.java    |    4 +-
 .../IgniteCacheClientNearCacheExpiryTest.java   |    6 +-
 .../IgniteCacheExpiryPolicyAbstractTest.java    |    4 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |    4 +-
 .../expiry/IgniteCacheTtlCleanupSelfTest.java   |    4 +-
 .../IgniteCacheLoadAllAbstractTest.java         |    4 +-
 ...iteCacheNoLoadPreviousValueAbstractTest.java |    8 +-
 .../IgniteCacheNoReadThroughAbstractTest.java   |    8 +-
 .../IgniteCacheNoWriteThroughAbstractTest.java  |    8 +-
 ...IgniteCacheStoreNodeRestartAbstractTest.java |    6 +-
 .../IgniteCacheStoreSessionAbstractTest.java    |    6 +-
 ...acheStoreSessionWriteBehindAbstractTest.java |    6 +-
 ...dCacheAtomicLocalMetricsNoStoreSelfTest.java |    4 +-
 .../GridCacheAtomicLocalMetricsSelfTest.java    |    4 +-
 .../local/GridCacheLocalBasicApiSelfTest.java   |    4 +-
 .../GridCacheLocalByteArrayValuesSelfTest.java  |    4 +-
 .../local/GridCacheLocalFullApiSelfTest.java    |    8 +-
 .../local/GridCacheLocalLoadAllSelfTest.java    |    4 +-
 .../local/GridCacheLocalMetricsSelfTest.java    |    8 +-
 .../GridCacheLocalTxMultiThreadedSelfTest.java  |    4 +-
 .../GridCacheLocalTxSingleThreadedSelfTest.java |    4 +-
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java |    4 +-
 .../BinaryTxCacheLocalEntriesSelfTest.java      |    8 +-
 .../GridCacheQueryTransformerSelfTest.java      |    4 +-
 .../GridCacheSwapScanQueryAbstractSelfTest.java |    4 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |    2 +-
 .../cache/query/IndexingSpiQueryTxSelfTest.java |    8 +-
 .../continuous/CacheContinuousBatchAckTest.java |   12 +-
 ...heContinuousBatchForceServerModeAckTest.java |    8 +-
 ...eContinuousQueryAsyncFilterListenerTest.java |    6 +-
 ...CacheContinuousQueryCounterAbstractTest.java |    8 +-
 ...acheContinuousQueryExecuteInPrimaryTest.java |    4 +-
 ...ContinuousQueryFailoverAbstractSelfTest.java |   35 +-
 .../CacheContinuousQueryLostPartitionTest.java  |    6 +-
 ...ontinuousQueryOperationFromCallbackTest.java |    4 +-
 .../CacheContinuousQueryOperationP2PTest.java   |    4 +-
 .../CacheContinuousQueryOrderingEventTest.java  |    4 +-
 ...acheContinuousQueryRandomOperationsTest.java |    4 +-
 .../CacheContinuousQueryVariationsTest.java     |    6 +-
 ...eEntryProcessorExternalizableFailedTest.java |    4 +-
 .../CacheEntryProcessorNonSerializableTest.java |    4 +-
 ...CacheKeepBinaryIterationSwapEnabledTest.java |    4 +-
 .../CacheKeepBinaryIterationTest.java           |    4 +-
 .../ClientReconnectContinuousQueryTest.java     |    2 +-
 ...yRemoteFilterMissingInClassPathSelfTest.java |    4 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   12 +-
 .../GridCacheContinuousQueryConcurrentTest.java |    6 +-
 ...eContinuousQueryMultiNodesFilteringTest.java |   10 +-
 ...eContinuousQueryReplicatedTxOneNodeTest.java |    4 +-
 ...niteCacheContinuousQueryBackupQueueTest.java |    4 +-
 ...CacheContinuousQueryClientReconnectTest.java |    4 +-
 .../IgniteCacheContinuousQueryClientTest.java   |    4 +-
 ...eCacheContinuousQueryImmutableEntryTest.java |    6 +-
 ...teCacheContinuousQueryNoUnsubscribeTest.java |    4 +-
 ...IgniteCacheContinuousQueryReconnectTest.java |    4 +-
 ...BehindStorePartitionedMultiNodeSelfTest.java |    4 +-
 .../IgniteCacheWriteBehindNoUpdateSelfTest.java |    4 +-
 ...CacheClientWriteBehindStoreAbstractTest.java |   12 +-
 .../TxDeadlockDetectionNoHangsTest.java         |    4 +-
 .../transactions/TxDeadlockDetectionTest.java   |    4 +-
 ...timisticDeadlockDetectionCrossCacheTest.java |    4 +-
 .../TxOptimisticDeadlockDetectionTest.java      |   34 +-
 ...simisticDeadlockDetectionCrossCacheTest.java |    4 +-
 .../TxPessimisticDeadlockDetectionTest.java     |    4 +-
 ...edEntryPartitionedAtomicOffHeapSelfTest.java |    4 +-
 ...PartitionedTransactionalOffHeapSelfTest.java |    4 +-
 ...nedEntryReplicatedAtomicOffHeapSelfTest.java |    4 +-
 ...yReplicatedTransactionalOffHeapSelfTest.java |    4 +-
 .../clock/GridTimeSyncProcessorSelfTest.java    |    4 +-
 .../closure/GridClosureProcessorRemoteTest.java |    2 +-
 .../closure/GridClosureProcessorSelfTest.java   |    4 +-
 .../closure/GridClosureSerializationTest.java   |    4 +-
 .../cluster/GridAddressResolverSelfTest.java    |    6 +-
 ...ComputeJobExecutionErrorToLogManualTest.java |    4 +-
 .../compute/PublicThreadpoolStarvationTest.java |    4 +-
 .../continuous/GridEventConsumeSelfTest.java    |    4 +-
 .../continuous/GridMessageListenSelfTest.java   |    4 +-
 .../IgniteNoCustomEventsOnNodeStart.java        |    4 +-
 .../DataStreamProcessorSelfTest.java            |  114 +-
 .../datastreamer/DataStreamerImplSelfTest.java  |    4 +-
 .../DataStreamerMultiThreadedSelfTest.java      |    4 +-
 .../DataStreamerMultinodeCreateCacheTest.java   |    4 +-
 .../datastreamer/DataStreamerTimeoutTest.java   |    4 +-
 .../DataStreamerUpdateAfterLoadTest.java        |    4 +-
 .../IgniteDataStreamerPerformanceTest.java      |    4 +-
 .../igfs/IgfsAbstractBaseSelfTest.java          |   23 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |   18 +-
 .../igfs/IgfsBackupFailoverSelfTest.java        |   20 +-
 ...lockMessageSystemPoolStarvationSelfTest.java |   23 +-
 ...sCachePerBlockLruEvictionPolicySelfTest.java |   22 +-
 .../processors/igfs/IgfsCacheSelfTest.java      |   23 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |    8 +-
 .../igfs/IgfsDataManagerSelfTest.java           |   37 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |   26 +-
 ...SecondaryFileSystemDualAbstractSelfTest.java |    5 -
 ...fsLocalSecondaryFileSystemProxySelfTest.java |   81 +
 ...IgfsLocalSecondaryFileSystemTestAdapter.java |    7 +-
 .../processors/igfs/IgfsMaxSizeSelfTest.java    |   12 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |   10 +-
 .../processors/igfs/IgfsMetricsSelfTest.java    |   20 +-
 .../processors/igfs/IgfsModesSelfTest.java      |   55 +-
 .../processors/igfs/IgfsOneClientNodeTest.java  |   17 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |   29 +-
 .../igfs/IgfsProcessorValidationSelfTest.java   |  324 +-
 ...gfsSecondaryFileSystemInjectionSelfTest.java |  268 ++
 ...IpcEndpointRegistrationAbstractSelfTest.java |   21 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |   31 +-
 .../processors/igfs/IgfsStartCacheTest.java     |   30 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |   50 +-
 .../processors/igfs/IgfsTaskSelfTest.java       |   10 +-
 .../IgfsAbstractRecordResolverSelfTest.java     |   10 +-
 ...niteMessagingConfigVariationFullApiTest.java |    6 +-
 .../odbc/OdbcProcessorValidationSelfTest.java   |    2 +-
 .../service/ClosureServiceClientsNodesTest.java |   10 +-
 .../service/GridServiceClientNodeTest.java      |    4 +-
 .../GridServiceContinuousQueryRedeploy.java     |  167 --
 .../GridServiceContinuousQueryRedeployTest.java |  167 ++
 .../GridServiceProcessorAbstractSelfTest.java   |    4 +-
 ...GridServiceProxyClientReconnectSelfTest.java |    6 +-
 .../GridServiceProxyNodeStopSelfTest.java       |    4 +-
 .../GridServiceSerializationSelfTest.java       |    6 +-
 ...gniteServiceConfigVariationsFullApiTest.java |    4 +-
 ...yment2ClassLoadersDefaultMarshallerTest.java |   18 +-
 ...oymentClassLoadingDefaultMarshallerTest.java |   16 +-
 .../IgniteServiceDynamicCachesSelfTest.java     |    4 +-
 ...gniteServiceProxyTimeoutInitializedTest.java |    6 +-
 .../service/IgniteServiceReassignmentTest.java  |    4 +-
 .../ServicePredicateAccessCacheTest.java        |    4 +-
 .../internal/util/IgniteUtilsSelfTest.java      |   66 +
 .../nio/GridNioEmbeddedFutureSelfTest.java      |    2 +-
 .../util/future/nio/GridNioFutureSelfTest.java  |   25 +-
 .../ipc/shmem/IpcSharedMemoryNodeStartup.java   |   30 +-
 .../ipc/shmem/IpcSharedMemorySpaceSelfTest.java |    6 +-
 .../ipc/shmem/IpcSharedMemoryUtilsSelfTest.java |    2 +-
 .../internal/util/nio/GridNioSelfTest.java      |    2 +-
 .../internal/util/nio/GridNioSslSelfTest.java   |    2 +-
 .../nio/IgniteExceptionInNioWorkerSelfTest.java |    6 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |   12 +-
 .../loadtests/GridCacheMultiNodeLoadTest.java   |    4 +-
 .../communication/GridIoManagerBenchmark.java   |    4 +-
 .../communication/GridIoManagerBenchmark0.java  |   18 +-
 .../GridTcpCommunicationBenchmark.java          |    2 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |    4 +-
 .../multisplit/GridMultiSplitsLoadTest.java     |    4 +-
 .../GridMultiSplitsRedeployLoadTest.java        |    2 +-
 ...ridSingleSplitsNewNodesAbstractLoadTest.java |    8 +-
 .../GridSingleSplitsRedeployLoadTest.java       |    2 +-
 .../direct/session/GridSessionLoadTest.java     |    2 +-
 .../direct/stealing/GridStealingLoadTest.java   |    4 +-
 .../ignite/loadtests/job/GridJobLoadTest.java   |    6 +-
 .../loadtests/nio/GridNioBenchmarkTest.java     |    2 +-
 ...namicProxySerializationMultiJvmSelfTest.java |    4 +-
 .../marshaller/GridMarshallerAbstractTest.java  |   15 +-
 .../marshaller/MarshallerContextSelfTest.java   |  150 +-
 .../marshaller/MarshallerContextTestImpl.java   |   33 +-
 .../OptimizedMarshallerEnumSelfTest.java        |   25 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    4 +-
 .../ignite/messaging/GridMessagingSelfTest.java |    4 +-
 .../IgniteMessagingWithClientTest.java          |    6 +-
 .../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     |    4 +-
 .../GridP2PMissedResourceCacheSizeSelfTest.java |    4 +-
 .../ignite/p2p/GridP2PNodeLeftSelfTest.java     |    4 +-
 .../p2p/GridP2PRecursionTaskSelfTest.java       |    4 +-
 .../p2p/GridP2PRemoteClassLoadersSelfTest.java  |    4 +-
 .../p2p/GridP2PSameClassLoaderSelfTest.java     |    4 +-
 .../ignite/p2p/GridP2PTimeoutSelfTest.java      |    4 +-
 .../ignite/p2p/GridP2PUndeploySelfTest.java     |    6 +-
 .../platform/PlatformComputeEchoTask.java       |   18 +
 .../PlatformJavaObjectFactoryProxySelfTest.java |    2 +-
 .../ignite/platform/PlatformStopIgniteTask.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      |   97 +
 .../plugin/PlatformTestPluginTarget.java        |  231 ++
 .../cache/PlatformGetCachePluginsTask.java      |   85 +
 .../PlatformTestCachePluginConfiguration.java   |   60 +
 ...formTestCachePluginConfigurationClosure.java |   48 +
 ...tCachePluginConfigurationClosureFactory.java |   37 +
 .../cache/PlatformTestCachePluginProvider.java  |   73 +
 ...SessionCancelSiblingsFromFutureSelfTest.java |    6 +-
 ...ridSessionCancelSiblingsFromJobSelfTest.java |    6 +-
 ...idSessionCancelSiblingsFromTaskSelfTest.java |    6 +-
 .../GridSessionCheckpointAbstractSelfTest.java  |    2 +-
 .../GridSessionCollisionSpiSelfTest.java        |    2 +-
 ...idSessionFutureWaitJobAttributeSelfTest.java |    6 +-
 ...dSessionFutureWaitTaskAttributeSelfTest.java |    6 +-
 .../session/GridSessionJobFailoverSelfTest.java |    4 +-
 ...GridSessionJobWaitTaskAttributeSelfTest.java |    6 +-
 .../ignite/session/GridSessionLoadSelfTest.java |    4 +-
 .../GridSessionSetFutureAttributeSelfTest.java  |    6 +-
 ...nSetFutureAttributeWaitListenerSelfTest.java |    6 +-
 .../GridSessionSetJobAttributeSelfTest.java     |    6 +-
 ...sionSetJobAttributeWaitListenerSelfTest.java |    6 +-
 .../GridSessionSetTaskAttributeSelfTest.java    |    6 +-
 ...GridSessionTaskWaitJobAttributeSelfTest.java |    6 +-
 .../GridSessionWaitAttributeSelfTest.java       |    8 +-
 .../spi/GridSpiStartStopAbstractTest.java       |    2 +-
 .../spi/GridTcpSpiForwardingSelfTest.java       |   10 +-
 .../CacheCheckpointSpiSecondCacheSelfTest.java  |    4 +-
 .../GridAbstractCommunicationSelfTest.java      |    4 +-
 .../communication/GridCacheMessageSelfTest.java |    6 +-
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |    4 +-
 .../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 +-
 .../discovery/AbstractDiscoverySelfTest.java    |    9 +-
 ...gniteClientReconnectMassiveShutdownTest.java |    4 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |    6 +-
 .../tcp/TcpClientDiscoverySpiMulticastTest.java |    4 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |   16 +-
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |    4 +-
 .../TcpDiscoveryMarshallerCheckSelfTest.java    |    4 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |    8 +-
 ...DiscoveryNodeConfigConsistentIdSelfTest.java |    6 +-
 .../TcpDiscoveryNodeConsistentIdSelfTest.java   |    4 +-
 .../discovery/tcp/TcpDiscoveryRestartTest.java  |    4 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  139 +-
 .../tcp/TcpDiscoverySnapshotHistoryTest.java    |    4 +-
 .../tcp/TcpDiscoverySpiConfigSelfTest.java      |    2 +-
 .../tcp/TcpDiscoverySpiStartStopSelfTest.java   |   11 +-
 .../tcp/TcpDiscoverySpiWildcardSelfTest.java    |    4 +-
 .../discovery/tcp/TcpDiscoverySslSelfTest.java  |    4 +-
 .../vm/TcpDiscoveryVmIpFinderSelfTest.java      |    2 +-
 .../spi/failover/GridFailoverTestContext.java   |    7 +-
 .../GridInternalTasksLoadBalancingSelfTest.java |    6 +-
 .../GridSwapSpaceSpiConsistencySelfTest.java    |   12 +-
 .../file/GridFileSwapSpaceSpiSelfTest.java      |    2 +-
 .../inmemory/GridTestSwapSpaceSpi.java          |    2 +-
 .../noop/GridNoopSwapSpaceSpiSelfTest.java      |    4 +-
 .../ignite/startup/GridVmNodesStarter.java      |    4 +-
 .../stream/socket/SocketStreamerSelfTest.java   |    4 +-
 .../ignite/testframework/GridTestNode.java      |    2 +-
 .../ignite/testframework/GridTestUtils.java     |   10 +-
 .../configvariations/ConfigFactory.java         |    8 +-
 .../ConfigVariationsFactory.java                |    6 +-
 .../testframework/junits/GridAbstractTest.java  |  213 +-
 .../junits/GridTestKernalContext.java           |   12 +-
 ...IgniteCacheConfigVariationsAbstractTest.java |   12 +-
 .../IgniteConfigVariationsAbstractTest.java     |   12 +-
 .../junits/common/GridAbstractExamplesTest.java |    2 +-
 .../junits/common/GridCommonAbstractTest.java   |   15 +-
 .../common/GridRollingRestartAbstractTest.java  |    6 +-
 .../junits/multijvm/IgniteNodeRunner.java       |   55 +-
 .../junits/multijvm/IgniteProcessProxy.java     |  109 +-
 .../junits/spi/GridSpiAbstractConfigTest.java   |    2 +-
 .../junits/spi/GridSpiAbstractTest.java         |   13 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |    5 +
 .../IgniteBinaryObjectsTestSuite.java           |    2 +
 .../IgniteCacheFullApiSelfTestSuite.java        |    8 +-
 .../IgniteCacheRestartTestSuite2.java           |    3 +-
 .../testsuites/IgniteCacheTestSuite5.java       |    3 +
 .../IgniteCacheTxRecoverySelfTestSuite.java     |    3 +
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |    3 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    4 +-
 .../ignite/thread/IgniteThreadPoolSizeTest.java |    8 -
 .../util/AttributeNodeFilterSelfTest.java       |    4 +-
 .../apache/ignite/util/GridRandomSelfTest.java  |   17 -
 .../util/GridTopologyHeapSizeSelfTest.java      |    4 +-
 .../h2/GridBinaryH2IndexingGeoSelfTest.java     |    4 +-
 modules/hadoop/pom.xml                          |    7 +
 .../fs/IgniteHadoopFileSystemCounterWriter.java |    6 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |   23 +-
 .../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         |   38 +-
 ...doopFileSystemCounterWriterDelegateImpl.java |    4 +-
 ...doopIgfsSecondaryFileSystemDelegateImpl.java |   61 +-
 .../hadoop/impl/igfs/HadoopIgfsInProc.java      |  190 +-
 .../hadoop/impl/igfs/HadoopIgfsOutProc.java     |   24 +-
 .../hadoop/impl/igfs/HadoopIgfsUtils.java       |    6 +
 .../hadoop/impl/igfs/HadoopIgfsWrapper.java     |  104 +-
 .../hadoop/impl/v1/HadoopV1MapTask.java         |    6 +-
 .../hadoop/impl/v1/HadoopV1ReduceTask.java      |    4 +-
 .../hadoop/impl/v1/HadoopV1Splitter.java        |    2 +-
 .../hadoop/impl/v2/HadoopV2Context.java         |    3 +-
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |   25 +-
 .../hadoop/impl/v2/HadoopV2Splitter.java        |    2 +-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |    6 +-
 .../hadoop/jobtracker/HadoopJobMetadata.java    |    4 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |   32 +-
 .../planner/HadoopAbstractMapReducePlanner.java |  116 -
 .../planner/HadoopDefaultMapReducePlan.java     |    4 +-
 .../hadoop/shuffle/HadoopShuffle.java           |    9 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |   13 +-
 .../shuffle/direct/HadoopDirectDataInput.java   |   75 +-
 .../shuffle/streams/HadoopDataInStream.java     |   34 +-
 .../shuffle/streams/HadoopOffheapBuffer.java    |   18 +
 .../HadoopEmbeddedTaskExecutor.java             |   10 +-
 .../taskexecutor/HadoopExecutorService.java     |   10 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |    6 +-
 .../taskexecutor/HadoopTaskExecutorAdapter.java |    8 +-
 .../external/HadoopExternalTaskExecutor.java    |   18 +-
 .../child/HadoopChildProcessRunner.java         |    4 +-
 .../HadoopExternalCommunication.java            |   29 +-
 .../communication/HadoopIpcToNioAdapter.java    |    7 +-
 .../communication/HadoopMarshallerFilter.java   |   10 +-
 .../resources/META-INF/classnames.properties    |    4 +-
 .../test/config/hadoop-fs-open-test/grid-0.xml  |  126 +
 .../test/config/hadoop-fs-open-test/grid-1.xml  |  126 +
 .../test/config/hadoop-fs-open-test/grid-2.xml  |  126 +
 .../test/config/igfs-cli-config-dual-async.xml  |  135 +
 .../test/config/igfs-cli-config-dual-sync.xml   |  133 +
 .../src/test/config/igfs-cli-config-primary.xml |  124 +
 .../src/test/config/igfs-cli-config-proxy.xml   |  133 +
 .../impl/HadoopAbstractMapReduceTest.java       |   14 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |   31 +-
 .../hadoop/impl/HadoopCommandLineTest.java      |    4 +-
 .../HadoopDefaultMapReducePlannerSelfTest.java  |  619 ----
 .../hadoop/impl/HadoopGroupingTest.java         |    4 +-
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |    5 +-
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |    4 +-
 .../impl/HadoopNoHadoopMapReduceTest.java       |    4 +-
 .../hadoop/impl/HadoopPlannerMockJob.java       |   10 +-
 .../hadoop/impl/HadoopSortingExternalTest.java  |    8 +-
 .../hadoop/impl/HadoopSortingTest.java          |    4 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |   23 +-
 .../hadoop/impl/HadoopTasksAllVersionsTest.java |   16 +-
 .../hadoop/impl/HadoopTasksV1Test.java          |    4 +-
 .../hadoop/impl/HadoopTasksV2Test.java          |    4 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |    4 +-
 .../impl/HadoopTestRoundRobinMrPlanner.java     |   75 -
 .../hadoop/impl/HadoopTestTaskContext.java      |    6 +-
 .../hadoop/impl/HadoopV2JobSelfTest.java        |    6 +-
 .../hadoop/impl/HadoopValidationSelfTest.java   |    4 +-
 .../HadoopWeightedMapReducePlannerTest.java     |    6 +-
 .../HadoopClientProtocolEmbeddedSelfTest.java   |    4 +-
 ...opClientProtocolMultipleServersSelfTest.java |    4 +-
 .../client/HadoopClientProtocolSelfTest.java    |    2 +-
 .../impl/igfs/Hadoop1DualAbstractTest.java      |   13 +-
 .../impl/igfs/Hadoop1OverIgfsProxyTest.java     |   67 +
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   16 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   74 +-
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |    6 +-
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |    6 +-
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   20 +-
 ...oopSecondaryFileSystemConfigurationTest.java |   95 +-
 .../hadoop/impl/igfs/IgfsEventsTestSuite.java   |    4 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |   14 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  208 +-
 ...opFileSystemClientBasedAbstractSelfTest.java |  193 ++
 ...pFileSystemClientBasedDualAsyncSelfTest.java |   38 +
 ...opFileSystemClientBasedDualSyncSelfTest.java |   38 +
 ...niteHadoopFileSystemClientBasedOpenTest.java |  305 ++
 ...oopFileSystemClientBasedPrimarySelfTest.java |   38 +
 ...adoopFileSystemClientBasedProxySelfTest.java |   37 +
 .../IgniteHadoopFileSystemClientSelfTest.java   |   62 +-
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |  111 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   53 +-
 ...niteHadoopFileSystemLoggerStateSelfTest.java |   30 +-
 ...adoopFileSystemLoopbackAbstractSelfTest.java |    4 +-
 ...oopbackExternalToClientAbstractSelfTest.java |   61 +
 ...opbackExternalToClientDualAsyncSelfTest.java |   33 +
 ...oopbackExternalToClientDualSyncSelfTest.java |   33 +
 ...LoopbackExternalToClientPrimarySelfTest.java |   33 +
 ...emLoopbackExternalToClientProxySelfTest.java |   33 +
 ...condaryFileSystemInitializationSelfTest.java |   38 +-
 ...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/HadoopAbstractMapTest.java      |    4 +-
 .../streams/HadoopDataStreamSelfTest.java       |  177 +-
 .../HadoopExternalTaskExecutionSelfTest.java    |   16 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   25 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   17 +-
 .../HibernateReadWriteAccessStrategy.java       |   12 +-
 .../cache/hibernate/HibernateRegionFactory.java |   25 +-
 .../src/test/config/factory-cache1.xml          |    2 +-
 .../HibernateL2CacheConfigurationSelfTest.java  |   20 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   24 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |    4 +-
 .../query/h2/DmlStatementsProcessor.java        |  100 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   45 +-
 .../query/h2/dml/UpdatePlanBuilder.java         |  109 +-
 .../query/h2/opt/GridH2IndexBase.java           |   10 -
 .../processors/query/h2/opt/GridH2Table.java    |   33 -
 .../query/h2/opt/GridH2TreeIndex.java           |   25 -
 .../query/h2/opt/GridLuceneIndex.java           |    4 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |    8 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |    7 +-
 ...CacheAbstractQueryDetailMetricsSelfTest.java |    4 +-
 .../CacheAbstractQueryMetricsSelfTest.java      |    4 +-
 .../CacheBinaryKeyConcurrentQueryTest.java      |    4 +-
 .../cache/CacheIndexStreamerTest.java           |    4 +-
 .../cache/CacheIndexingOffheapCleanupTest.java  |   21 -
 .../CacheOffheapBatchIndexingBaseTest.java      |    4 +-
 .../cache/CacheQueryBuildValueTest.java         |    4 +-
 .../cache/CacheQueryNewClientSelfTest.java      |    4 +-
 .../CacheRandomOperationsMultithreadedTest.java |    4 +-
 ...CacheScanPartitionQueryFallbackSelfTest.java |    6 +-
 .../cache/CacheSqlQueryValueCopySelfTest.java   |    6 +-
 .../ClientReconnectAfterClusterRestartTest.java |    6 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |    4 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |    4 +-
 .../cache/GridCacheOffHeapSelfTest.java         |    4 +-
 .../GridCacheOffheapIndexEntryEvictTest.java    |    4 +-
 .../cache/GridCacheOffheapIndexGetSelfTest.java |    4 +-
 .../GridCacheQueryIndexDisabledSelfTest.java    |    4 +-
 .../GridCacheQuerySerializationSelfTest.java    |    4 +-
 .../cache/GridCacheQuerySimpleBenchmark.java    |    4 +-
 .../processors/cache/GridCacheSwapSelfTest.java |    4 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    4 +-
 .../IgniteBinaryObjectFieldsQuerySelfTest.java  |    6 +-
 .../IgniteBinaryObjectQueryArgumentsTest.java   |    4 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |   25 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |    8 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  103 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |    6 +-
 .../IgniteCacheCollocatedQuerySelfTest.java     |    4 +-
 ...acheConfigurationPrimitiveTypesSelfTest.java |    4 +-
 .../IgniteCacheCrossCacheJoinRandomTest.java    |    4 +-
 ...acheDistributedJoinCollocatedAndNotTest.java |    4 +-
 ...acheDistributedJoinCustomAffinityMapper.java |    4 +-
 .../IgniteCacheDistributedJoinNoIndexTest.java  |    4 +-
 ...ributedJoinPartitionedAndReplicatedTest.java |    4 +-
 ...CacheDistributedJoinQueryConditionsTest.java |    4 +-
 .../cache/IgniteCacheDistributedJoinTest.java   |    4 +-
 ...cheDuplicateEntityConfigurationSelfTest.java |    4 +-
 .../IgniteCacheFieldsQueryNoDataSelfTest.java   |    4 +-
 ...teCacheFullTextQueryNodeJoiningSelfTest.java |    4 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |   26 +-
 ...PartitionedAndReplicatedCollocationTest.java |    4 +-
 ...teCacheJoinPartitionedAndReplicatedTest.java |    4 +-
 ...IgniteCacheJoinQueryWithAffinityKeyTest.java |    4 +-
 .../cache/IgniteCacheLargeResultSelfTest.java   |    4 +-
 ...eLockPartitionOnAffinityRunAbstractTest.java |    4 +-
 ...PartitionOnAffinityRunAtomicCacheOpTest.java |    4 +-
 ...titionOnAffinityRunWithCollisionSpiTest.java |    6 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |   24 +
 .../cache/IgniteCacheNoClassQuerySelfTest.java  |    4 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java |    4 +-
 .../cache/IgniteCacheOffheapIndexScanTest.java  |    4 +-
 ...CacheOffheapTieredMultithreadedSelfTest.java |    4 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    4 +-
 ...hePartitionedQueryMultiThreadedSelfTest.java |    4 +-
 ...IgniteCachePrimitiveFieldsQuerySelfTest.java |    4 +-
 .../cache/IgniteCacheQueriesLoadTest1.java      |    4 +-
 .../IgniteCacheQueryH2IndexingLeakTest.java     |    4 +-
 .../cache/IgniteCacheQueryLoadSelfTest.java     |    8 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |    4 +-
 ...gniteCacheSqlQueryMultiThreadedSelfTest.java |    4 +-
 .../IgniteCacheStarvationOnRebalanceTest.java   |    4 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |   65 +-
 ...ClientReconnectCacheQueriesFailoverTest.java |    4 +-
 .../cache/IgniteClientReconnectQueriesTest.java |    4 +-
 .../cache/IgniteCrossCachesJoinsQueryTest.java  |    4 +-
 .../cache/IncorrectCacheTypeMetadataTest.java   |    4 +-
 .../cache/IncorrectQueryEntityTest.java         |    4 +-
 .../cache/SqlFieldsQuerySelfTest.java           |    4 +-
 ...lientQueryReplicatedNodeRestartSelfTest.java |   10 +-
 ...niteCacheDistributedQueryCancelSelfTest.java |    6 +-
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java |    6 +-
 ...artitionedFieldsQueryP2PEnabledSelfTest.java |    4 +-
 ...achePartitionedQueryP2PDisabledSelfTest.java |    4 +-
 ...cheQueryAbstractDistributedJoinSelfTest.java |    6 +-
 .../near/IgniteCacheQueryNodeFailTest.java      |    4 +-
 .../IgniteCacheQueryNodeRestartSelfTest.java    |    4 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |    4 +-
 ...ReplicatedFieldsQueryP2PEnabledSelfTest.java |    4 +-
 ...CacheReplicatedQueryP2PDisabledSelfTest.java |    4 +-
 ...eCacheLocalQueryCancelOrTimeoutSelfTest.java |    4 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |    4 +-
 .../query/IgniteSqlQueryDedicatedPoolTest.java  |  110 +
 .../query/IgniteSqlSplitterSelfTest.java        |    4 +-
 .../query/h2/GridH2IndexRebuildTest.java        |  261 --
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   35 +-
 .../query/h2/opt/GridH2TableSelfTest.java       |   54 -
 .../h2/sql/AbstractH2CompareQueryTest.java      |    4 +-
 .../query/h2/sql/GridQueryParsingTest.java      |    4 +-
 .../FetchingQueryCursorStressTest.java          |    2 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |    4 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    4 +-
 .../processors/cache/jta/CacheJtaManager.java   |    3 +-
 .../processors/cache/jta/CacheJtaResource.java  |    8 +-
 .../cache/AbstractCacheJtaSelfTest.java         |    8 +-
 ...CacheJtaConfigurationValidationSelfTest.java |    4 +-
 ...CacheJtaFactoryConfigValidationSelfTest.java |    4 +-
 .../cache/GridJtaLifecycleAwareSelfTest.java    |    4 +-
 .../log4j/GridLog4jCorrectFileNameTest.java     |    6 +-
 .../log4j2/GridLog4j2CorrectFileNameTest.java   |    6 +-
 .../log4j2/GridLog4j2InitializedTest.java       |    2 +-
 .../logger/log4j2/Log4j2LoggerSelfTest.java     |    6 +-
 .../log4j2/Log4j2LoggerVerboseModeSelfTest.java |    6 +-
 .../IgniteAbstractOsgiContextActivator.java     |    2 +-
 .../activators/BasicIgniteTestActivator.java    |    2 +-
 .../include/ignite/binary/binary_object.h       |   22 +-
 .../binary/include/ignite/binary/binary_type.h  |    8 +-
 .../include/ignite/impl/binary/binary_common.h  |    6 +
 .../ignite/impl/binary/binary_object_header.h   |    9 +-
 .../ignite/impl/binary/binary_object_impl.h     |   61 +-
 .../ignite/impl/binary/binary_reader_impl.h     |   29 +-
 .../ignite/impl/binary/binary_type_updater.h    |    2 +-
 .../binary/binary_array_identity_resolver.cpp   |    2 +-
 .../src/impl/binary/binary_object_header.cpp    |   26 +-
 .../src/impl/binary/binary_object_impl.cpp      |   61 +-
 .../src/impl/binary/binary_type_manager.cpp     |    2 +-
 modules/platforms/cpp/common/Makefile.am        |    1 +
 .../common/include/ignite/common/concurrent.h   |   29 +-
 .../include/ignite/common/platform_utils.h      |    6 +-
 .../cpp/common/include/ignite/ignite_error.h    |    2 +-
 .../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/dynamic_load_os.h |  133 +
 .../os/win/src/common/dynamic_load_os.cpp       |  115 +
 .../common/os/win/src/common/platform_utils.cpp |   26 +-
 .../cpp/common/project/vs/common.vcxproj        |    2 +
 .../common/project/vs/common.vcxproj.filters    |    6 +
 .../platforms/cpp/common/src/common/utils.cpp   |    3 +-
 .../platforms/cpp/common/src/ignite_error.cpp   |   68 +-
 modules/platforms/cpp/core-test/Makefile.am     |    5 +-
 .../core-test/include/ignite/binary_test_defs.h |  106 +-
 .../cpp/core-test/project/vs/core-test.vcxproj  |    2 +
 .../project/vs/core-test.vcxproj.filters        |    6 +
 .../cpp/core-test/src/binary_object_test.cpp    |    6 +-
 .../src/binary_reader_writer_raw_test.cpp       |   23 +-
 .../cpp/core-test/src/cache_invoke_test.cpp     |  553 ++++
 .../platforms/cpp/core-test/src/cache_test.cpp  |    5 +-
 .../cpp/core-test/src/cluster_test.cpp          |   98 +
 .../cpp/core-test/src/interop_memory_test.cpp   |    3 +-
 .../platforms/cpp/core-test/src/test_utils.cpp  |    6 +-
 modules/platforms/cpp/core/Makefile.am          |    1 +
 modules/platforms/cpp/core/include/Makefile.am  |   10 +-
 .../cpp/core/include/ignite/cache/cache.h       |  217 +-
 .../ignite/cache/cache_entry_processor.h        |  111 +
 .../include/ignite/cache/mutable_cache_entry.h  |  176 ++
 .../include/ignite/cache/query/query_argument.h |  134 -
 .../include/ignite/cache/query/query_cursor.h   |    2 +-
 .../core/include/ignite/cache/query/query_sql.h |   16 +-
 .../ignite/cache/query/query_sql_fields.h       |   16 +-
 .../platforms/cpp/core/include/ignite/ignite.h  |   18 +
 .../cpp/core/include/ignite/ignite_binding.h    |  119 +
 .../include/ignite/ignite_binding_context.h     |   88 +
 .../core/include/ignite/ignite_configuration.h  |    2 -
 .../impl/binary/binary_type_updater_impl.h      |    2 +-
 .../impl/cache/cache_entry_processor_holder.h   |  282 ++
 .../core/include/ignite/impl/cache/cache_impl.h |   71 +-
 .../ignite/impl/cache/query/query_argument.h    |  137 +
 .../ignite/impl/cluster/cluster_group_impl.h    |   77 +
 .../include/ignite/impl/ignite_binding_impl.h   |  121 +
 .../include/ignite/impl/ignite_environment.h    |   61 +-
 .../cpp/core/include/ignite/impl/ignite_impl.h  |   50 +-
 .../ignite/impl/interop/interop_target.h        |   39 +-
 .../core/include/ignite/impl/module_manager.h   |  131 +
 .../cpp/core/include/ignite/impl/operations.h   |   62 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   11 +-
 .../cpp/core/project/vs/core.vcxproj.filters    |   36 +-
 modules/platforms/cpp/core/src/ignite.cpp       |   12 +-
 modules/platforms/cpp/core/src/ignition.cpp     |  343 ++-
 .../impl/binary/binary_type_updater_impl.cpp    |    2 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   74 +-
 .../continuous/continuous_query_handle_impl.cpp |    2 +-
 .../core/src/impl/cache/query/query_impl.cpp    |    8 +-
 .../src/impl/cluster/cluster_group_impl.cpp     |   64 +
 .../cpp/core/src/impl/ignite_environment.cpp    |   92 +-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |   32 +-
 .../core/src/impl/interop/interop_target.cpp    |   51 +-
 .../src/impl/transactions/transactions_impl.cpp |   14 +-
 .../cpp/jni/include/ignite/jni/exports.h        |    2 +
 .../platforms/cpp/jni/include/ignite/jni/java.h |    6 +-
 .../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 +-
 modules/platforms/cpp/jni/project/vs/module.def |    2 +
 modules/platforms/cpp/jni/src/exports.cpp       |    8 +
 modules/platforms/cpp/jni/src/java.cpp          |   76 +-
 .../cpp/odbc/install/install_amd64.cmd          |   10 +-
 .../platforms/cpp/odbc/install/install_x86.cmd  |    6 +-
 modules/platforms/cpp/odbc/src/utility.cpp      |   17 +-
 .../Apache.Ignite.AspNet.Tests/App.config       |    6 +-
 .../IgniteOutputCacheProviderTest.cs            |    2 +-
 .../IgniteSessionStateStoreProviderTest.cs      |    2 +-
 .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs     |    4 +-
 .../Apache.Ignite.Core.Tests.csproj             |   10 +
 .../Binary/BinaryBuilderSelfTest.cs             |   22 +-
 .../Binary/BinaryReaderWriterTest.cs            |    8 +
 .../Cache/Affinity/AffinityFieldTest.cs         |    2 +-
 .../Cache/Affinity/AffinityFunctionTest.cs      |    4 +-
 .../Cache/Affinity/AffinityTest.cs              |    2 +-
 .../Cache/CacheAbstractTest.cs                  |    2 +-
 .../Cache/CacheConfigurationTest.cs             |   15 +-
 .../Cache/CacheDynamicStartTest.cs              |    2 +-
 .../Cache/CacheMetricsTest.cs                   |    2 +-
 .../Cache/CacheNearTest.cs                      |    4 +-
 .../Cache/Query/CacheLinqTest.cs                |    2 +-
 .../Cache/Query/CacheQueriesTest.cs             |    2 +-
 .../Continuous/ContinuousQueryAbstractTest.cs   |    4 +-
 .../Continuous/ContinuousQueryJavaFilterTest.cs |    2 +-
 .../Cache/Store/CacheStoreSessionTest.cs        |    2 +-
 .../Cache/Store/CacheStoreTest.cs               |    2 +-
 .../Compute/ComputeApiTest.cs                   |   87 +-
 .../Cache/Affinity/affinity-function2.xml       |    2 +-
 .../Config/Compute/compute-grid1.xml            |    2 +-
 .../Config/Compute/compute-grid2.xml            |    2 +-
 .../Config/Compute/compute-grid3.xml            |    2 +-
 .../Config/Lifecycle/lifecycle-beans.xml        |    2 +-
 .../Config/Lifecycle/lifecycle-no-beans.xml     |    2 +-
 .../Config/start-test-grid1.xml                 |    2 +-
 .../Config/start-test-grid2.xml                 |    2 +-
 .../ConsoleRedirectTest.cs                      |    4 +-
 .../Dataload/DataStreamerTest.cs                |    2 +-
 .../Dataload/DataStreamerTestTopologyChange.cs  |    2 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |   30 +-
 .../IgniteConfigurationSectionTest.cs           |    2 +-
 .../IgniteConfigurationSerializerTest.cs        |   44 +-
 .../IgniteConfigurationTest.cs                  |   23 +-
 .../Log/CustomLoggerTest.cs                     |    2 +-
 .../Cache/CacheJavaPluginConfiguration.cs       |   45 +
 .../Plugin/Cache/CacheJavaPluginTest.cs         |  113 +
 .../Plugin/Cache/CachePlugin.cs                 |  127 +
 .../Plugin/Cache/CachePluginConfiguration.cs    |   64 +
 .../Plugin/Cache/CachePluginTest.cs             |  218 ++
 .../Plugin/PluginTest.cs                        |  342 +++
 .../Plugin/TestIgnitePlugin.cs                  |   39 +
 .../Plugin/TestIgnitePluginConfiguration.cs     |   53 +
 .../Plugin/TestIgnitePluginException.cs         |   64 +
 .../Plugin/TestIgnitePluginProvider.cs          |  153 +
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |    2 +-
 .../SerializationTest.cs                        |    2 +-
 .../dotnet/Apache.Ignite.Core.Tests/app.config  |    4 +-
 .../Apache.Ignite.Core.Tests/custom_app.config  |    2 +-
 .../Apache.Ignite.Core.csproj                   |   20 +
 .../Cache/Configuration/CacheConfiguration.cs   |   46 +
 .../Cache/Configuration/QueryEntity.cs          |    2 +-
 .../Common/ExceptionFactory.cs                  |   31 +
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |   47 +-
 .../Apache.Ignite.Core/Common/JavaException.cs  |   62 +-
 .../Communication/Tcp/TcpCommunicationSpi.cs    |    2 +-
 .../Discovery/Tcp/TcpDiscoverySpi.cs            |    2 +-
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |    2 +-
 .../Apache.Ignite.Core/Events/EventBase.cs      |    4 +-
 .../Apache.Ignite.Core/Events/JobEvent.cs       |    4 +-
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |    2 +-
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |   10 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |   78 +-
 .../IgniteConfigurationSection.xsd              |   49 +-
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |    2 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |    2 +-
 .../Impl/Binary/BinaryReader.cs                 |    4 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |   16 +-
 .../Impl/Binary/BinaryUtils.cs                  |   55 +-
 .../Impl/Binary/BinaryWriter.cs                 |    6 +-
 .../Impl/Binary/Marshaller.cs                   |    4 +-
 .../Common/IgniteConfigurationXmlSerializer.cs  |   40 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   96 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |   26 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |    6 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |   99 +-
 .../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                |  100 +
 .../Impl/Plugin/PluginProcessor.cs              |  359 +++
 .../Impl/Plugin/PluginProviderProxy.cs          |   90 +
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |    6 +
 .../Impl/Unmanaged/UnmanagedCallbackOp.cs       |    9 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |   94 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   12 +
 .../Interop/IPlatformTarget.cs                  |  103 +
 .../Cache/CachePluginProviderTypeAttribute.cs   |   52 +
 .../Plugin/Cache/ICachePluginConfiguration.cs   |   66 +
 .../Plugin/Cache/ICachePluginContext.cs         |   47 +
 .../Plugin/Cache/ICachePluginProvider.cs        |   52 +
 .../Plugin/IPluginConfiguration.cs              |   61 +
 .../Apache.Ignite.Core/Plugin/IPluginContext.cs |   76 +
 .../Plugin/IPluginProvider.cs                   |   69 +
 .../Apache.Ignite.Core/Plugin/PluginCallback.cs |   29 +
 .../Plugin/PluginNotFoundException.cs           |   65 +
 .../Plugin/PluginProviderTypeAttribute.cs       |   52 +
 .../App.config                                  |    4 +-
 .../EntityFrameworkCacheInitializationTest.cs   |    2 +-
 .../EntityFrameworkCacheTest.cs                 |    2 +-
 .../IgniteDbConfiguration.cs                    |    4 +-
 .../Misc/ClientReconnectExample.cs              |    2 +-
 .../scala/org/apache/ignite/scalar/scalar.scala |   17 +-
 .../test/resources/spring-ping-pong-partner.xml |    2 +-
 .../scalar/tests/ScalarProjectionSpec.scala     |    4 +-
 .../schedule/IgniteScheduleProcessor.java       |    2 +-
 .../processors/schedule/ScheduleFutureImpl.java |    4 +-
 .../org/apache/ignite/spark/IgniteContext.scala |    2 +-
 .../spark/JavaEmbeddedIgniteRDDSelfTest.java    |   14 +-
 .../spark/JavaStandaloneIgniteRDDSelfTest.java  |    6 +-
 .../org/apache/ignite/spark/IgniteRDDSpec.scala |   14 +-
 .../org/apache/ignite/IgniteSpringBean.java     |    2 +-
 .../ignite/cache/spring/SpringCacheManager.java |   42 +-
 .../spring/SpringTransactionManager.java        |   50 +-
 .../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 +-
 .../spring/GridSpringCacheManagerSelfTest.java  |    6 +-
 .../SpringCacheManagerContextInjectionTest.java |    2 +-
 .../ignite/cache/spring/spring-caching.xml      |    2 +-
 .../store/jdbc/CachePojoStoreXmlSelfTest.java   |    4 +-
 .../ignite/internal/GridFactorySelfTest.java    |  154 +-
 .../GridSpringBeanSerializationSelfTest.java    |    2 +-
 .../internal/IgniteDynamicCacheConfigTest.java  |    4 +-
 .../GridSpringResourceInjectionSelfTest.java    |    4 +-
 .../GridTransformSpringInjectionSelfTest.java   |    2 +-
 .../p2p/GridP2PUserVersionChangeSelfTest.java   |    6 +-
 .../org/apache/ignite/spring-injection-test.xml |    2 +-
 .../spring/injection/spring-resource.tmpl.xml   |    2 +-
 .../org/apache/ignite/spring/sprint-exclude.xml |    2 -
 .../GridSpringTransactionManagerSelfTest.java   |    6 +-
 ...gTransactionManagerContextInjectionTest.java |    2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |    6 +-
 .../scanners/UriDeploymentScannerManager.java   |   12 +-
 .../GridTaskUriDeploymentDeadlockSelfTest.java  |    6 +-
 .../ignite/p2p/GridP2PDisabledSelfTest.java     |    4 +-
 .../config/VisorConfigurationCommand.scala      |    2 +-
 .../visor/commands/node/VisorNodeCommand.scala  |    6 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   10 +-
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |    4 +-
 .../commands/alert/VisorAlertCommandSpec.scala  |    4 +-
 .../cache/VisorCacheClearCommandSpec.scala      |    4 +-
 .../commands/cache/VisorCacheCommandSpec.scala  |    4 +-
 .../cache/VisorCacheResetCommandSpec.scala      |    4 +-
 .../config/VisorConfigurationCommandSpec.scala  |    4 +-
 .../cswap/VisorCacheSwapCommandSpec.scala       |    4 +-
 .../disco/VisorDiscoveryCommandSpec.scala       |    4 +-
 .../events/VisorEventsCommandSpec.scala         |    4 +-
 .../visor/commands/gc/VisorGcCommandSpec.scala  |    4 +-
 .../commands/tasks/VisorTasksCommandSpec.scala  |    4 +-
 modules/web-console/frontend/app/app.js         |   10 +-
 .../app/components/input-dialog/index.js        |   24 +
 .../input-dialog/input-dialog.controller.js     |   35 +
 .../input-dialog/input-dialog.service.js        |   88 +
 .../input-dialog/input-dialog.tpl.pug           |   39 +
 .../frontend/app/modules/sql/sql.controller.js  |   84 +-
 .../frontend/app/services/Clone.service.js      |   66 -
 .../frontend/controllers/caches-controller.js   |   10 +-
 .../frontend/controllers/clusters-controller.js |    6 +-
 .../frontend/controllers/domains-controller.js  |   12 +-
 .../frontend/controllers/igfs-controller.js     |   10 +-
 .../web-console/frontend/views/sql/sql.tpl.pug  |    9 +-
 .../frontend/views/templates/clone.tpl.pug      |   39 -
 .../frontend/views/templates/message.tpl.pug    |    4 +-
 .../ignite/console/demo/AgentClusterDemo.java   |    4 +-
 .../cache/websession/WebSessionFilter.java      |   12 +-
 .../servlet/ServletContextListenerStartup.java  |   12 +-
 .../ignite/startup/servlet/ServletStartup.java  |    8 +-
 .../internal/websession/WebSessionSelfTest.java |   21 +-
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |    2 +-
 .../cache/IgniteGetAndPutBenchmark.java         |    2 +-
 .../cache/IgniteGetAndPutTxBenchmark.java       |    2 +-
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  |   10 +-
 1928 files changed, 40484 insertions(+), 22726 deletions(-)
----------------------------------------------------------------------



[05/23] ignite git commit: IGNITE-4436 API for collecting list of running queries and cancel them. (cherry picked from commit 4923734)

Posted by vo...@apache.org.
IGNITE-4436 API for collecting list of running queries and cancel them.
(cherry picked from commit 4923734)


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

Branch: refs/heads/ignite-4565-ddl
Commit: 840ab997436dfad8cdbb4ad182de24bceec37962
Parents: 620235f
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Feb 14 20:54:31 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Mar 2 11:17:39 2017 +0700

----------------------------------------------------------------------
 .../processors/query/GridQueryIndexing.java     |  17 +-
 .../processors/query/GridQueryProcessor.java    |  32 ++-
 .../processors/query/GridRunningQueryInfo.java  | 132 ++++++++++++
 .../internal/visor/VisorMultiNodeTask.java      |   2 +-
 .../visor/query/VisorCancelQueriesTask.java     |  72 +++++++
 .../query/VisorCollectRunningQueriesTask.java   |  96 +++++++++
 .../internal/visor/query/VisorRunningQuery.java | 132 ++++++++++++
 .../cache/query/GridCacheTwoStepQuery.java      |  18 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  86 +++++++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  60 +++++-
 .../cache/CacheSqlQueryValueCopySelfTest.java   | 208 +++++++++++++++++--
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   2 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   7 +
 14 files changed, 826 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index ef39d96..ca04724 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -241,7 +241,22 @@ public interface GridQueryIndexing {
     public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException;
 
     /**
+     * Collect queries that already running more than specified duration.
+     *
+     * @param duration Duration to check.
+     * @return Collection of long running queries.
+     */
+    public Collection<GridRunningQueryInfo> runningQueries(long duration);
+
+    /**
+     * Cancel specified queries.
+     *
+     * @param queries Queries ID's to cancel.
+     */
+    public void cancelQueries(Collection<Long> queries);
+
+    /**
      * Cancels all executing queries.
      */
     public void cancelAllQueries();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index f8919a1..10bf75a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -44,7 +44,6 @@ import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.binary.BinaryField;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
@@ -119,7 +118,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     private static final int QRY_DETAIL_METRICS_EVICTION_FREQ = 3_000;
 
     /** */
-    private static Set<Class<?>> SQL_TYPES = new HashSet<>(F.<Class<?>>asList(
+    private static final Set<Class<?>> SQL_TYPES = new HashSet<>(F.<Class<?>>asList(
         Integer.class,
         Boolean.class,
         Byte.class,
@@ -920,6 +919,29 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Collect queries that already running more than specified duration.
+     *
+     * @param duration Duration to check.
+     * @return Collection of long running queries.
+     */
+    public Collection<GridRunningQueryInfo> runningQueries(long duration) {
+        if (moduleEnabled())
+            return idx.runningQueries(duration);
+
+        return Collections.emptyList();
+    }
+
+    /**
+     * Cancel specified queries.
+     *
+     * @param queries Queries ID's to cancel.
+     */
+    public void cancelQueries(Collection<Long> queries) {
+        if (moduleEnabled())
+            idx.cancelQueries(queries);
+    }
+
+    /**
      * @param sqlQry Sql query.
      * @param params Params.
      */
@@ -2722,7 +2744,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /** Accessor that deals with fields. */
-    private final static class FieldAccessor implements PropertyAccessor {
+    private static final class FieldAccessor implements PropertyAccessor {
         /** Field to access. */
         private final Field fld;
 
@@ -2765,7 +2787,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /** Getter and setter methods based accessor. */
-    private final static class MethodsAccessor implements PropertyAccessor {
+    private static final class MethodsAccessor implements PropertyAccessor {
         /** */
         private final Method getter;
 
@@ -2823,7 +2845,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /** Accessor with getter only. */
-    private final static class ReadOnlyMethodsAccessor implements PropertyAccessor {
+    private static final class ReadOnlyMethodsAccessor implements PropertyAccessor {
         /** */
         private final Method getter;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java
new file mode 100644
index 0000000..d77c8c0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query;
+
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+
+/**
+ * Query descriptor.
+ */
+public class GridRunningQueryInfo {
+    /** */
+    private final long id;
+
+    /** */
+    private final String qry;
+
+    /** Query type. */
+    private final GridCacheQueryType qryType;
+
+    /** */
+    private final String cache;
+
+    /** */
+    private final long startTime;
+
+    /** */
+    private final GridQueryCancel cancel;
+
+    /** */
+    private final boolean loc;
+
+    /**
+     * @param id Query ID.
+     * @param qry Query text.
+     * @param qryType Query type.
+     * @param cache Cache where query was executed.
+     * @param startTime Query start time.
+     * @param cancel Query cancel.
+     * @param loc Local query flag.
+     */
+    public GridRunningQueryInfo(Long id, String qry, GridCacheQueryType qryType, String cache, long startTime,
+        GridQueryCancel cancel, boolean loc) {
+        this.id = id;
+        this.qry = qry;
+        this.qryType = qryType;
+        this.cache = cache;
+        this.startTime = startTime;
+        this.cancel = cancel;
+        this.loc = loc;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public Long id() {
+        return id;
+    }
+
+    /**
+     * @return Query text.
+     */
+    public String query() {
+        return qry;
+    }
+
+    /**
+     * @return Query type.
+     */
+    public GridCacheQueryType queryType() {
+        return qryType;
+    }
+
+    /**
+     * @return Cache where query was executed.
+     */
+    public String cache() {
+        return cache;
+    }
+
+    /**
+     * @return Query start time.
+     */
+    public long startTime() {
+        return startTime;
+    }
+
+    /**
+     * @param curTime Current time.
+     * @param duration Duration of long query.
+     * @return {@code true} if this query should be considered as long running query.
+     */
+    public boolean longQuery(long curTime, long duration) {
+        return curTime - startTime > duration;
+    }
+
+    /**
+     * Cancel query.
+     */
+    public void cancel() {
+        if (cancel != null)
+            cancel.cancel();
+    }
+
+    /**
+     * @return {@code true} if query can be cancelled.
+     */
+    public boolean cancelable() {
+        return cancel != null;
+    }
+
+    /**
+     * @return {@code true} if query is local.
+     */
+    public boolean local() {
+        return loc;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
index 57f1346..ece1a17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
@@ -130,4 +130,4 @@ public abstract class VisorMultiNodeTask<A, R, J> implements ComputeTask<VisorTa
                 logFinish(ignite.log(), getClass(), start);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java
new file mode 100644
index 0000000..a6f2d82
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.query;
+
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorOneNodeTask;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Task to cancel queries.
+ */
+@GridInternal
+public class VisorCancelQueriesTask extends VisorOneNodeTask<Collection<Long>, Void> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorCancelQueriesJob job(Collection<Long> arg) {
+        return new VisorCancelQueriesJob(arg, debug);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override protected Void reduce0(List<ComputeJobResult> results) throws IgniteException {
+        return null;
+    }
+
+    /**
+     * Job to cancel queries on node.
+     */
+    private static class VisorCancelQueriesJob extends VisorJob<Collection<Long>, Void> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Flag indicating whether debug information should be printed into node log.
+         */
+        protected VisorCancelQueriesJob(@Nullable Collection<Long> arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Void run(@Nullable Collection<Long> queries) throws IgniteException {
+            ignite.context().query().cancelQueries(queries);
+
+            return null;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java
new file mode 100644
index 0000000..2b40e61
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java
@@ -0,0 +1,96 @@
+/*
+ * 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.visor.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorMultiNodeTask;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Task to collect currently running queries.
+ */
+@GridInternal
+public class VisorCollectRunningQueriesTask extends VisorMultiNodeTask<Long, Map<UUID, Collection<VisorRunningQuery>>, Collection<VisorRunningQuery>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorCollectRunningQueriesJob job(Long arg) {
+        return new VisorCollectRunningQueriesJob(arg, debug);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override protected Map<UUID, Collection<VisorRunningQuery>> reduce0(List<ComputeJobResult> results) throws IgniteException {
+        Map<UUID, Collection<VisorRunningQuery>> map = new HashMap<>();
+
+        for (ComputeJobResult res : results)
+            if (res.getException() == null) {
+                Collection<VisorRunningQuery> queries = res.getData();
+
+                map.put(res.getNode().id(), queries);
+            }
+
+        return map;
+    }
+
+    /**
+     * Job to collect currently running queries from node.
+     */
+    private static class VisorCollectRunningQueriesJob extends VisorJob<Long, Collection<VisorRunningQuery>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Flag indicating whether debug information should be printed into node log.
+         */
+        protected VisorCollectRunningQueriesJob(@Nullable Long arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Collection<VisorRunningQuery> run(@Nullable Long duration) throws IgniteException {
+            Collection<GridRunningQueryInfo> queries = ignite.context().query()
+                .runningQueries(duration != null ? duration : 0);
+
+            Collection<VisorRunningQuery> res = new ArrayList<>(queries.size());
+
+            long curTime = U.currentTimeMillis();
+
+            for (GridRunningQueryInfo qry : queries)
+                res.add(new VisorRunningQuery(qry.id(), qry.query(), qry.queryType(), qry.cache(),
+                    qry.startTime(), curTime - qry.startTime(),
+                    qry.cancelable(), qry.local()));
+
+            return res;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java
new file mode 100644
index 0000000..fc6bc7a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java
@@ -0,0 +1,132 @@
+/*
+ * 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.visor.query;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+
+/**
+ * Descriptor of running query.
+ */
+public class VisorRunningQuery implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private long id;
+
+    /** Query text. */
+    private String qry;
+
+    /** Query type. */
+    private GridCacheQueryType qryType;
+
+    /** Cache name for query. */
+    private String cache;
+
+    /** */
+    private long startTime;
+
+    /** */
+    private long duration;
+
+    /** */
+    private boolean cancellable;
+
+    /** */
+    private boolean loc;
+
+    /**
+     * @param id Query ID.
+     * @param qry Query text.
+     * @param qryType Query type.
+     * @param cache Cache where query was executed.
+     * @param startTime Query start time.
+     * @param duration Query current duration.
+     * @param cancellable {@code true} if query can be canceled.
+     * @param loc {@code true} if query is local.
+     */
+    public VisorRunningQuery(long id, String qry, GridCacheQueryType qryType, String cache,
+        long startTime, long duration,
+        boolean cancellable, boolean loc) {
+        this.id = id;
+        this.qry = qry;
+        this.qryType = qryType;
+        this.cache = cache;
+        this.startTime = startTime;
+        this.duration = duration;
+        this.cancellable = cancellable;
+        this.loc = loc;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public long getId() {
+        return id;
+    }
+
+    /**
+     * @return Query txt.
+     */
+    public String getQuery() {
+        return qry;
+    }
+
+    /**
+     * @return Query type.
+     */
+    public GridCacheQueryType getQueryType() {
+        return qryType;
+    }
+
+    /**
+     * @return Cache name.
+     */
+    public String getCache() {
+        return cache;
+    }
+
+    /**
+     * @return Query start time.
+     */
+    public long getStartTime() {
+        return startTime;
+    }
+
+    /**
+     * @return Query duration.
+     */
+    public long getDuration() {
+        return duration;
+    }
+
+    /**
+     * @return {@code true} if query can be cancelled.
+     */
+    public boolean isCancelable() {
+        return cancellable;
+    }
+
+    /**
+     * @return {@code true} if query is local.
+     */
+    public boolean isLocal() {
+        return loc;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
index 8dcba2f..f53936f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
@@ -46,6 +46,9 @@ public class GridCacheTwoStepQuery {
     private boolean explain;
 
     /** */
+    private String originalSql;
+
+    /** */
     private Collection<String> spaces;
 
     /** */
@@ -67,10 +70,12 @@ public class GridCacheTwoStepQuery {
     private List<Integer> extraCaches;
 
     /**
+     * @param originalSql Original query SQL.
      * @param schemas Schema names in query.
      * @param tbls Tables in query.
      */
-    public GridCacheTwoStepQuery(Set<String> schemas, Set<String> tbls) {
+    public GridCacheTwoStepQuery(String originalSql, Set<String> schemas, Set<String> tbls) {
+        this.originalSql = originalSql;
         this.schemas = schemas;
         this.tbls = tbls;
     }
@@ -196,6 +201,13 @@ public class GridCacheTwoStepQuery {
     }
 
     /**
+     * @return Original query SQL.
+     */
+    public String originalSql() {
+        return originalSql;
+    }
+
+    /**
      * @return Spaces.
      */
     public Collection<String> spaces() {
@@ -223,7 +235,7 @@ public class GridCacheTwoStepQuery {
     public GridCacheTwoStepQuery copy(Object[] args) {
         assert !explain;
 
-        GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(schemas, tbls);
+        GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(originalSql, schemas, tbls);
 
         cp.caches = caches;
         cp.extraCaches = extraCaches;
@@ -250,4 +262,4 @@ public class GridCacheTwoStepQuery {
     @Override public String toString() {
         return S.toString(GridCacheTwoStepQuery.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/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 e375867..8c5c2a3 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
@@ -52,6 +52,7 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
@@ -81,6 +82,7 @@ import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
@@ -176,7 +178,11 @@ import org.jsr166.ConcurrentHashMap8;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT;
+import static org.apache.ignite.IgniteSystemProperties.getInteger;
 import static org.apache.ignite.IgniteSystemProperties.getString;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.GEO_SPATIAL;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.SORTED;
@@ -284,9 +290,15 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private final Map<String, String> space2schema = new ConcurrentHashMap8<>();
 
     /** */
+    private AtomicLong qryIdGen;
+
+    /** */
     private GridSpinBusyLock busyLock;
 
     /** */
+    private final ConcurrentMap<Long, GridRunningQueryInfo> runs = new ConcurrentHashMap8<>();
+
+    /** */
     private final ThreadLocal<ConnectionWrapper> connCache = new ThreadLocal<ConnectionWrapper>() {
         @Nullable @Override public ConnectionWrapper get() {
             ConnectionWrapper c = super.get();
@@ -771,8 +783,19 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         IndexingQueryFilter filters) throws IgniteCheckedException {
         TableDescriptor tbl = tableDescriptor(spaceName, type);
 
-        if (tbl != null && tbl.luceneIdx != null)
-            return tbl.luceneIdx.query(qry, filters);
+        if (tbl != null && tbl.luceneIdx != null) {
+            GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, spaceName,
+                U.currentTimeMillis(), null, true);
+
+            try {
+                runs.put(run.id(), run);
+
+                return tbl.luceneIdx.query(qry, filters);
+            }
+            finally {
+                runs.remove(run.id());
+            }
+        }
 
         return new GridEmptyCloseableIterator<>();
     }
@@ -830,6 +853,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
                 GridH2QueryContext.set(ctx);
 
+                GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL_FIELDS,
+                    spaceName, U.currentTimeMillis(), cancel, true);
+
+                runs.putIfAbsent(run.id(), run);
+
                 try {
                     ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel);
 
@@ -837,6 +865,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 }
                 finally {
                     GridH2QueryContext.clearThreadLocal();
+
+                    runs.remove(run.id());
                 }
             }
         };
@@ -1086,6 +1116,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter).distributedJoins(false));
 
+        GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL, spaceName,
+            U.currentTimeMillis(), null, true);
+
+        runs.put(run.id(), run);
+
         try {
             ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true, 0, null);
 
@@ -1093,6 +1128,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
         finally {
             GridH2QueryContext.clearThreadLocal();
+
+            runs.remove(run.id());
         }
     }
 
@@ -1232,7 +1269,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                             try {
                                 ctx.cache().createMissingCaches();
                             }
-                            catch (IgniteCheckedException e1) {
+                            catch (IgniteCheckedException ignored) {
                                 throw new CacheException("Failed to create missing caches.", e);
                             }
 
@@ -1737,6 +1774,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         this.busyLock = busyLock;
 
+        qryIdGen = new AtomicLong();
+
         if (SysProperties.serializeJavaObject) {
             U.warn(log, "Serialization of Java objects in H2 was enabled.");
 
@@ -1787,7 +1826,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             marshaller = ctx.config().getMarshaller();
 
             mapQryExec = new GridMapQueryExecutor(busyLock);
-            rdcQryExec = new GridReduceQueryExecutor(busyLock);
+            rdcQryExec = new GridReduceQueryExecutor(qryIdGen, busyLock);
 
             mapQryExec.start(ctx, this);
             rdcQryExec.start(ctx, this);
@@ -2241,6 +2280,37 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         return cols;
     }
 
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRunningQueryInfo> runningQueries(long duration) {
+        Collection<GridRunningQueryInfo> res = new ArrayList<>();
+
+        res.addAll(runs.values());
+        res.addAll(rdcQryExec.longRunningQueries(duration));
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancelQueries(Collection<Long> queries) {
+        if (!F.isEmpty(queries)) {
+            for (Long qryId : queries) {
+                GridRunningQueryInfo run = runs.get(qryId);
+
+                if (run != null)
+                    run.cancel();
+            }
+
+            rdcQryExec.cancelQueries(queries);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancelAllQueries() {
+        for (Connection conn : conns)
+            U.close(conn, log);
+    }
+
     /**
      * Wrapper to store connection and flag is schema set or not.
      */
@@ -3151,10 +3221,4 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             lastUsage = U.currentTimeMillis();
         }
     }
-
-    /** {@inheritDoc} */
-    @Override public void cancelAllQueries() {
-        for (Connection conn : conns)
-            U.close(conn, log);
-    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 09952cf..e164315 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -174,7 +174,7 @@ public class GridSqlQuerySplitter {
         qry = collectAllTables(qry, schemas, tbls);
 
         // Build resulting two step query.
-        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(schemas, tbls);
+        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(qry.getSQL(), schemas, tbls);
 
         // Map query will be direct reference to the original query AST.
         // Thus all the modifications will be performed on the original AST, so we should be careful when
@@ -958,4 +958,4 @@ public class GridSqlQuerySplitter {
     private static GridSqlFunction function(GridSqlFunctionType type) {
         return new GridSqlFunction(type);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 1f00ed2..ee9976c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -62,6 +62,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
+import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
@@ -98,6 +99,7 @@ import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
 import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.REDUCE;
 
 /**
@@ -120,7 +122,7 @@ public class GridReduceQueryExecutor {
     private IgniteLogger log;
 
     /** */
-    private final AtomicLong reqIdGen = new AtomicLong();
+    private final AtomicLong qryIdGen;
 
     /** */
     private final ConcurrentMap<Long, QueryRun> runs = new ConcurrentHashMap8<>();
@@ -167,9 +169,11 @@ public class GridReduceQueryExecutor {
     };
 
     /**
+     * @param qryIdGen Query ID generator.
      * @param busyLock Busy lock.
      */
-    public GridReduceQueryExecutor(GridSpinBusyLock busyLock) {
+    public GridReduceQueryExecutor(AtomicLong qryIdGen, GridSpinBusyLock busyLock) {
+        this.qryIdGen = qryIdGen;
         this.busyLock = busyLock;
     }
 
@@ -493,11 +497,13 @@ public class GridReduceQueryExecutor {
                 }
             }
 
-            final long qryReqId = reqIdGen.incrementAndGet();
+            final long qryReqId = qryIdGen.incrementAndGet();
 
             final String space = cctx.name();
 
-            final QueryRun r = new QueryRun(h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize());
+            final QueryRun r = new QueryRun(qryReqId, qry.originalSql(), space,
+                h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize(),
+                U.currentTimeMillis(), cancel);
 
             AffinityTopologyVersion topVer = h2.readyTopologyVersion();
 
@@ -1303,10 +1309,46 @@ public class GridReduceQueryExecutor {
     }
 
     /**
+     * Collect queries that already running more than specified duration.
+     *
+     * @param duration Duration to check.
+     * @return Collection of IDs and statements of long running queries.
+     */
+    public Collection<GridRunningQueryInfo> longRunningQueries(long duration) {
+        Collection<GridRunningQueryInfo> res = new ArrayList<>();
+
+        long curTime = U.currentTimeMillis();
+
+        for (QueryRun run : runs.values()) {
+            if (run.qry.longQuery(curTime, duration))
+                res.add(run.qry);
+        }
+
+        return res;
+    }
+
+    /**
+     * Cancel specified queries.
+     *
+     * @param queries Queries IDs to cancel.
+     */
+    public void cancelQueries(Collection<Long> queries) {
+        for (Long qryId : queries) {
+            QueryRun run = runs.get(qryId);
+
+            if (run != null)
+                run.qry.cancel();
+        }
+    }
+
+    /**
      * Query run.
      */
     private static class QueryRun {
         /** */
+        private final GridRunningQueryInfo qry;
+
+        /** */
         private final List<GridMergeIndex> idxs;
 
         /** */
@@ -1322,11 +1364,17 @@ public class GridReduceQueryExecutor {
         private final AtomicReference<Object> state = new AtomicReference<>();
 
         /**
+         * @param id Query ID.
+         * @param qry Query text.
+         * @param cache Cache where query was executed.
          * @param conn Connection.
          * @param idxsCnt Number of indexes.
          * @param pageSize Page size.
+         * @param startTime Start time.
+         * @param cancel Query cancel handler.
          */
-        private QueryRun(Connection conn, int idxsCnt, int pageSize) {
+        private QueryRun(Long id, String qry, String cache, Connection conn, int idxsCnt, int pageSize, long startTime, GridQueryCancel cancel) {
+            this.qry = new GridRunningQueryInfo(id, qry, SQL_FIELDS, cache, startTime, cancel, false);
             this.conn = (JdbcConnection)conn;
             this.idxs = new ArrayList<>(idxsCnt);
             this.pageSize = pageSize > 0 ? pageSize : GridCacheTwoStepQuery.DFLT_PAGE_SIZE;
@@ -1384,4 +1432,4 @@ public class GridReduceQueryExecutor {
             return copy(msg, n, partsMap);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
index e47e893..66e7e4a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
@@ -17,15 +17,23 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.Query;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cache.query.annotations.QuerySqlFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -54,6 +62,7 @@ public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest {
 
         cc.setCopyOnRead(true);
         cc.setIndexedTypes(Integer.class, Value.class);
+        cc.setSqlFunctionClasses(TestSQLFunctions.class);
 
         cfg.setCacheConfiguration(cc);
 
@@ -72,7 +81,7 @@ public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest {
         IgniteCache<Integer, Value> cache = grid(0).cache(null);
 
         for (int i = 0; i < KEYS; i++)
-            cache.put(i, new Value("before"));
+            cache.put(i, new Value(i, "before-" + i));
     }
 
     /** {@inheritDoc} */
@@ -195,17 +204,148 @@ public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest {
         check(cache);
     }
 
-    /** */
-    private static class Value {
-        /** */
-        private String str;
+    /**
+     * Run specified query in separate thread.
+     *
+     * @param qry Query to execute.
+     */
+    private IgniteInternalFuture<?> runQueryAsync(final Query<?> qry) throws Exception {
+        return multithreadedAsync(new Runnable() {
+            @Override public void run() {
+                try {
+                    log.info(">>> Query started");
+
+                    grid(0).cache(null).query(qry).getAll();
+
+                    log.info(">>> Query finished");
+                }
+                catch (Throwable e) {
+                    e.printStackTrace();
+                }
+            }
+        }, 1, "run-query");
+    }
 
-        /**
-         * @param str String.
-         */
-        public Value(String str) {
-            this.str = str;
+    /**
+     * Test collecting info about running.
+     *
+     * @throws Exception If failed.
+     */
+    public void testRunningSqlFieldsQuery() throws Exception {
+        IgniteInternalFuture<?> fut = runQueryAsync(new SqlFieldsQuery("select _val, sleep(1000) from Value limit 3"));
+
+        Thread.sleep(500);
+
+        GridQueryProcessor qryProc = grid(0).context().query();
+
+        Collection<GridRunningQueryInfo> queries = qryProc.runningQueries(0);
+
+        assertEquals(1, queries.size());
+
+        fut.get();
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(0, queries.size());
+
+        SqlFieldsQuery qry = new SqlFieldsQuery("select _val, sleep(1000) from Value limit 3");
+        qry.setLocal(true);
+
+        fut = runQueryAsync(qry);
+
+        Thread.sleep(500);
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(1, queries.size());
+
+        fut.get();
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(0, queries.size());
+    }
+
+    /**
+     * Test collecting info about running.
+     *
+     * @throws Exception If failed.
+     */
+    public void testRunningSqlQuery() throws Exception {
+        IgniteInternalFuture<?> fut = runQueryAsync(new SqlQuery<Integer, Value>(Value.class, "id > sleep(100)"));
+
+        Thread.sleep(500);
+
+        GridQueryProcessor qryProc = grid(0).context().query();
+
+        Collection<GridRunningQueryInfo> queries = qryProc.runningQueries(0);
+
+        assertEquals(1, queries.size());
+
+        fut.get();
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(0, queries.size());
+
+        SqlQuery<Integer, Value> qry = new SqlQuery<>(Value.class, "id > sleep(100)");
+        qry.setLocal(true);
+
+        fut = runQueryAsync(qry);
+
+        Thread.sleep(500);
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(1, queries.size());
+
+        fut.get();
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(0, queries.size());
+    }
+
+    /**
+     * Test collecting info about running.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCancelingSqlFieldsQuery() throws Exception {
+        runQueryAsync(new SqlFieldsQuery("select * from (select _val, sleep(100) from Value limit 50)"));
+
+        Thread.sleep(500);
+
+        final GridQueryProcessor qryProc = grid(0).context().query();
+
+        Collection<GridRunningQueryInfo> queries = qryProc.runningQueries(0);
+
+        assertEquals(1, queries.size());
+
+        final Collection<GridRunningQueryInfo> finalQueries = queries;
+
+        for (GridRunningQueryInfo query : finalQueries)
+            qryProc.cancelQueries(Collections.singleton(query.id()));
+
+        int n = 100;
+
+        // Give cluster some time to cancel query and cleanup resources.
+        while (n > 0) {
+            Thread.sleep(100);
+
+            queries = qryProc.runningQueries(0);
+
+            if (queries.isEmpty())
+                break;
+
+            log.info(">>>> Wait for cancel: " + n);
+
+            n--;
         }
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(0, queries.size());
     }
 
     /**
@@ -218,9 +358,53 @@ public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest {
         for (Cache.Entry<Integer, Value> entry : cache) {
             cnt++;
 
-            assertEquals("before", entry.getValue().str);
+            assertEquals("before-" + entry.getKey(), entry.getValue().str);
         }
 
         assertEquals(KEYS, cnt);
     }
-}
\ No newline at end of file
+
+    /** */
+    private static class Value {
+        /** */
+        @QuerySqlField
+        private int id;
+
+        /** */
+        @QuerySqlField
+        private String str;
+
+        /**
+         * @param id ID.
+         * @param str String.
+         */
+        public Value(int id, String str) {
+            this.id = id;
+            this.str = str;
+        }
+    }
+
+    /**
+     * Utility class with custom SQL functions.
+     */
+    public static class TestSQLFunctions {
+        /**
+         * Sleep function to simulate long running queries.
+         *
+         * @param x Time to sleep.
+         * @return Return specified argument.
+         */
+        @QuerySqlFunction
+        public static long sleep(long x) {
+            if (x >= 0)
+                try {
+                    Thread.sleep(x);
+                }
+                catch (InterruptedException ignored) {
+                    // No-op.
+                }
+
+            return x;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
index 1f10593..01fefa3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
@@ -477,4 +477,4 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
             return storeId;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 17151d8..09b27d2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -106,6 +106,9 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         spi.registerCache(null, cacheCfg("B"));
     }
 
+    /**
+     * @param name Name.
+     */
     private CacheConfiguration cacheCfg(String name) {
         CacheConfiguration<?,?> cfg = new CacheConfiguration<>();
 
@@ -114,6 +117,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         return cfg;
     }
 
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         idx.stop();
 
@@ -182,6 +186,9 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         return idx;
     }
 
+    /**
+     * @return {@code true} if OFF-HEAP mode should be tested.
+     */
     protected boolean offheap() {
         return false;
     }


[03/23] ignite git commit: IGNITE-4717 Fixed hangs in VisorCacheClearTask. (cherry picked from commit 76f3060)

Posted by vo...@apache.org.
IGNITE-4717 Fixed hangs in VisorCacheClearTask.
(cherry picked from commit 76f3060)


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

Branch: refs/heads/ignite-4565-ddl
Commit: 7292a5306c90d5c48c520bf9ffad388769250488
Parents: 5f5cce4
Author: Andrey Novikov <an...@gridgain.com>
Authored: Mon Feb 20 18:23:33 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Mar 2 10:56:21 2017 +0700

----------------------------------------------------------------------
 .../visor/cache/VisorCacheClearTask.java        | 88 +++++---------------
 .../visor/compute/VisorGatewayTask.java         | 30 ++++++-
 2 files changed, 49 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7292a530/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
index 1f1a6fb..0c8476f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.visor.cache;
 
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.compute.ComputeJobContext;
 import org.apache.ignite.internal.processors.task.GridInternal;
@@ -26,7 +25,6 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.visor.VisorJob;
 import org.apache.ignite.internal.visor.VisorOneNodeTask;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.resources.JobContextResource;
@@ -90,17 +88,11 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
         }
 
         /**
-         * @param subJob Sub job to execute asynchronously.
+         * @param fut Future for asynchronous cache operation.
          * @param idx Index.
          * @return {@code true} If subJob was not completed and this job should be suspended.
          */
-        private boolean callAsync(IgniteCallable<Integer> subJob, int idx) {
-            IgniteCompute compute = ignite.compute(ignite.cluster().forCacheNodes(cacheName)).withAsync();
-
-            compute.call(subJob);
-
-            IgniteFuture<Integer> fut = compute.future();
-
+        private boolean callAsync(IgniteFuture<Integer> fut, int idx) {
             futs[idx] = fut;
 
             if (fut.isDone())
@@ -119,16 +111,28 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
                 futs = new IgniteFuture[3];
 
             if (futs[0] == null || futs[1] == null || futs[2] == null) {
-                IgniteCache cache = ignite.cache(cacheName);
+                IgniteCache cache = ignite.cache(cacheName).withAsync();
+
+                if (futs[0] == null) {
+                    cache.size(CachePeekMode.PRIMARY);
+
+                    if (callAsync(cache.<Integer>future(), 0))
+                        return null;
+                }
 
-                if (futs[0] == null && callAsync(new VisorCacheSizeCallable(cache), 0))
-                    return null;
+                if (futs[1] == null) {
+                    cache.clear();
 
-                if (futs[1] == null && callAsync(new VisorCacheClearCallable(cache), 1))
-                    return null;
+                    if (callAsync(cache.<Integer>future(), 1))
+                        return null;
+                }
+                
+                if (futs[2] == null) {
+                    cache.size(CachePeekMode.PRIMARY);
 
-                if (futs[2] == null && callAsync(new VisorCacheSizeCallable(cache), 2))
-                    return null;
+                    if (callAsync(cache.<Integer>future(), 2))
+                        return null;
+                }
             }
 
             assert futs[0].isDone() && futs[1].isDone() && futs[2].isDone();
@@ -141,54 +145,4 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
             return S.toString(VisorCacheClearJob.class, this);
         }
     }
-
-    /**
-     * Callable to get cache size.
-     */
-    @GridInternal
-    private static class VisorCacheSizeCallable implements IgniteCallable<Integer> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        private final IgniteCache cache;
-
-        /**
-         * @param cache Cache to take size from.
-         */
-        private VisorCacheSizeCallable(IgniteCache cache) {
-            this.cache = cache;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Integer call() throws Exception {
-            return cache.size(CachePeekMode.PRIMARY);
-        }
-    }
-
-    /**
-     * Callable to clear cache.
-     */
-    @GridInternal
-    private static class VisorCacheClearCallable implements IgniteCallable<Integer> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        private final IgniteCache cache;
-
-        /**
-         * @param cache Cache to clear.
-         */
-        private VisorCacheClearCallable(IgniteCache cache) {
-            this.cache = cache;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Integer call() throws Exception {
-            cache.clear();
-
-            return 0;
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7292a530/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java
index 2539a26..a64ec6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java
@@ -29,21 +29,26 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
+import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobContext;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeJobResultPolicy;
 import org.apache.ignite.compute.ComputeTask;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.util.lang.GridTuple3;
+import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.visor.VisorTaskArgument;
 import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.resources.JobContextResource;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -101,9 +106,16 @@ public class VisorGatewayTask implements ComputeTask<Object[], Object> {
         @IgniteInstanceResource
         protected transient IgniteEx ignite;
 
+        /** Auto-inject job context. */
+        @JobContextResource
+        protected transient ComputeJobContext jobCtx;
+
         /** Arguments count. */
         private final int argsCnt;
 
+        /** Future for spawned task. */
+        private transient IgniteFuture fut;
+
         /**
          * Create job with specified argument.
          *
@@ -284,6 +296,9 @@ public class VisorGatewayTask implements ComputeTask<Object[], Object> {
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
         @Override public Object execute() throws IgniteException {
+            if (fut != null)
+                return fut.get();
+
             String nidsArg = argument(0);
             String taskName = argument(1);
 
@@ -355,8 +370,19 @@ public class VisorGatewayTask implements ComputeTask<Object[], Object> {
                 }
             }
 
-            return ignite.compute(ignite.cluster().forNodeIds(nids))
-                .execute(taskName, new VisorTaskArgument<>(nids, jobArgs, false));
+            IgniteCompute comp = ignite.compute(ignite.cluster().forNodeIds(nids)).withAsync();
+            
+            comp.execute(taskName, new VisorTaskArgument<>(nids, jobArgs, false));
+
+            fut = comp.future();
+
+            fut.listen(new CI1<IgniteFuture<Object>>() {
+                @Override public void apply(IgniteFuture<Object> f) {
+                    jobCtx.callcc();
+                }
+            });
+
+            return jobCtx.holdcc();
         }
     }
 }


[09/23] ignite git commit: IGNITE-3386 Reverted: "Reentrant lock is lost when lock owner leaves topology"

Posted by vo...@apache.org.
IGNITE-3386 Reverted: "Reentrant lock is lost when lock owner leaves topology"


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

Branch: refs/heads/ignite-4565-ddl
Commit: 6633da64d57eadacf30bf437a9c3a3d205903dcd
Parents: 590b82d
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Mar 14 16:40:58 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Mar 14 16:40:58 2017 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java |  1 -
 .../datastructures/DataStructuresProcessor.java |  3 +-
 .../datastructures/GridCacheLockImpl.java       | 17 +-----
 .../internal/GridCacheRecreateLockTest.java     | 58 --------------------
 .../testsuites/IgniteComputeGridTestSuite.java  |  2 -
 5 files changed, 3 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6633da64/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 7f8974c..0de08d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -527,7 +527,6 @@ public interface Ignite extends AutoCloseable {
      *      all threads on other nodes waiting to acquire lock are interrupted.
      * @param fair If {@code True}, fair lock will be created.
      * @param create Boolean flag indicating whether data structure should be created if does not exist.
-     *      Will re-create lock if the node that stored the lock left topology and there are no backups left.
      * @return ReentrantLock for the given name.
      * @throws IgniteException If reentrant lock could not be fetched or created.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6633da64/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 698efd9..1cad22f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -1396,8 +1396,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                         name,
                         key,
                         reentrantLockView,
-                        dsCacheCtx,
-                        create);
+                        dsCacheCtx);
 
                     dsMap.put(key, reentrantLock0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6633da64/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
index 1cf78fa..3ab7289 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
@@ -105,9 +105,6 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
     /** Flag indicating that every operation on this lock should be interrupted. */
     private volatile boolean interruptAll;
 
-    /** Re-create flag. */
-    private volatile boolean reCreate;
-
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -525,14 +522,7 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
                                 GridCacheLockState val = lockView.get(key);
 
                                 if (val == null)
-                                    if (reCreate) {
-                                        val = new GridCacheLockState(0, ctx.nodeId(), 0, failoverSafe, fair);
-
-                                        lockView.put(key, val);
-                                    }
-                                    else
-                                        throw new IgniteCheckedException("Failed to find reentrant lock with " +
-                                            "the given name: " + name);
+                                    throw new IgniteCheckedException("Failed to find reentrant lock with given name: " + name);
 
                                 final long newThreadID = newThread.getId();
 
@@ -1058,14 +1048,12 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
      * @param key Reentrant lock key.
      * @param lockView Reentrant lock projection.
      * @param ctx Cache context.
-     * @param reCreate If {@code true} reentrant lock will be re-created in case it is not in cache.
      */
     @SuppressWarnings("unchecked")
     public GridCacheLockImpl(String name,
         GridCacheInternalKey key,
         IgniteInternalCache<GridCacheInternalKey, GridCacheLockState> lockView,
-        GridCacheContext ctx,
-        boolean reCreate) {
+        GridCacheContext ctx) {
         assert name != null;
         assert key != null;
         assert ctx != null;
@@ -1075,7 +1063,6 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
         this.key = key;
         this.lockView = lockView;
         this.ctx = ctx;
-        this.reCreate = reCreate;
 
         log = ctx.logger(getClass());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6633da64/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
deleted file mode 100644
index ae850f7..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-package org.apache.ignite.internal;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLock;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.testframework.junits.common.GridCommonTest;
-
-/**
- * Create lock after owner node left topology test
- */
-@GridCommonTest(group = "Kernal Self")
-public class GridCacheRecreateLockTest extends GridCommonAbstractTest {
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testLockOwnerLeavesGrid() throws Exception {
-        final Ignite ignite = startNodeAndLock("node1");
-
-        new Thread(new Runnable() {
-            @Override public void run() {
-                try {
-                    Thread.sleep(2000);
-                }
-                catch (InterruptedException e) {
-                    e.printStackTrace();
-                }
-
-                ignite.close();
-            }
-        }).start();
-
-        startNodeAndLock("node2");
-    }
-
-    /**
-     * @param name Grid name.
-     * @return Started Ignite instance.
-     */
-    private Ignite startNodeAndLock(String name) throws Exception {
-        IgniteConfiguration cfg = new IgniteConfiguration();
-        cfg.setGridName(name);
-
-        Ignite ignite = Ignition.start(cfg);
-
-        IgniteLock lock = ignite.reentrantLock("lock", true, true, true);
-
-        System.out.println("acquiring lock");
-
-        lock.lock();
-
-        System.out.println("acquired lock");
-
-        return ignite;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6633da64/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
index 7c8b6a9..8a501fd 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
@@ -22,7 +22,6 @@ import org.apache.ignite.internal.ClusterNodeMetricsSelfTest;
 import org.apache.ignite.internal.GridAffinityNoCacheSelfTest;
 import org.apache.ignite.internal.GridAffinitySelfTest;
 import org.apache.ignite.internal.GridAlwaysFailoverSpiFailSelfTest;
-import org.apache.ignite.internal.GridCacheRecreateLockTest;
 import org.apache.ignite.internal.GridCancelOnGridStopSelfTest;
 import org.apache.ignite.internal.GridCancelUnusedJobSelfTest;
 import org.apache.ignite.internal.GridCancelledJobsMetricsSelfTest;
@@ -153,7 +152,6 @@ public class IgniteComputeGridTestSuite {
         suite.addTestSuite(TaskNodeRestartTest.class);
         suite.addTestSuite(IgniteRoundRobinErrorAfterClientReconnectTest.class);
         suite.addTestSuite(PublicThreadpoolStarvationTest.class);
-        suite.addTestSuite(GridCacheRecreateLockTest.class);
 
         return suite;
     }


[14/23] ignite git commit: IGNITE-4690: CPP: Added ignite::Time type

Posted by vo...@apache.org.
IGNITE-4690: CPP: Added ignite::Time type


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

Branch: refs/heads/ignite-4565-ddl
Commit: 02991d3066b73c1a006afb55b4138e33a2e99ed3
Parents: 19ba8f4
Author: Igor Sapego <is...@gridgain.com>
Authored: Tue Mar 21 14:53:29 2017 +0300
Committer: Igor Sapego <is...@gridgain.com>
Committed: Tue Mar 21 14:53:29 2017 +0300

----------------------------------------------------------------------
 .../include/ignite/binary/binary_raw_reader.h   |  19 +
 .../include/ignite/binary/binary_raw_writer.h   |  15 +
 .../include/ignite/binary/binary_reader.h       |  21 +
 .../include/ignite/binary/binary_writer.h       |  17 +
 .../include/ignite/impl/binary/binary_common.h  |   6 +
 .../ignite/impl/binary/binary_reader_impl.h     | 128 ++-
 .../include/ignite/impl/binary/binary_utils.h   |  25 +-
 .../ignite/impl/binary/binary_writer_impl.h     |  46 +-
 .../cpp/binary/src/binary/binary_raw_reader.cpp |  10 +
 .../cpp/binary/src/binary/binary_raw_writer.cpp |  10 +
 .../cpp/binary/src/binary/binary_reader.cpp     |  12 +-
 .../cpp/binary/src/binary/binary_writer.cpp     |  10 +
 .../src/impl/binary/binary_reader_impl.cpp      | 177 +++-
 .../cpp/binary/src/impl/binary/binary_utils.cpp |  12 +
 .../src/impl/binary/binary_writer_impl.cpp      | 105 ++-
 modules/platforms/cpp/common/Makefile.am        |   3 +-
 .../platforms/cpp/common/include/Makefile.am    |   1 +
 .../cpp/common/include/ignite/common/utils.h    |  53 +-
 .../platforms/cpp/common/include/ignite/time.h  | 138 +++
 .../cpp/common/project/vs/common.vcxproj        |   2 +
 .../common/project/vs/common.vcxproj.filters    |   6 +
 .../platforms/cpp/common/src/common/utils.cpp   |  30 +
 modules/platforms/cpp/common/src/time.cpp       |  83 ++
 modules/platforms/cpp/core-test/Makefile.am     |   1 +
 .../cpp/core-test/config/cache-query.xml        |  23 +
 .../include/ignite/binary_test_utils.h          |  53 ++
 .../cpp/core-test/include/ignite/test_type.h    |  11 +-
 .../cpp/core-test/project/vs/core-test.vcxproj  |   1 +
 .../project/vs/core-test.vcxproj.filters        |   3 +
 .../cpp/core-test/src/binary_object_test.cpp    | 110 +--
 .../src/binary_reader_writer_raw_test.cpp       | 712 +++------------
 .../core-test/src/binary_reader_writer_test.cpp | 913 +++----------------
 .../cpp/core-test/src/binary_session_test.cpp   |  18 +
 .../cpp/core-test/src/cache_query_test.cpp      |  65 +-
 .../platforms/cpp/core-test/src/cache_test.cpp  |  12 +
 .../cpp/core-test/src/date_time_test.cpp        | 265 ++++++
 36 files changed, 1491 insertions(+), 1625 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
index d15848b..d110078 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
@@ -275,6 +275,25 @@ namespace ignite
             int32_t ReadTimestampArray(Timestamp* res, int32_t len);
 
             /**
+             * Read Time. Maps to "Time" type in Java.
+             *
+             * @return Result.
+             */
+            Time ReadTime();
+
+            /**
+             * Read array of Times. Maps to "Time[]" type in Java.
+             *
+             * @param res Array to store data to.
+             * @param len Expected length of array.
+             * @return Actual amount of elements read. If "len" argument is less than actual
+             *     array size or resulting array is set to null, nothing will be written
+             *     to resulting array and returned value will contain required array length.
+             *     -1 will be returned in case array in stream was null.
+             */
+            int32_t ReadTimeArray(Time* res, int32_t len);
+
+            /**
              * Read string.
              *
              * @param res Array to store data to. 

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
index c960406..88a8014 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
@@ -230,6 +230,21 @@ namespace ignite
             void WriteTimestampArray(const Timestamp* val, int32_t len);
 
             /**
+             * Write Time. Maps to "Time" type in Java.
+             *
+             * @param val Value.
+             */
+            void WriteTime(const Time& val);
+
+            /**
+             * Write array of Time. Maps to "Time[]" type in Java.
+             *
+             * @param val Array.
+             * @param len Array length.
+             */
+            void WriteTimeArray(const Time* val, const int32_t len);
+
+            /**
              * Write string.
              *
              * @param val Null-terminated character array.

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
index e4eb690..e07ec14 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
@@ -296,6 +296,27 @@ namespace ignite
             int32_t ReadTimestampArray(const char* fieldName, Timestamp* res, const int32_t len);
 
             /**
+             * Read Time. Maps to "Time" type in Java.
+             *
+             * @param fieldName Field name.
+             * @return Result.
+             */
+            Time ReadTime(const char* fieldName);
+
+            /**
+             * Read array of Times. Maps to "Time[]" type in Java.
+             *
+             * @param fieldName Field name.
+             * @param res Array to store data to.
+             * @param len Expected length of array.
+             * @return Actual amount of elements read. If "len" argument is less than actual
+             *     array size or resulting array is set to null, nothing will be written
+             *     to resulting array and returned value will contain required array length.
+             *     -1 will be returned in case array in stream was null.
+             */
+            int32_t ReadTimeArray(const char* fieldName, Time* res, const int32_t len);
+
+            /**
              * Read string.
              *
              * @param fieldName Field name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
index 1923694..e3962cf 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
@@ -248,6 +248,23 @@ namespace ignite
             void WriteTimestampArray(const char* fieldName, const Timestamp* val, const int32_t len);
 
             /**
+             * Write Time. Maps to "Time" type in Java.
+             *
+             * @param fieldName Field name.
+             * @param val Value.
+             */
+            void WriteTime(const char* fieldName, const Time& val);
+
+            /**
+             * Write array of Times. Maps to "Time[]" type in Java.
+             *
+             * @param fieldName Field name.
+             * @param val Array.
+             * @param len Array length.
+             */
+            void WriteTimeArray(const char* fieldName, const Time* val, const int32_t len);
+
+            /**
              * Write string.
              *
              * @param fieldName Field name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_common.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_common.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_common.h
index aa15cc5..a4275c7 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_common.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_common.h
@@ -164,6 +164,12 @@ namespace ignite
             /** Type: timestamp array. */
             const int8_t IGNITE_TYPE_ARRAY_TIMESTAMP = 34;
 
+            /** Type: time. */
+            const int8_t IGNITE_TYPE_TIME = 36;
+
+            /** Type: time array. */
+            const int8_t IGNITE_TYPE_ARRAY_TIME = 37;
+
             /** Read/write single object. */
             const int32_t IGNITE_BINARY_MODE_SINGLE = 0;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
index 814651d..242bb1e 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
@@ -33,6 +33,7 @@
 #include "ignite/guid.h"
 #include "ignite/date.h"
 #include "ignite/timestamp.h"
+#include "ignite/time.h"
 
 namespace ignite
 {
@@ -513,6 +514,46 @@ namespace ignite
                 int32_t ReadTimestampArray(const char* fieldName, Timestamp* res, const int32_t len);
 
                 /**
+                 * Read Time. Maps to "Time" type in Java.
+                 *
+                 * @return Result.
+                 */
+                Time ReadTime();
+
+                /**
+                 * Read array of Times. Maps to "Time[]" type in Java.
+                 *
+                 * @param res Array to store data to.
+                 * @param len Expected length of array.
+                 * @return Actual amount of elements read. If "len" argument is less than actual
+                 *     array size or resulting array is set to null, nothing will be written
+                 *     to resulting array and returned value will contain required array length.
+                 *     -1 will be returned in case array in stream was null.
+                 */
+                int32_t ReadTimeArray(Time* res, int32_t len);
+
+                /**
+                 * Read Time. Maps to "Time" type in Java.
+                 *
+                 * @param fieldName Field name.
+                 * @return Result.
+                 */
+                Time ReadTime(const char* fieldName);
+
+                /**
+                 * Read array of Times. Maps to "Time[]" type in Java.
+                 *
+                 * @param fieldName Field name.
+                 * @param res Array to store data to.
+                 * @param len Expected length of array.
+                 * @return Actual amount of elements read. If "len" argument is less than actual
+                 *     array size or resulting array is set to null, nothing will be written
+                 *     to resulting array and returned value will contain required array length.
+                 *     -1 will be returned in case array in stream was null.
+                 */
+                int32_t ReadTimeArray(const char* fieldName, Time* res, const int32_t len);
+
+                /**
                  * Read string.
                  *
                  * @param len Expected length of string.
@@ -1065,16 +1106,26 @@ namespace ignite
                 );
 
                 /**
+                 * Internal routine to read Time array.
+                 *
+                 * @param stream Stream.
+                 * @param res Resulting array.
+                 * @param len Length.
+                 */
+                static void ReadTimeArrayInternal(
+                    interop::InteropInputStream* stream, 
+                    Time* res,
+                    const int32_t len
+                );
+
+                /**
                  * Read single value in raw mode.
                  * 
-                 * @param stream Stream.
                  * @param func Function to be invoked on stream.
                  * @return Result.
                  */
                 template<typename T>
-                T ReadRaw(
-                    T(*func) (interop::InteropInputStream*)
-                )
+                T ReadRaw(T(*func)(interop::InteropInputStream*))
                 {
                     {
                         CheckRawMode(true);
@@ -1347,43 +1398,7 @@ namespace ignite
                  * @param func Function to be applied to the stream.
                  */
                 template<typename T>
-                T ReadTopObject0(const int8_t expHdr, T(*func) (ignite::impl::interop::InteropInputStream*))
-                {
-                    int8_t typeId = stream->ReadInt8();
-
-                    if (typeId == expHdr)
-                        return func(stream);
-                    else if (typeId == IGNITE_HDR_NULL)
-                        return GetNull<T>();
-                    else {
-                        int32_t pos = stream->Position() - 1;
-
-                        IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY, "Invalid header", "position", pos, "expected", (int)expHdr, "actual", (int)typeId)
-                    }
-                }
-
-                /**
-                 * Read value.
-                 *
-                 * @param expHdr Expected header.
-                 * @param func Function to be applied to the stream.
-                 * @param dflt Default value.
-                 */
-                template<typename T>
-                T ReadTopObject0(const int8_t expHdr, T(*func) (ignite::impl::interop::InteropInputStream*), T dflt)
-                {
-                    int8_t typeId = stream->ReadInt8();
-
-                    if (typeId == expHdr)
-                        return func(stream);
-                    else if (typeId == IGNITE_HDR_NULL)
-                        return dflt;
-                    else {
-                        int32_t pos = stream->Position() - 1;
-
-                        IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY, "Invalid header", "position", pos, "expected", (int)expHdr, "actual", (int)typeId)
-                    }
-                }
+                T ReadTopObject0(const int8_t expHdr, T (*func)(ignite::impl::interop::InteropInputStream*));
             };
 
             template<>
@@ -1420,35 +1435,10 @@ namespace ignite
             Timestamp IGNITE_IMPORT_EXPORT BinaryReaderImpl::ReadTopObject<Timestamp>();
 
             template<>
-            inline std::string IGNITE_IMPORT_EXPORT BinaryReaderImpl::ReadTopObject<std::string>()
-            {
-                int8_t typeId = stream->ReadInt8();
-
-                if (typeId == IGNITE_TYPE_STRING)
-                {
-                    int32_t realLen = stream->ReadInt32();
-
-                    std::string res;
+            Time IGNITE_IMPORT_EXPORT BinaryReaderImpl::ReadTopObject<Time>();
 
-                    if (realLen > 0)
-                    {
-                        res.resize(realLen, 0);
-
-                        stream->ReadInt8Array(reinterpret_cast<int8_t*>(&res[0]), realLen);
-                    }
-
-                    return res;
-                }
-                else if (typeId == IGNITE_HDR_NULL)
-                    return std::string();
-                else
-                {
-                    int32_t pos = stream->Position() - 1;
-
-                    IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY, "Invalid header", "position", pos,
-                        "expected", static_cast<int>(IGNITE_TYPE_STRING), "actual", static_cast<int>(typeId))
-                }
-            }
+            template<>
+            std::string IGNITE_IMPORT_EXPORT BinaryReaderImpl::ReadTopObject<std::string>();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_utils.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_utils.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_utils.h
index b541483..6cd90b0 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_utils.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_utils.h
@@ -25,6 +25,7 @@
 #include "ignite/guid.h"
 #include "ignite/date.h"
 #include "ignite/timestamp.h"
+#include "ignite/time.h"
 
 #include "ignite/binary/binary_type.h"
 
@@ -424,7 +425,7 @@ namespace ignite
                  * Utility method to read Timestamp from stream.
                  *
                  * @param stream Stream.
-                 * @param res Value.
+                 * @return Value.
                  */
                 static Timestamp ReadTimestamp(interop::InteropInputStream* stream);
 
@@ -437,6 +438,22 @@ namespace ignite
                 static void WriteTimestamp(interop::InteropOutputStream* stream, const Timestamp val);
 
                 /**
+                 * Utility method to read Time from stream.
+                 *
+                 * @param stream Stream.
+                 * @return Value.
+                 */
+                static Time ReadTime(interop::InteropInputStream* stream);
+
+                /**
+                 * Utility method to write Timestamp to stream.
+                 *
+                 * @param stream Stream.
+                 * @param val Value.
+                 */
+                static void WriteTime(interop::InteropOutputStream* stream, const Time val);
+
+                /**
                  * Utility method to write string to stream.
                  *
                  * @param stream Stream.
@@ -526,6 +543,12 @@ namespace ignite
             }
 
             template<>
+            inline Time BinaryUtils::GetDefaultValue<Time>()
+            {
+                return Time();
+            }
+
+            template<>
             inline std::string BinaryUtils::GetDefaultValue<std::string>()
             {
                 return std::string();

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
index fc38837..ea5f81f 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
@@ -424,6 +424,38 @@ namespace ignite
                 void WriteTimestampArray(const char* fieldName, const Timestamp* val, const int32_t len);
 
                 /**
+                 * Write Time. Maps to "Time" type in Java.
+                 *
+                 * @param val Value.
+                 */
+                void WriteTime(const Time& val);
+
+                /**
+                 * Write array of Time. Maps to "Time[]" type in Java.
+                 *
+                 * @param val Array.
+                 * @param len Array length.
+                 */
+                void WriteTimeArray(const Time* val, const int32_t len);
+
+                /**
+                 * Write Time. Maps to "Time" type in Java.
+                 *
+                 * @param fieldName Field name.
+                 * @param val Value.
+                 */
+                void WriteTime(const char* fieldName, const Time& val);
+
+                /**
+                 * Write array of Times. Maps to "Time[]" type in Java.
+                 *
+                 * @param fieldName Field name.
+                 * @param val Array.
+                 * @param len Array length.
+                 */
+                void WriteTimeArray(const char* fieldName, const Time* val, const int32_t len);
+
+                /**
                  * Write string.
                  *
                  * @param val String.
@@ -914,7 +946,7 @@ namespace ignite
                 /**
                  * Check whether session ID matches.
                  *
-                 * @param ses Expected session ID.
+                 * @param expSes Expected session ID.
                  */
                 void CheckSession(int32_t expSes) const;
 
@@ -975,16 +1007,10 @@ namespace ignite
             void IGNITE_IMPORT_EXPORT BinaryWriterImpl::WriteTopObject(const Timestamp& obj);
 
             template<>
-            inline void IGNITE_IMPORT_EXPORT BinaryWriterImpl::WriteTopObject(const std::string& obj)
-            {
-                const char* obj0 = obj.c_str();
+            void IGNITE_IMPORT_EXPORT BinaryWriterImpl::WriteTopObject(const Time& obj);
 
-                int32_t len = static_cast<int32_t>(obj.size());
-
-                stream->WriteInt8(IGNITE_TYPE_STRING);
-
-                BinaryUtils::WriteString(stream, obj0, len);
-            }
+            template<>
+            void IGNITE_IMPORT_EXPORT BinaryWriterImpl::WriteTopObject(const std::string& obj);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/src/binary/binary_raw_reader.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/binary/binary_raw_reader.cpp b/modules/platforms/cpp/binary/src/binary/binary_raw_reader.cpp
index e472588..9c1d6d9 100644
--- a/modules/platforms/cpp/binary/src/binary/binary_raw_reader.cpp
+++ b/modules/platforms/cpp/binary/src/binary/binary_raw_reader.cpp
@@ -139,6 +139,16 @@ namespace ignite
             return impl->ReadTimestampArray(res, len);
         }
 
+        Time BinaryRawReader::ReadTime()
+        {
+            return impl->ReadTime();
+        }
+
+        int32_t BinaryRawReader::ReadTimeArray(Time* res, int32_t len)
+        {
+            return impl->ReadTimeArray(res, len);
+        }
+
         int32_t BinaryRawReader::ReadString(char* res, const int32_t len)
         {
             return impl->ReadString(res, len);

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/src/binary/binary_raw_writer.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/binary/binary_raw_writer.cpp b/modules/platforms/cpp/binary/src/binary/binary_raw_writer.cpp
index a83c74b..f2e94a5 100644
--- a/modules/platforms/cpp/binary/src/binary/binary_raw_writer.cpp
+++ b/modules/platforms/cpp/binary/src/binary/binary_raw_writer.cpp
@@ -139,6 +139,16 @@ namespace ignite
             impl->WriteTimestampArray(val, len);
         }
 
+        void BinaryRawWriter::WriteTime(const Time& val)
+        {
+            impl->WriteTime(val);
+        }
+
+        void BinaryRawWriter::WriteTimeArray(const Time* val, const int32_t len)
+        {
+            impl->WriteTimeArray(val, len);
+        }
+
         void BinaryRawWriter::WriteString(const char* val)
         {
             if (val)

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/src/binary/binary_reader.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/binary/binary_reader.cpp b/modules/platforms/cpp/binary/src/binary/binary_reader.cpp
index 814db83..f9fa72e 100644
--- a/modules/platforms/cpp/binary/src/binary/binary_reader.cpp
+++ b/modules/platforms/cpp/binary/src/binary/binary_reader.cpp
@@ -138,7 +138,17 @@ namespace ignite
         {
             return impl->ReadTimestampArray(fieldName, res, len);
         }
-        
+
+        Time BinaryReader::ReadTime(const char* fieldName)
+        {
+            return impl->ReadTime(fieldName);
+        }
+
+        int32_t BinaryReader::ReadTimeArray(const char* fieldName, Time* res, const int32_t len)
+        {
+            return impl->ReadTimeArray(fieldName, res, len);
+        }
+
         int32_t BinaryReader::ReadString(const char* fieldName, char* res, int32_t len)
         {
             return impl->ReadString(fieldName, res, len);

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/src/binary/binary_writer.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/binary/binary_writer.cpp b/modules/platforms/cpp/binary/src/binary/binary_writer.cpp
index efbc0ce..6c549bb 100644
--- a/modules/platforms/cpp/binary/src/binary/binary_writer.cpp
+++ b/modules/platforms/cpp/binary/src/binary/binary_writer.cpp
@@ -139,6 +139,16 @@ namespace ignite
             impl->WriteTimestampArray(fieldName, val, len);
         }
 
+        void BinaryWriter::WriteTime(const char* fieldName, const Time& val)
+        {
+            impl->WriteTime(fieldName, val);
+        }
+
+        void BinaryWriter::WriteTimeArray(const char* fieldName, const Time* val, const int32_t len)
+        {
+            impl->WriteTimeArray(fieldName, val, len);
+        }
+
         void BinaryWriter::WriteString(const char* fieldName, const char* val)
         {
             if (val)

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
index c128df6..1c4104b 100644
--- a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
@@ -378,7 +378,63 @@ namespace ignite
             void BinaryReaderImpl::ReadTimestampArrayInternal(interop::InteropInputStream* stream, Timestamp* res, const int32_t len)
             {
                 for (int i = 0; i < len; i++)
-                    *(res + i) = ReadNullable<Timestamp>(stream, BinaryUtils::ReadTimestamp, IGNITE_TYPE_TIMESTAMP);
+                    res[i] = ReadNullable<Timestamp>(stream, BinaryUtils::ReadTimestamp, IGNITE_TYPE_TIMESTAMP);
+            }
+
+            Time BinaryReaderImpl::ReadTime()
+            {
+                CheckRawMode(true);
+                CheckSingleMode(true);
+
+                return ReadNullable(stream, BinaryUtils::ReadTime, IGNITE_TYPE_TIME);
+            }
+
+            int32_t BinaryReaderImpl::ReadTimeArray(Time * res, int32_t len)
+            {
+                CheckRawMode(true);
+                CheckSingleMode(true);
+
+                return ReadArrayInternal<Time>(res, len, stream, ReadTimeArrayInternal, IGNITE_TYPE_ARRAY_TIME);
+            }
+
+            Time BinaryReaderImpl::ReadTime(const char * fieldName)
+            {
+                CheckRawMode(false);
+                CheckSingleMode(true);
+
+                int32_t fieldId = idRslvr->GetFieldId(typeId, fieldName);
+                int32_t fieldPos = FindField(fieldId);
+
+                if (fieldPos <= 0)
+                    return Time();
+
+                stream->Position(fieldPos);
+
+                return ReadNullable(stream, BinaryUtils::ReadTime, IGNITE_TYPE_TIME);
+            }
+
+            int32_t BinaryReaderImpl::ReadTimeArray(const char * fieldName, Time * res, const int32_t len)
+            {
+                CheckRawMode(false);
+                CheckSingleMode(true);
+
+                int32_t fieldId = idRslvr->GetFieldId(typeId, fieldName);
+                int32_t fieldPos = FindField(fieldId);
+
+                if (fieldPos <= 0)
+                    return -1;
+
+                stream->Position(fieldPos);
+
+                int32_t realLen = ReadArrayInternal<Time>(res, len, stream, ReadTimeArrayInternal, IGNITE_TYPE_ARRAY_TIME);
+
+                return realLen;
+            }
+
+            void BinaryReaderImpl::ReadTimeArrayInternal(interop::InteropInputStream* stream, Time* res, const int32_t len)
+            {
+                for (int i = 0; i < len; i++)
+                    res[i] = ReadNullable<Time>(stream, BinaryUtils::ReadTime, IGNITE_TYPE_TIME);
             }
 
             int32_t BinaryReaderImpl::ReadString(char* res, const int32_t len)
@@ -692,73 +748,55 @@ namespace ignite
             template <>
             int8_t BinaryReaderImpl::ReadTopObject<int8_t>()
             {
-                return ReadTopObject0(IGNITE_TYPE_BYTE, BinaryUtils::ReadInt8,
-                    BinaryUtils::GetDefaultValue<int8_t>());
+                return ReadTopObject0<int8_t>(IGNITE_TYPE_BYTE, BinaryUtils::ReadInt8);
             }
 
             template <>
             bool BinaryReaderImpl::ReadTopObject<bool>()
             {
-                return ReadTopObject0(IGNITE_TYPE_BOOL, BinaryUtils::ReadBool,
-                    BinaryUtils::GetDefaultValue<bool>());
+                return ReadTopObject0<bool>(IGNITE_TYPE_BOOL, BinaryUtils::ReadBool);
             }
 
             template <>
             int16_t BinaryReaderImpl::ReadTopObject<int16_t>()
             {
-                return ReadTopObject0(IGNITE_TYPE_SHORT, BinaryUtils::ReadInt16,
-                    BinaryUtils::GetDefaultValue<int16_t>());
+                return ReadTopObject0<int16_t>(IGNITE_TYPE_SHORT, BinaryUtils::ReadInt16);
             }
 
             template <>
             uint16_t BinaryReaderImpl::ReadTopObject<uint16_t>()
             {
-                return ReadTopObject0(IGNITE_TYPE_CHAR, BinaryUtils::ReadUInt16,
-                    BinaryUtils::GetDefaultValue<uint16_t>());
+                return ReadTopObject0<uint16_t>(IGNITE_TYPE_CHAR, BinaryUtils::ReadUInt16);
             }
 
             template <>
             int32_t BinaryReaderImpl::ReadTopObject<int32_t>()
             {
-                return ReadTopObject0(IGNITE_TYPE_INT, BinaryUtils::ReadInt32,
-                    BinaryUtils::GetDefaultValue<int32_t>());
+                return ReadTopObject0<int32_t>(IGNITE_TYPE_INT, BinaryUtils::ReadInt32);
             }
 
             template <>
             int64_t BinaryReaderImpl::ReadTopObject<int64_t>()
             {
-                return ReadTopObject0(IGNITE_TYPE_LONG, BinaryUtils::ReadInt64,
-                    BinaryUtils::GetDefaultValue<int64_t>());
+                return ReadTopObject0<int64_t>(IGNITE_TYPE_LONG, BinaryUtils::ReadInt64);
             }
 
             template <>
             float BinaryReaderImpl::ReadTopObject<float>()
             {
-                return ReadTopObject0(IGNITE_TYPE_FLOAT, BinaryUtils::ReadFloat,
-                    BinaryUtils::GetDefaultValue<float>());
+                return ReadTopObject0<float>(IGNITE_TYPE_FLOAT, BinaryUtils::ReadFloat);
             }
 
             template <>
             double BinaryReaderImpl::ReadTopObject<double>()
             {
-                return ReadTopObject0(IGNITE_TYPE_DOUBLE, BinaryUtils::ReadDouble,
-                    BinaryUtils::GetDefaultValue<double>());
+                return ReadTopObject0<double>(IGNITE_TYPE_DOUBLE, BinaryUtils::ReadDouble);
             }
 
             template <>
             Guid BinaryReaderImpl::ReadTopObject<Guid>()
             {
-                int8_t typeId = stream->ReadInt8();
-
-                if (typeId == IGNITE_TYPE_UUID)
-                    return BinaryUtils::ReadGuid(stream);
-                else if (typeId == IGNITE_HDR_NULL)
-                    return BinaryUtils::GetDefaultValue<Guid>();
-                else {
-                    int32_t pos = stream->Position() - 1;
-
-                    IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY, "Invalid header", "position", pos, "expected", (int)IGNITE_TYPE_UUID, "actual", (int)typeId)
-                }
+                return ReadTopObject0<Guid>(IGNITE_TYPE_UUID, BinaryUtils::ReadGuid);
             }
 
             template <>
@@ -775,23 +813,69 @@ namespace ignite
                 else {
                     int32_t pos = stream->Position() - 1;
 
-                    IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY, "Invalid header", "position", pos, "expected", (int)IGNITE_TYPE_DATE, "actual", (int)typeId)
+                    IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY,
+                        "Invalid header", "position", pos, "expected", (int)IGNITE_TYPE_DATE, "actual", (int)typeId)
                 }
             }
 
             template <>
             Timestamp BinaryReaderImpl::ReadTopObject<Timestamp>()
             {
+                return ReadTopObject0<Timestamp>(IGNITE_TYPE_TIMESTAMP, BinaryUtils::ReadTimestamp);
+            }
+
+            template<>
+            Time BinaryReaderImpl::ReadTopObject<Time>()
+            {
+                return ReadTopObject0<Time>(IGNITE_TYPE_TIME, BinaryUtils::ReadTime);
+            }
+
+            template<>
+            std::string BinaryReaderImpl::ReadTopObject<std::string>()
+            {
+                int8_t typeId = stream->ReadInt8();
+
+                if (typeId == IGNITE_TYPE_STRING)
+                {
+                    int32_t realLen = stream->ReadInt32();
+
+                    std::string res;
+
+                    if (realLen > 0)
+                    {
+                        res.resize(realLen, 0);
+
+                        stream->ReadInt8Array(reinterpret_cast<int8_t*>(&res[0]), realLen);
+                    }
+
+                    return res;
+                }
+                else if (typeId == IGNITE_HDR_NULL)
+                    return std::string();
+                else
+                {
+                    int32_t pos = stream->Position() - 1;
+
+                    IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY, "Invalid header", "position", pos,
+                        "expected", static_cast<int>(IGNITE_TYPE_STRING), "actual", static_cast<int>(typeId))
+                }
+            }
+
+            template <typename T>
+            T BinaryReaderImpl::ReadTopObject0(const int8_t expHdr, T(*func)(ignite::impl::interop::InteropInputStream*))
+            {
                 int8_t typeId = stream->ReadInt8();
 
-                if (typeId == IGNITE_TYPE_TIMESTAMP)
-                    return BinaryUtils::ReadTimestamp(stream);
+                if (typeId == expHdr)
+                    return func(stream);
                 else if (typeId == IGNITE_HDR_NULL)
-                    return BinaryUtils::GetDefaultValue<Timestamp>();
-                else {
+                    return BinaryUtils::GetDefaultValue<T>();
+                else
+                {
                     int32_t pos = stream->Position() - 1;
 
-                    IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY, "Invalid header", "position", pos, "expected", (int)IGNITE_TYPE_TIMESTAMP, "actual", (int)typeId)
+                    IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY,
+                        "Invalid header", "position", pos, "expected", (int)expHdr, "actual", (int)typeId)
                 }
             }
 
@@ -814,9 +898,7 @@ namespace ignite
                         {
                             int32_t currentFieldId = stream->ReadInt32(schemaPos);
 
-                            if (fieldId != currentFieldId)
-                                continue;
-                            else
+                            if (fieldId == currentFieldId)
                                 return static_cast<uint8_t>(stream->ReadInt8(schemaPos + 4)) + pos;
                         }
                         break;
@@ -828,9 +910,7 @@ namespace ignite
                         {
                             int32_t currentFieldId = stream->ReadInt32(schemaPos);
 
-                            if (fieldId != currentFieldId)
-                                continue;
-                            else
+                            if (fieldId == currentFieldId)
                                 return static_cast<uint16_t>(stream->ReadInt16(schemaPos + 4)) + pos;
                         }
                         break;
@@ -842,9 +922,7 @@ namespace ignite
                         {
                             int32_t currentFieldId = stream->ReadInt32(schemaPos);
 
-                            if (fieldId != currentFieldId)
-                                continue;
-                            else
+                            if (fieldId == currentFieldId)
                                 return stream->ReadInt32(schemaPos + 4) + pos;
                         }
                         break;
@@ -867,10 +945,12 @@ namespace ignite
             void BinaryReaderImpl::CheckSingleMode(bool expected) const
             {
                 if (expected && elemId != 0) {
-                    IGNITE_ERROR_1(IgniteError::IGNITE_ERR_BINARY, "Operation cannot be performed when container is being read.");
+                    IGNITE_ERROR_1(IgniteError::IGNITE_ERR_BINARY,
+                        "Operation cannot be performed when container is being read.");
                 }
                 else if (!expected && elemId == 0) {
-                    IGNITE_ERROR_1(IgniteError::IGNITE_ERR_BINARY, "Operation can be performed only when container is being read.");
+                    IGNITE_ERROR_1(IgniteError::IGNITE_ERR_BINARY,
+                        "Operation can be performed only when container is being read.");
                 }
             }
 
@@ -917,13 +997,16 @@ namespace ignite
             void BinaryReaderImpl::CheckSession(int32_t expSes) const
             {
                 if (elemId != expSes) {
-                    IGNITE_ERROR_1(IgniteError::IGNITE_ERR_BINARY, "Containter read session has been finished or is not started yet.");
+                    IGNITE_ERROR_1(IgniteError::IGNITE_ERR_BINARY,
+                        "Containter read session has been finished or is not started yet.");
                 }
             }
 
             void BinaryReaderImpl::ThrowOnInvalidHeader(int32_t pos, int8_t expHdr, int8_t hdr)
             {
-                IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY, "Invalid header", "position", pos, "expected", (int)expHdr, "actual", (int)hdr)
+                IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY,
+                    "Invalid header", "position", pos, "expected",
+                    static_cast<int>(expHdr), "actual", static_cast<int>(hdr))
             }
 
             void BinaryReaderImpl::ThrowOnInvalidHeader(int8_t expHdr, int8_t hdr) const

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/src/impl/binary/binary_utils.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_utils.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_utils.cpp
index 5fe8d83..a583128 100644
--- a/modules/platforms/cpp/binary/src/impl/binary/binary_utils.cpp
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_utils.cpp
@@ -328,6 +328,18 @@ namespace ignite
                 stream->WriteInt32(val.GetSecondFraction() % 1000000);
             }
 
+            Time BinaryUtils::ReadTime(interop::InteropInputStream* stream)
+            {
+                int64_t ms = stream->ReadInt64();
+
+                return Time(ms);
+            }
+
+            void BinaryUtils::WriteTime(interop::InteropOutputStream* stream, const Time val)
+            {
+                stream->WriteInt64(val.GetMilliseconds());
+            }
+
             void BinaryUtils::WriteString(interop::InteropOutputStream* stream, const char* val, const int32_t len)
             {
                 stream->WriteInt32(len);

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/binary/src/impl/binary/binary_writer_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_writer_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_writer_impl.cpp
index b8f95b3..15e1ab9 100644
--- a/modules/platforms/cpp/binary/src/impl/binary/binary_writer_impl.cpp
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_writer_impl.cpp
@@ -227,10 +227,8 @@ namespace ignite
 
                     for (int i = 0; i < len; i++)
                     {
-                        Guid elem = *(val + i);
-
                         stream->WriteInt8(IGNITE_TYPE_UUID);
-                        BinaryUtils::WriteGuid(stream, elem);
+                        BinaryUtils::WriteGuid(stream, val[i]);
                     }
                 }
                 else
@@ -262,11 +260,7 @@ namespace ignite
                     stream->WriteInt32(len);
 
                     for (int i = 0; i < len; i++)
-                    {
-                        Guid elem = *(val + i);
-
-                        WriteTopObject(elem);
-                    }
+                        WriteTopObject(val[i]);
                 }
                 else
                 {
@@ -296,10 +290,8 @@ namespace ignite
 
                     for (int i = 0; i < len; i++)
                     {
-                        const Date& elem = *(val + i);
-
                         stream->WriteInt8(IGNITE_TYPE_DATE);
-                        BinaryUtils::WriteDate(stream, elem);
+                        BinaryUtils::WriteDate(stream, val[i]);
                     }
                 }
                 else
@@ -331,11 +323,7 @@ namespace ignite
                     stream->WriteInt32(len);
 
                     for (int i = 0; i < len; i++)
-                    {
-                        const Date& elem = *(val + i);
-
-                        WriteTopObject(elem);
-                    }
+                        WriteTopObject(val[i]);
                 }
                 else
                     stream->WriteInt8(IGNITE_HDR_NULL);
@@ -363,10 +351,8 @@ namespace ignite
 
                     for (int i = 0; i < len; i++)
                     {
-                        const Timestamp& elem = *(val + i);
-
                         stream->WriteInt8(IGNITE_TYPE_TIMESTAMP);
-                        BinaryUtils::WriteTimestamp(stream, elem);
+                        BinaryUtils::WriteTimestamp(stream, val[i]);
                     }
                 }
                 else
@@ -398,16 +384,73 @@ namespace ignite
                     stream->WriteInt32(len);
 
                     for (int i = 0; i < len; i++)
-                    {
-                        const Timestamp& elem = *(val + i);
+                        WriteTopObject(val[i]);
+                }
+                else
+                    stream->WriteInt8(IGNITE_HDR_NULL);
+            }
+
+            void BinaryWriterImpl::WriteTime(const Time& val)
+            {
+                CheckRawMode(true);
+                CheckSingleMode(true);
+
+                stream->WriteInt8(IGNITE_TYPE_TIME);
+
+                BinaryUtils::WriteTime(stream, val);
+            }
+
+            void BinaryWriterImpl::WriteTimeArray(const Time* val, const int32_t len)
+            {
+                CheckRawMode(true);
+                CheckSingleMode(true);
+
+                if (val)
+                {
+                    stream->WriteInt8(IGNITE_TYPE_ARRAY_TIME);
+                    stream->WriteInt32(len);
 
-                        WriteTopObject(elem);
+                    for (int i = 0; i < len; i++)
+                    {
+                        stream->WriteInt8(IGNITE_TYPE_TIME);
+                        BinaryUtils::WriteTime(stream, val[i]);
                     }
                 }
                 else
                     stream->WriteInt8(IGNITE_HDR_NULL);
             }
 
+            void BinaryWriterImpl::WriteTime(const char* fieldName, const Time& val)
+            {
+                CheckRawMode(false);
+                CheckSingleMode(true);
+
+                WriteFieldId(fieldName, IGNITE_TYPE_TIME);
+
+                stream->WriteInt8(IGNITE_TYPE_TIME);
+
+                BinaryUtils::WriteTime(stream, val);
+            }
+
+            void BinaryWriterImpl::WriteTimeArray(const char* fieldName, const Time* val, const int32_t len)
+            {
+                CheckRawMode(false);
+                CheckSingleMode(true);
+
+                WriteFieldId(fieldName, IGNITE_TYPE_ARRAY_TIME);
+
+                if (val)
+                {
+                    stream->WriteInt8(IGNITE_TYPE_ARRAY_TIME);
+                    stream->WriteInt32(len);
+
+                    for (int i = 0; i < len; i++)
+                        WriteTopObject(val[i]);
+                }
+                else
+                    stream->WriteInt8(IGNITE_HDR_NULL);
+            }
+
             void BinaryWriterImpl::WriteString(const char* val, const int32_t len)
             {
                 CheckRawMode(true);
@@ -705,6 +748,24 @@ namespace ignite
                 WriteTopObject0<Timestamp>(obj, BinaryUtils::WriteTimestamp, IGNITE_TYPE_TIMESTAMP);
             }
 
+            template <>
+            void BinaryWriterImpl::WriteTopObject<Time>(const Time& obj)
+            {
+                WriteTopObject0<Time>(obj, BinaryUtils::WriteTime, IGNITE_TYPE_TIME);
+            }
+
+            template<>
+            void BinaryWriterImpl::WriteTopObject(const std::string& obj)
+            {
+                const char* obj0 = obj.c_str();
+
+                int32_t len = static_cast<int32_t>(obj.size());
+
+                stream->WriteInt8(IGNITE_TYPE_STRING);
+
+                BinaryUtils::WriteString(stream, obj0, len);
+            }
+
             void BinaryWriterImpl::PostWrite()
             {
                 int32_t lenWithoutSchema = stream->Position() - start;

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/common/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/Makefile.am b/modules/platforms/cpp/common/Makefile.am
index c4f4827..8515d5b 100644
--- a/modules/platforms/cpp/common/Makefile.am
+++ b/modules/platforms/cpp/common/Makefile.am
@@ -53,7 +53,8 @@ libignite_common_la_SOURCES = \
     src/date.cpp \
     src/ignite_error.cpp \
     src/guid.cpp \
-    src/timestamp.cpp
+    src/timestamp.cpp \
+    src/time.cpp
 
 clean-local:
 	$(RM) *.gcno *.gcda

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/common/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/Makefile.am b/modules/platforms/cpp/common/include/Makefile.am
index 7f7ab83..cba68e9 100644
--- a/modules/platforms/cpp/common/include/Makefile.am
+++ b/modules/platforms/cpp/common/include/Makefile.am
@@ -32,6 +32,7 @@ nobase_include_HEADERS = \
 	ignite/guid.h \
 	ignite/ignite_error.h \
 	ignite/timestamp.h \
+	ignite/time.h \
 	ignite/reference.h
 
 uninstall-hook:

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/common/include/ignite/common/utils.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/utils.h b/modules/platforms/cpp/common/include/ignite/common/utils.h
index bf1f39b..32d250f 100644
--- a/modules/platforms/cpp/common/include/ignite/common/utils.h
+++ b/modules/platforms/cpp/common/include/ignite/common/utils.h
@@ -29,6 +29,7 @@
 
 #include <ignite/date.h>
 #include <ignite/timestamp.h>
+#include "ignite/time.h"
 
 #ifdef IGNITE_FRIEND
 #   define IGNITE_FRIEND_EXPORT IGNITE_EXPORT
@@ -217,7 +218,7 @@ namespace ignite
         }
 
         /**
-         * Convert standard C type time_t to Date struct tm.
+         * Convert standard C type time_t to Date.
          *
          * @param ctime Standard C type time_t.
          * @return Corresponding value of Date.
@@ -228,6 +229,17 @@ namespace ignite
         }
 
         /**
+         * Convert standard C type time_t to Time.
+         *
+         * @param ctime Standard C type time_t.
+         * @return Corresponding value of Time.
+         */
+        inline Time CTimeToTime(time_t ctime)
+        {
+            return Time(ctime * 1000);
+        }
+
+        /**
          * Convert standard C type time_t to Timestamp type.
          *
          * @param ctime Standard C type time_t.
@@ -253,6 +265,19 @@ namespace ignite
         }
 
         /**
+         * Convert standard C type struct tm to Time type.
+         *
+         * @param ctime Standard C type struct tm.
+         * @return Corresponding value of Time.
+         */
+        inline Time CTmToTime(const tm& ctime)
+        {
+            time_t time = common::IgniteTimeGm(ctime);
+
+            return CTimeToTime(time);
+        }
+
+        /**
          * Convert standard C type struct tm to Timestamp type.
          *
          * @param ctime Standard C type struct tm.
@@ -299,7 +324,31 @@ namespace ignite
             int day = 1, int hour = 0, int min = 0, int sec = 0);
 
         /**
-         * Make Date in human understandable way.
+         * Make Time in human understandable way.
+         *
+         * Created Time uses GMT timezone.
+         *
+         * @param hour Hour.
+         * @param min Minute.
+         * @param sec Second.
+         * @return Time.
+         */
+        Time MakeTimeGmt(int hour = 0, int min = 0, int sec = 0);
+
+        /**
+         * Make Time in human understandable way.
+         *
+         * Created Time uses Local timezone.
+         *
+         * @param hour Hour.
+         * @param min Minute.
+         * @param sec Second.
+         * @return Time.
+         */
+        Time MakeTimeLocal(int hour = 0, int min = 0, int sec = 0);
+
+        /**
+         * Make Timestamp in human understandable way.
          *
          * Created Timestamp uses GMT timezone.
          *

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/common/include/ignite/time.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/time.h b/modules/platforms/cpp/common/include/ignite/time.h
new file mode 100644
index 0000000..b922c6a
--- /dev/null
+++ b/modules/platforms/cpp/common/include/ignite/time.h
@@ -0,0 +1,138 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::Time class.
+ */
+
+#ifndef _IGNITE_TIME
+#define _IGNITE_TIME
+
+#include <stdint.h>
+
+#include <ignite/common/common.h>
+
+namespace ignite
+{
+    /**
+     * %Time type.
+     */
+    class IGNITE_IMPORT_EXPORT Time
+    {
+    public:
+        /**
+         * Default constructor.
+         */
+        Time();
+
+        /**
+         * Copy constructor.
+         *
+         * @param another Another instance.
+         */
+        Time(const Time& another);
+
+        /**
+         * Constructor.
+         *
+         * @param ms Time in milliseconds since 00:00:00 UTC.
+         */
+        Time(int64_t ms);
+
+        /**
+         * Copy operator.
+         *
+         * @param another Another instance.
+         * @return This.
+         */
+        Time& operator=(const Time& another);
+
+        /**
+         * Returns number of milliseconds since 00:00:00 UTC.
+         *
+         * @return Number of milliseconds since 00:00:00 UTC.
+         */
+        int64_t GetMilliseconds() const;
+
+        /**
+         * Returns number of seconds since 00:00:00 UTC.
+         *
+         * @return Number of seconds since 00:00:00 UTC.
+         */
+        int64_t GetSeconds() const;
+
+        /**
+         * Comparison operator override.
+         *
+         * @param val1 First value.
+         * @param val2 Second value.
+         * @return True if equal.
+         */
+        friend bool IGNITE_IMPORT_EXPORT operator==(const Time& val1, const Time& val2);
+
+        /**
+         * Comparison operator override.
+         *
+         * @param val1 First value.
+         * @param val2 Second value.
+         * @return True if not equal.
+         */
+        friend bool IGNITE_IMPORT_EXPORT operator!=(const Time& val1, const Time& val2);
+
+        /**
+         * Comparison operator override.
+         *
+         * @param val1 First value.
+         * @param val2 Second value.
+         * @return True if less.
+         */
+        friend bool IGNITE_IMPORT_EXPORT operator<(const Time& val1, const Time& val2);
+
+        /**
+         * Comparison operator override.
+         *
+         * @param val1 First value.
+         * @param val2 Second value.
+         * @return True if less or equal.
+         */
+        friend bool IGNITE_IMPORT_EXPORT operator<=(const Time& val1, const Time& val2);
+
+        /**
+         * Comparison operator override.
+         *
+         * @param val1 First value.
+         * @param val2 Second value.
+         * @return True if gretter.
+         */
+        friend bool IGNITE_IMPORT_EXPORT operator>(const Time& val1, const Time& val2);
+
+        /**
+         * Comparison operator override.
+         *
+         * @param val1 First value.
+         * @param val2 Second value.
+         * @return True if gretter or equal.
+         */
+        friend bool IGNITE_IMPORT_EXPORT operator>=(const Time& val1, const Time& val2);
+    private:
+        /** Time in milliseconds since 00:00:00 UTC. */
+        int64_t milliseconds;
+    };
+}
+
+#endif //_IGNITE_TIME

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/common/project/vs/common.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj b/modules/platforms/cpp/common/project/vs/common.vcxproj
index 45ac15e..3062112 100644
--- a/modules/platforms/cpp/common/project/vs/common.vcxproj
+++ b/modules/platforms/cpp/common/project/vs/common.vcxproj
@@ -179,6 +179,7 @@
     <ClInclude Include="..\..\include\ignite\guid.h" />
     <ClInclude Include="..\..\include\ignite\ignite_error.h" />
     <ClInclude Include="..\..\include\ignite\reference.h" />
+    <ClInclude Include="..\..\include\ignite\time.h" />
     <ClInclude Include="..\..\include\ignite\timestamp.h" />
     <ClInclude Include="..\..\os\win\include\ignite\common\common.h" />
     <ClInclude Include="..\..\os\win\include\ignite\common\concurrent_os.h" />
@@ -196,6 +197,7 @@
     <ClCompile Include="..\..\src\date.cpp" />
     <ClCompile Include="..\..\src\guid.cpp" />
     <ClCompile Include="..\..\src\ignite_error.cpp" />
+    <ClCompile Include="..\..\src\time.cpp" />
     <ClCompile Include="..\..\src\timestamp.cpp" />
   </ItemGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj.filters b/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
index 1613d5d..ed709f7 100644
--- a/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
+++ b/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
@@ -64,6 +64,9 @@
     <ClInclude Include="..\..\include\ignite\common\platform_utils.h">
       <Filter>Code\common</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\time.h">
+      <Filter>Code</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <ClCompile Include="..\..\src\date.cpp">
@@ -102,5 +105,8 @@
     <ClCompile Include="..\..\src\common\utils.cpp">
       <Filter>Code\common</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\time.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/common/src/common/utils.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/common/utils.cpp b/modules/platforms/cpp/common/src/common/utils.cpp
index 2b425c1..786d0fb 100644
--- a/modules/platforms/cpp/common/src/common/utils.cpp
+++ b/modules/platforms/cpp/common/src/common/utils.cpp
@@ -119,6 +119,36 @@ namespace ignite
             return CTimeToDate(localTime);
         }
 
+        Time MakeTimeGmt(int hour, int min, int sec)
+        {
+            tm date = { 0 };
+
+            date.tm_year = 70;
+            date.tm_mon = 0;
+            date.tm_mday = 1;
+            date.tm_hour = hour;
+            date.tm_min = min;
+            date.tm_sec = sec;
+
+            return CTmToTime(date);
+        }
+
+        Time MakeTimeLocal(int hour, int min, int sec)
+        {
+            tm date = { 0 };
+
+            date.tm_year = 70;
+            date.tm_mon = 0;
+            date.tm_mday = 1;
+            date.tm_hour = hour;
+            date.tm_min = min;
+            date.tm_sec = sec;
+
+            time_t localTime = common::IgniteTimeLocal(date);
+
+            return CTimeToTime(localTime);
+        }
+
         Timestamp MakeTimestampGmt(int year, int month, int day,
             int hour, int min, int sec, long ns)
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/common/src/time.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/time.cpp b/modules/platforms/cpp/common/src/time.cpp
new file mode 100644
index 0000000..a37a9c9
--- /dev/null
+++ b/modules/platforms/cpp/common/src/time.cpp
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+
+#include "ignite/time.h"
+
+namespace ignite
+{
+    Time::Time() : milliseconds(0)
+    {
+        // No-op.
+    }
+
+    Time::Time(const Time& another) : milliseconds(another.milliseconds)
+    {
+        // No-op.
+    }
+
+    Time::Time(int64_t ms) : milliseconds(ms)
+    {
+        // No-op.
+    }
+
+    Time& Time::operator=(const Time& another)
+    {
+        milliseconds = another.milliseconds;
+
+        return *this;
+    }
+
+    int64_t Time::GetMilliseconds() const
+    {
+        return milliseconds;
+    }
+
+    int64_t Time::GetSeconds() const
+    {
+        return milliseconds / 1000;
+    }
+
+    bool operator==(const Time& val1, const Time& val2)
+    {
+        return val1.milliseconds == val2.milliseconds;
+    }
+
+    bool operator!=(const Time& val1, const Time& val2)
+    {
+        return val1.milliseconds != val2.milliseconds;
+    }
+
+    bool operator<(const Time& val1, const Time& val2)
+    {
+        return val1.milliseconds < val2.milliseconds;
+    }
+
+    bool operator<=(const Time& val1, const Time& val2)
+    {
+        return val1.milliseconds <= val2.milliseconds;
+    }
+
+    bool operator>(const Time& val1, const Time& val2)
+    {
+        return val1.milliseconds > val2.milliseconds;
+    }
+
+    bool operator>=(const Time& val1, const Time& val2)
+    {
+        return val1.milliseconds >= val2.milliseconds;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/Makefile.am b/modules/platforms/cpp/core-test/Makefile.am
index 873db71..25dee58 100644
--- a/modules/platforms/cpp/core-test/Makefile.am
+++ b/modules/platforms/cpp/core-test/Makefile.am
@@ -73,6 +73,7 @@ ignite_tests_SOURCES = \
     src/binary_reader_writer_raw_test.cpp \
     src/binary_reader_writer_test.cpp \
     src/binary_session_test.cpp \
+    src/date_time_test.cpp \
     src/decimal_test.cpp \
     src/dynamic_size_array_test.cpp \
     src/fixed_size_array_test.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/config/cache-query.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-query.xml b/modules/platforms/cpp/core-test/config/cache-query.xml
index bb18f7c..036c464 100644
--- a/modules/platforms/cpp/core-test/config/cache-query.xml
+++ b/modules/platforms/cpp/core-test/config/cache-query.xml
@@ -99,6 +99,29 @@
                         </list>
                     </property>
                 </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="TimeCache"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+
+                    <property name="affinity">
+                        <bean class="org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction">
+                            <property name="partitions" value="256"/>
+                        </bean>
+                    </property>
+                    
+                    <!-- Configure type metadata to enable queries. -->
+                    <property name="queryEntities">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryEntity">
+                                <property name="keyType" value="java.lang.Integer"/>
+                                <property name="valueType" value="java.sql.Time"/>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
             </list>
         </property>
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/include/ignite/binary_test_utils.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/include/ignite/binary_test_utils.h b/modules/platforms/cpp/core-test/include/ignite/binary_test_utils.h
index bcec9fe..7701253 100644
--- a/modules/platforms/cpp/core-test/include/ignite/binary_test_utils.h
+++ b/modules/platforms/cpp/core-test/include/ignite/binary_test_utils.h
@@ -30,6 +30,11 @@ namespace ignite_test
     {
         namespace binary
         {
+            inline bool IsBinaryError(const IgniteError& err)
+            {
+                return err.GetCode() == IgniteError::IGNITE_ERR_BINARY;
+            }
+
             template<typename T>
             inline void Write(BinaryRawWriter& writer, T val)
             {
@@ -163,6 +168,18 @@ namespace ignite_test
             }
 
             template<>
+            inline void Write(BinaryRawWriter& writer, Time val)
+            {
+                writer.WriteTime(val);
+            }
+
+            template<>
+            inline Time Read(BinaryRawReader& reader)
+            {
+                return reader.ReadTime();
+            }
+
+            template<>
             inline void Write(BinaryRawWriter& writer, Timestamp val)
             {
                 writer.WriteTimestamp(val);
@@ -307,6 +324,18 @@ namespace ignite_test
             }
 
             template<>
+            inline void WriteArray(BinaryRawWriter& writer, Time* val, int32_t len)
+            {
+                writer.WriteTimeArray(val, len);
+            }
+
+            template<>
+            inline int32_t ReadArray(BinaryRawReader& reader, Time* val, int32_t len)
+            {
+                return reader.ReadTimeArray(val, len);
+            }
+
+            template<>
             inline void WriteArray(BinaryRawWriter& writer, Timestamp* val, int32_t len)
             {
                 writer.WriteTimestampArray(val, len);
@@ -451,6 +480,18 @@ namespace ignite_test
             }
 
             template<>
+            inline void Write(BinaryWriter& writer, const char* fieldName, Time val)
+            {
+                writer.WriteTime(fieldName, val);
+            }
+
+            template<>
+            inline Time Read(BinaryReader& reader, const char* fieldName)
+            {
+                return reader.ReadTime(fieldName);
+            }
+
+            template<>
             inline void Write(BinaryWriter& writer, const char* fieldName, Timestamp val)
             {
                 writer.WriteTimestamp(fieldName, val);
@@ -595,6 +636,18 @@ namespace ignite_test
             }
 
             template<>
+            inline void WriteArray(BinaryWriter& writer, const char* fieldName, Time* val, int32_t len)
+            {
+                writer.WriteTimeArray(fieldName, val, len);
+            }
+
+            template<>
+            inline int32_t ReadArray(BinaryReader& reader, const char* fieldName, Time* val, int32_t len)
+            {
+                return reader.ReadTimeArray(fieldName, val, len);
+            }
+
+            template<>
             inline void WriteArray(BinaryWriter& writer, const char* fieldName, Timestamp* val, int32_t len)
             {
                 writer.WriteTimestampArray(fieldName, val, len);

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/include/ignite/test_type.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/include/ignite/test_type.h b/modules/platforms/cpp/core-test/include/ignite/test_type.h
index b399afe..7c7e6a2 100644
--- a/modules/platforms/cpp/core-test/include/ignite/test_type.h
+++ b/modules/platforms/cpp/core-test/include/ignite/test_type.h
@@ -37,6 +37,7 @@ namespace ignite
             doubleField(0.0),
             boolField(false),
             dateField(),
+            timeField(),
             timestampField()
         {
             // No-op.
@@ -45,7 +46,7 @@ namespace ignite
         TestType(int8_t i8Field, int16_t i16Field, int32_t i32Field,
             int64_t i64Field, const std::string& strField, float floatField,
             double doubleField, bool boolField, const Guid& guidField,
-            const Date& dateField, const Timestamp& timestampField) :
+            const Date& dateField, const Time& timeField, const Timestamp& timestampField) :
             allNulls(false),
             i8Field(i8Field),
             i16Field(i16Field),
@@ -57,6 +58,7 @@ namespace ignite
             boolField(boolField),
             guidField(guidField),
             dateField(dateField),
+            timeField(timeField),
             timestampField(timestampField)
         {
             // No-op.
@@ -76,6 +78,7 @@ namespace ignite
                 one.boolField == two.boolField &&
                 one.guidField == two.guidField &&
                 one.dateField == two.dateField &&
+                one.timeField == two.timeField &&
                 one.timestampField == two.timestampField &&
                 one.i8ArrayField == two.i8ArrayField;
         }
@@ -91,6 +94,7 @@ namespace ignite
         bool boolField;
         Guid guidField;
         Date dateField;
+        Time timeField;
         Timestamp timestampField;
         std::vector<int8_t> i8ArrayField;
     };
@@ -125,6 +129,7 @@ namespace ignite
                     writer.WriteBool("boolField", obj.boolField);
                     writer.WriteGuid("guidField", obj.guidField);
                     writer.WriteDate("dateField", obj.dateField);
+                    writer.WriteTime("timeField", obj.timeField);
                     writer.WriteTimestamp("timestampField", obj.timestampField);
                     if (obj.i8ArrayField.empty())
                     {
@@ -147,6 +152,7 @@ namespace ignite
                     writer.WriteNull("boolField");
                     writer.WriteNull("guidField");
                     writer.WriteNull("dateField");
+                    writer.WriteNull("timeField");
                     writer.WriteNull("timestampField");
                     writer.WriteNull("i8ArrayField");
                 }
@@ -164,11 +170,12 @@ namespace ignite
                 bool boolField = reader.ReadBool("boolField");
                 Guid guidField = reader.ReadGuid("guidField");
                 Date dateField = reader.ReadDate("dateField");
+                Time timeField = reader.ReadTime("timeField");
                 Timestamp timestampField = reader.ReadTimestamp("timestampField");
 
                 TestType result(i8Field, i16Field, i32Field, i64Field, strField,
                     floatField, doubleField, boolField, guidField, dateField,
-                    timestampField);
+                    timeField, timestampField);
 
                 int32_t len = reader.ReadInt8Array("i8ArrayField", 0, 0);
                 if (len > 0)

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
index 4d252b9..f22e885 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
@@ -50,6 +50,7 @@
     <ClCompile Include="..\..\src\cluster_test.cpp" />
     <ClCompile Include="..\..\src\cache_invoke_test.cpp" />
     <ClCompile Include="..\..\src\concurrent_test.cpp" />
+    <ClCompile Include="..\..\src\date_time_test.cpp" />
     <ClCompile Include="..\..\src\decimal_test.cpp" />
     <ClCompile Include="..\..\src\dynamic_size_array_test.cpp" />
     <ClCompile Include="..\..\src\fixed_size_array_test.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
index c318644..fb0be1b 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
@@ -79,6 +79,9 @@
     <ClCompile Include="..\..\src\test_utils.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\date_time_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
     <ClCompile Include="..\..\src\cache_store_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>

http://git-wip-us.apache.org/repos/asf/ignite/blob/02991d30/modules/platforms/cpp/core-test/src/binary_object_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/binary_object_test.cpp b/modules/platforms/cpp/core-test/src/binary_object_test.cpp
index fb3725f..6378266 100644
--- a/modules/platforms/cpp/core-test/src/binary_object_test.cpp
+++ b/modules/platforms/cpp/core-test/src/binary_object_test.cpp
@@ -112,119 +112,12 @@ void CheckData(const T& obj)
 
 BOOST_AUTO_TEST_SUITE(BinaryObjectTestSuite)
 
-#ifdef CHECK_BINARY_OBJECT_WITH_PRIMITIVES
-
-BOOST_AUTO_TEST_CASE(PrimitiveInt8)
-{
-    CheckSimple<int8_t>(0);
-    CheckSimple<int8_t>(INT8_MAX);
-    CheckSimple<int8_t>(INT8_MIN);
-    CheckSimple<int8_t>(42);
-    CheckSimple<int8_t>(-12);
-    CheckSimple<int8_t>(0x7D);
-}
-
-BOOST_AUTO_TEST_CASE(PrimitiveInt16)
-{
-    CheckSimple<int32_t>(0);
-    CheckSimple<int32_t>(INT16_MAX);
-    CheckSimple<int32_t>(INT16_MIN);
-    CheckSimple<int32_t>(42);
-    CheckSimple<int32_t>(12321);
-    CheckSimple<int32_t>(0x7AB0);
-}
-
-BOOST_AUTO_TEST_CASE(PrimitiveInt32)
-{
-    CheckSimple<int32_t>(0);
-    CheckSimple<int32_t>(INT32_MAX);
-    CheckSimple<int32_t>(INT32_MIN);
-    CheckSimple<int32_t>(42);
-    CheckSimple<int32_t>(1337);
-    CheckSimple<int32_t>(0xA2496BC9);
-}
-
-BOOST_AUTO_TEST_CASE(PrimitiveInt64)
-{
-    CheckSimple<int64_t>(0);
-    CheckSimple<int64_t>(INT64_MAX);
-    CheckSimple<int64_t>(INT64_MIN);
-    CheckSimple<int64_t>(42);
-    CheckSimple<int64_t>(13371337133713371337LL);
-    CheckSimple<int64_t>(0xA928673F501CC09E);
-}
-
-BOOST_AUTO_TEST_CASE(PrimitiveBool)
-{
-    CheckSimple<bool>(true);
-    CheckSimple<bool>(false);
-}
-
-BOOST_AUTO_TEST_CASE(PrimitiveFloat)
-{
-    CheckSimple<float>(0.0);
-    CheckSimple<float>(1E38f);
-    CheckSimple<float>(-1E38f);
-    CheckSimple<float>(1E-38f);
-    CheckSimple<float>(-1E-38f);
-    CheckSimple<float>(42.0f);
-    CheckSimple<float>(42.42f);
-    CheckSimple<float>(1337.1337f);
-}
-
-BOOST_AUTO_TEST_CASE(PrimitiveDouble)
-{
-    CheckSimple<double>(0);
-    CheckSimple<double>(1E127);
-    CheckSimple<double>(-1E127);
-    CheckSimple<double>(1E-127);
-    CheckSimple<double>(-1E-127);
-    CheckSimple<double>(42);
-    CheckSimple<double>(42.42);
-    CheckSimple<double>(1337.1337 * 1337.1337);
-}
-
-BOOST_AUTO_TEST_CASE(PrimitiveString)
-{
-    CheckSimple<std::string>("");
-    CheckSimple<std::string>("Lorem ipsum");
-    CheckSimple<std::string>("Lorem ipsum dolor sit amet, consectetur adipiscing elit, sed do "
-        "eiusmod tempor incididunt ut labore et dolore magna aliqua. Ut enim ad minim veniam, "
-        "quis nostrud exercitation");
-
-    CheckSimple<std::string>(std::string(1000, '.'));
-}
-
-BOOST_AUTO_TEST_CASE(PrimitiveGuid)
-{
-    CheckSimple<Guid>(Guid(0, 0));
-    CheckSimple<Guid>(Guid(0xFFFFFFFFFFFFFFFF, 0xFFFFFFFFFFFFFFFF));
-    CheckSimple<Guid>(Guid(0x4F9039DEF0FB8000, 0x905AE8A2D6FD49C1));
-}
-
-BOOST_AUTO_TEST_CASE(PrimitiveDate)
-{
-    CheckSimpleNP<Date>(Date(0));
-    CheckSimpleNP<Date>(common::MakeDateGmt(1998, 12, 3, 18, 32, 01));
-    CheckSimpleNP<Date>(common::MakeDateGmt(2017, 1, 18, 20, 50, 41));
-    CheckSimpleNP<Date>(common::MakeDateLocal(1998, 12, 3, 18, 32, 01));
-}
-
-BOOST_AUTO_TEST_CASE(PrimitiveTimestamp)
-{
-    CheckSimpleNP<Timestamp>(Timestamp(0));
-    CheckSimpleNP<Timestamp>(common::MakeTimestampGmt(1998, 12, 3, 18, 32, 01, 593846589));
-    CheckSimpleNP<Timestamp>(common::MakeTimestampGmt(2017, 1, 18, 20, 50, 41, 920700532));
-    CheckSimpleNP<Timestamp>(common::MakeTimestampLocal(1998, 12, 3, 18, 32, 01, 2385));
-}
-
-#endif //CHECK_BINARY_OBJECT_WITH_PRIMITIVES
-
 BOOST_AUTO_TEST_CASE(UserTestType)
 {
     CheckSimpleNP(TestType());
     CheckSimpleNP(TestType(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9),
         common::MakeDateGmt(1987, 6, 5),
+        common::MakeTimeGmt(13, 32, 9),
         common::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)));
 }
 
@@ -270,6 +163,7 @@ BOOST_AUTO_TEST_CASE(UserTestTypeGetData)
     CheckData(TestType());
     CheckData(TestType(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9),
         common::MakeDateGmt(1987, 6, 5),
+        common::MakeTimeGmt(13, 32, 9),
         common::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)));
 }
 


[15/23] ignite git commit: IGNITE-4822 Fixed change jvm options for benchmarks

Posted by vo...@apache.org.
IGNITE-4822 Fixed change jvm options for benchmarks


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

Branch: refs/heads/ignite-4565-ddl
Commit: b4cc8a70e8a9497c599bddce44e8b95a82127f2d
Parents: 02991d3
Author: oleg-ostanin <oo...@gridgain.com>
Authored: Tue Mar 21 15:05:37 2017 +0300
Committer: oleg-ostanin <oo...@gridgain.com>
Committed: Tue Mar 21 15:05:37 2017 +0300

----------------------------------------------------------------------
 .../config/benchmark-atomic.properties          | 18 +++++++--------
 .../config/benchmark-bin-identity.properties    | 10 +++------
 .../config/benchmark-cache-load.properties      | 10 +++------
 .../config/benchmark-client-mode.properties     | 10 +++------
 .../config/benchmark-compute.properties         | 18 +++++++--------
 .../config/benchmark-failover.properties        | 18 ++++++---------
 .../yardstick/config/benchmark-full.properties  | 18 ++++++---------
 .../config/benchmark-multicast.properties       | 10 +++------
 .../config/benchmark-put-indexed-val.properties | 18 +++++++--------
 .../benchmark-query-put-separated.properties    | 10 +++------
 .../yardstick/config/benchmark-query.properties | 18 +++++++--------
 .../config/benchmark-remote-sample.properties   | 18 +++++++--------
 .../config/benchmark-remote.properties          | 18 ++++++---------
 .../config/benchmark-sample.properties          | 18 +++++++--------
 .../config/benchmark-sql-dml.properties         | 18 +++++++--------
 .../yardstick/config/benchmark-store.properties | 10 +++------
 .../yardstick/config/benchmark-tx.properties    | 18 +++++++--------
 modules/yardstick/config/benchmark.properties   | 18 ++++++---------
 .../test-max-int-values-offheap.properties      | 23 +++++++-------------
 .../test-max-int-values-onheap.properties       | 23 +++++++-------------
 .../config/test-max-int-values-swap.properties  | 23 +++++++-------------
 21 files changed, 134 insertions(+), 211 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-atomic.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-atomic.properties b/modules/yardstick/config/benchmark-atomic.properties
index 7226384..967a603 100644
--- a/modules/yardstick/config/benchmark-atomic.properties
+++ b/modules/yardstick/config/benchmark-atomic.properties
@@ -22,16 +22,14 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 # JVM_OPTS=${JVM_OPTS}" \
-#  -XX:+UseParNewGC \
-#  -XX:+UseConcMarkSweepGC \
-#  -XX:+UseTLAB \
-#  -XX:NewSize=128m \
-#  -XX:MaxNewSize=128m \
-#  -XX:MaxTenuringThreshold=0 \
-#  -XX:SurvivorRatio=1024 \
-#  -XX:+UseCMSInitiatingOccupancyOnly \
-#  -XX:CMSInitiatingOccupancyFraction=60 \
-#"
+# -Xms6g \
+# -Xmx6g \
+# -Xloggc:./gc${now0}.log \
+# -XX:+PrintGCDetails \
+# -verbose:gc \
+# -XX:+UseParNewGC \
+# -XX:+UseConcMarkSweepGC \
+# "
 
 # List of default probes.
 # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-bin-identity.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-bin-identity.properties b/modules/yardstick/config/benchmark-bin-identity.properties
index 6468d8f..d5d18b6 100644
--- a/modules/yardstick/config/benchmark-bin-identity.properties
+++ b/modules/yardstick/config/benchmark-bin-identity.properties
@@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
 -XX:+PrintGCDetails \
 -verbose:gc \
 -XX:+UseParNewGC \
 -XX:+UseConcMarkSweepGC \
--XX:+UseTLAB \
--XX:NewSize=128m \
--XX:MaxNewSize=128m \
--XX:MaxTenuringThreshold=0 \
--XX:SurvivorRatio=1024 \
--XX:+UseCMSInitiatingOccupancyOnly \
--XX:CMSInitiatingOccupancyFraction=60 \
 "
 
 #Ignite version

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-cache-load.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-cache-load.properties b/modules/yardstick/config/benchmark-cache-load.properties
index 12e07c3..9c1ab09 100644
--- a/modules/yardstick/config/benchmark-cache-load.properties
+++ b/modules/yardstick/config/benchmark-cache-load.properties
@@ -21,17 +21,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
 -XX:+PrintGCDetails \
 -verbose:gc \
 -XX:+UseParNewGC \
 -XX:+UseConcMarkSweepGC \
--XX:+UseTLAB \
--XX:NewSize=128m \
--XX:MaxNewSize=128m \
--XX:MaxTenuringThreshold=0 \
--XX:SurvivorRatio=1024 \
--XX:+UseCMSInitiatingOccupancyOnly \
--XX:CMSInitiatingOccupancyFraction=60 \
 "
 # List of default probes.
 # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-client-mode.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-client-mode.properties b/modules/yardstick/config/benchmark-client-mode.properties
index 0723fcb..ad501f1 100644
--- a/modules/yardstick/config/benchmark-client-mode.properties
+++ b/modules/yardstick/config/benchmark-client-mode.properties
@@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
 -XX:+PrintGCDetails \
 -verbose:gc \
 -XX:+UseParNewGC \
 -XX:+UseConcMarkSweepGC \
--XX:+UseTLAB \
--XX:NewSize=128m \
--XX:MaxNewSize=128m \
--XX:MaxTenuringThreshold=0 \
--XX:SurvivorRatio=1024 \
--XX:+UseCMSInitiatingOccupancyOnly \
--XX:CMSInitiatingOccupancyFraction=60 \
 "
 #Ignite version
 ver="RELEASE-"

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-compute.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-compute.properties b/modules/yardstick/config/benchmark-compute.properties
index 66cb930..df35c54 100644
--- a/modules/yardstick/config/benchmark-compute.properties
+++ b/modules/yardstick/config/benchmark-compute.properties
@@ -22,16 +22,14 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 # JVM_OPTS=${JVM_OPTS}" \
-#  -XX:+UseParNewGC \
-#  -XX:+UseConcMarkSweepGC \
-#  -XX:+UseTLAB \
-#  -XX:NewSize=128m \
-#  -XX:MaxNewSize=128m \
-#  -XX:MaxTenuringThreshold=0 \
-#  -XX:SurvivorRatio=1024 \
-#  -XX:+UseCMSInitiatingOccupancyOnly \
-#  -XX:CMSInitiatingOccupancyFraction=60 \
-#"
+# -Xms6g \
+# -Xmx6g \
+# -Xloggc:./gc${now0}.log \
+# -XX:+PrintGCDetails \
+# -verbose:gc \
+# -XX:+UseParNewGC \
+# -XX:+UseConcMarkSweepGC \
+# "
 
 # List of default probes.
 # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-failover.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-failover.properties b/modules/yardstick/config/benchmark-failover.properties
index 4c6a12a..a85d165 100644
--- a/modules/yardstick/config/benchmark-failover.properties
+++ b/modules/yardstick/config/benchmark-failover.properties
@@ -27,17 +27,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false -ea"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
-  -XX:+PrintGCDetails \
-  -verbose:gc \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=128m \
-  -XX:MaxNewSize=128m \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
 "
 
 #Ignite version

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-full.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-full.properties b/modules/yardstick/config/benchmark-full.properties
index b64612e..96da0a6 100644
--- a/modules/yardstick/config/benchmark-full.properties
+++ b/modules/yardstick/config/benchmark-full.properties
@@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
-  -XX:+PrintGCDetails \
-  -verbose:gc \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=128m \
-  -XX:MaxNewSize=128m \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
 "
 
 #Ignite version

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-multicast.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties
index c10d0c6..7f76495 100644
--- a/modules/yardstick/config/benchmark-multicast.properties
+++ b/modules/yardstick/config/benchmark-multicast.properties
@@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
 -XX:+PrintGCDetails \
 -verbose:gc \
 -XX:+UseParNewGC \
 -XX:+UseConcMarkSweepGC \
--XX:+UseTLAB \
--XX:NewSize=128m \
--XX:MaxNewSize=128m \
--XX:MaxTenuringThreshold=0 \
--XX:SurvivorRatio=1024 \
--XX:+UseCMSInitiatingOccupancyOnly \
--XX:CMSInitiatingOccupancyFraction=60 \
 "
 
 #Ignite version

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-put-indexed-val.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-put-indexed-val.properties b/modules/yardstick/config/benchmark-put-indexed-val.properties
index 0d699c5..152d72f 100644
--- a/modules/yardstick/config/benchmark-put-indexed-val.properties
+++ b/modules/yardstick/config/benchmark-put-indexed-val.properties
@@ -21,16 +21,14 @@
 # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
-JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=128m \
-  -XX:MaxNewSize=128m \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+JVM_OPTS=${JVM_OPTS}" \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
 "
 
 # List of default probes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-query-put-separated.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-query-put-separated.properties b/modules/yardstick/config/benchmark-query-put-separated.properties
index 500e4fa..b4437bf 100644
--- a/modules/yardstick/config/benchmark-query-put-separated.properties
+++ b/modules/yardstick/config/benchmark-query-put-separated.properties
@@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
 -XX:+PrintGCDetails \
 -verbose:gc \
 -XX:+UseParNewGC \
 -XX:+UseConcMarkSweepGC \
--XX:+UseTLAB \
--XX:NewSize=128m \
--XX:MaxNewSize=128m \
--XX:MaxTenuringThreshold=0 \
--XX:SurvivorRatio=1024 \
--XX:+UseCMSInitiatingOccupancyOnly \
--XX:CMSInitiatingOccupancyFraction=60 \
 "
 
 #Ignite version

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-query.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-query.properties b/modules/yardstick/config/benchmark-query.properties
index af9b07e..9192512 100644
--- a/modules/yardstick/config/benchmark-query.properties
+++ b/modules/yardstick/config/benchmark-query.properties
@@ -21,16 +21,14 @@
 # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
-JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=128m \
-  -XX:MaxNewSize=128m \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+JVM_OPTS=${JVM_OPTS}" \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
 "
 
 # List of default probes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-remote-sample.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-remote-sample.properties b/modules/yardstick/config/benchmark-remote-sample.properties
index 5f158ac..7e97f43 100644
--- a/modules/yardstick/config/benchmark-remote-sample.properties
+++ b/modules/yardstick/config/benchmark-remote-sample.properties
@@ -22,16 +22,14 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 # JVM_OPTS=${JVM_OPTS}" \
-#  -XX:+UseParNewGC \
-#  -XX:+UseConcMarkSweepGC \
-#  -XX:+UseTLAB \
-#  -XX:NewSize=128m \
-#  -XX:MaxNewSize=128m \
-#  -XX:MaxTenuringThreshold=0 \
-#  -XX:SurvivorRatio=1024 \
-#  -XX:+UseCMSInitiatingOccupancyOnly \
-#  -XX:CMSInitiatingOccupancyFraction=60 \
-#"
+# -Xms6g \
+# -Xmx6g \
+# -Xloggc:./gc${now0}.log \
+# -XX:+PrintGCDetails \
+# -verbose:gc \
+# -XX:+UseParNewGC \
+# -XX:+UseConcMarkSweepGC \
+# "
 
 # List of default probes.
 # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-remote.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-remote.properties b/modules/yardstick/config/benchmark-remote.properties
index 0cda79e..4d671d3 100644
--- a/modules/yardstick/config/benchmark-remote.properties
+++ b/modules/yardstick/config/benchmark-remote.properties
@@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
-  -XX:+PrintGCDetails \
-  -verbose:gc \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=128m \
-  -XX:MaxNewSize=128m \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
 "
 
 #Ignite version

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-sample.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-sample.properties b/modules/yardstick/config/benchmark-sample.properties
index 1932ae7..81e9a1b 100644
--- a/modules/yardstick/config/benchmark-sample.properties
+++ b/modules/yardstick/config/benchmark-sample.properties
@@ -22,16 +22,14 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 # JVM_OPTS=${JVM_OPTS}" \
-#  -XX:+UseParNewGC \
-#  -XX:+UseConcMarkSweepGC \
-#  -XX:+UseTLAB \
-#  -XX:NewSize=128m \
-#  -XX:MaxNewSize=128m \
-#  -XX:MaxTenuringThreshold=0 \
-#  -XX:SurvivorRatio=1024 \
-#  -XX:+UseCMSInitiatingOccupancyOnly \
-#  -XX:CMSInitiatingOccupancyFraction=60 \
-#"
+# -Xms6g \
+# -Xmx6g \
+# -Xloggc:./gc${now0}.log \
+# -XX:+PrintGCDetails \
+# -verbose:gc \
+# -XX:+UseParNewGC \
+# -XX:+UseConcMarkSweepGC \
+# "
 
 # List of default probes.
 # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-sql-dml.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-sql-dml.properties b/modules/yardstick/config/benchmark-sql-dml.properties
index a8bb453..faa62c9 100644
--- a/modules/yardstick/config/benchmark-sql-dml.properties
+++ b/modules/yardstick/config/benchmark-sql-dml.properties
@@ -21,16 +21,14 @@
 # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
-JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=128m \
-  -XX:MaxNewSize=128m \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+JVM_OPTS=${JVM_OPTS}" \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
 "
 
 # List of default probes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-store.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-store.properties b/modules/yardstick/config/benchmark-store.properties
index ea8487f..2f89d0e 100644
--- a/modules/yardstick/config/benchmark-store.properties
+++ b/modules/yardstick/config/benchmark-store.properties
@@ -26,17 +26,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
 -XX:+PrintGCDetails \
 -verbose:gc \
 -XX:+UseParNewGC \
 -XX:+UseConcMarkSweepGC \
--XX:+UseTLAB \
--XX:NewSize=128m \
--XX:MaxNewSize=128m \
--XX:MaxTenuringThreshold=0 \
--XX:SurvivorRatio=1024 \
--XX:+UseCMSInitiatingOccupancyOnly \
--XX:CMSInitiatingOccupancyFraction=60 \
 "
 
 #Ignite version

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark-tx.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-tx.properties b/modules/yardstick/config/benchmark-tx.properties
index 4744dee..cc48c53 100644
--- a/modules/yardstick/config/benchmark-tx.properties
+++ b/modules/yardstick/config/benchmark-tx.properties
@@ -22,16 +22,14 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 # JVM_OPTS=${JVM_OPTS}" \
-#  -XX:+UseParNewGC \
-#  -XX:+UseConcMarkSweepGC \
-#  -XX:+UseTLAB \
-#  -XX:NewSize=128m \
-#  -XX:MaxNewSize=128m \
-#  -XX:MaxTenuringThreshold=0 \
-#  -XX:SurvivorRatio=1024 \
-#  -XX:+UseCMSInitiatingOccupancyOnly \
-#  -XX:CMSInitiatingOccupancyFraction=60 \
-#"
+# -Xms6g \
+# -Xmx6g \
+# -Xloggc:./gc${now0}.log \
+# -XX:+PrintGCDetails \
+# -verbose:gc \
+# -XX:+UseParNewGC \
+# -XX:+UseConcMarkSweepGC \
+# "
 
 # List of default probes, comma separated.
 # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/benchmark.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark.properties b/modules/yardstick/config/benchmark.properties
index 1b5d5fe..cccbf2a 100644
--- a/modules/yardstick/config/benchmark.properties
+++ b/modules/yardstick/config/benchmark.properties
@@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
-  -XX:+PrintGCDetails \
-  -verbose:gc \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=128m \
-  -XX:MaxNewSize=128m \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+-Xms6g \
+-Xmx6g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
 "
 
 #Ignite version

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/test-max-int-values-offheap.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/test-max-int-values-offheap.properties b/modules/yardstick/config/test-max-int-values-offheap.properties
index 42769ec..838986d 100644
--- a/modules/yardstick/config/test-max-int-values-offheap.properties
+++ b/modules/yardstick/config/test-max-int-values-offheap.properties
@@ -21,21 +21,14 @@
 # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
-JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
-  -XX:+PrintGCDetails \
-  -XX:-PrintGCTimeStamps \
-  -verbose:gc \
-  -Xmx8g \
-  -Xms8g \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=1g \
-  -XX:MaxNewSize=1g \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+JVM_OPTS=${JVM_OPTS}" \
+-Xms8g \
+-Xmx8g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
 "
 
 # List of default probes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/test-max-int-values-onheap.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/test-max-int-values-onheap.properties b/modules/yardstick/config/test-max-int-values-onheap.properties
index 0faf5f3..c83f469 100644
--- a/modules/yardstick/config/test-max-int-values-onheap.properties
+++ b/modules/yardstick/config/test-max-int-values-onheap.properties
@@ -21,21 +21,14 @@
 # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
-JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
-  -XX:+PrintGCDetails \
-  -XX:-PrintGCTimeStamps \
-  -verbose:gc \
-  -Xmx92g \
-  -Xms32g \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=4g \
-  -XX:MaxNewSize=4g \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+JVM_OPTS=${JVM_OPTS}" \
+-Xms92g \
+-Xmx32g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
 "
 
 # List of default probes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4cc8a70/modules/yardstick/config/test-max-int-values-swap.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/test-max-int-values-swap.properties b/modules/yardstick/config/test-max-int-values-swap.properties
index 900e8ac..c462ada 100644
--- a/modules/yardstick/config/test-max-int-values-swap.properties
+++ b/modules/yardstick/config/test-max-int-values-swap.properties
@@ -21,21 +21,14 @@
 # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
-JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
-  -XX:+PrintGCDetails \
-  -XX:-PrintGCTimeStamps \
-  -verbose:gc \
-  -Xmx8g \
-  -Xms8g \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=1g \
-  -XX:MaxNewSize=1g \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+JVM_OPTS=${JVM_OPTS}" \
+-Xms8g \
+-Xmx8g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
 "
 
 # List of default probes.


[11/23] ignite git commit: Merge branch ignite-1.8.4 into master

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


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

Branch: refs/heads/ignite-4565-ddl
Commit: a140a8362bc5f9485ab21c3b8829e4c605d3f7d5
Parents: 84880a8 2263283
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Mar 21 13:23:58 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Mar 21 13:23:58 2017 +0300

----------------------------------------------------------------------
 .../affinity/fair/FairAffinityFunction.java     |  33 ++---
 .../affinity/fair/FairAffinityNodesRestart.java | 130 +++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 3 files changed, 149 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a140a836/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------


[07/23] ignite git commit: IGNITE-3386 - Minor code style changes

Posted by vo...@apache.org.
IGNITE-3386 - Minor code style changes


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

Branch: refs/heads/ignite-4565-ddl
Commit: 3310f10b9d48d64280d0ebf2464ad892f3f31c52
Parents: c2b201e
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Mar 2 10:37:00 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Mar 2 10:37:00 2017 +0300

----------------------------------------------------------------------
 .../internal/GridCacheRecreateLockTest.java     | 32 +++++++++-----------
 1 file changed, 14 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3310f10b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
index ae4ef87..ae850f7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
@@ -13,11 +13,10 @@ import org.apache.ignite.testframework.junits.common.GridCommonTest;
  */
 @GridCommonTest(group = "Kernal Self")
 public class GridCacheRecreateLockTest extends GridCommonAbstractTest {
-
     /**
      * @throws IgniteCheckedException If failed.
      */
-    public void test() throws Exception {
+    public void testLockOwnerLeavesGrid() throws Exception {
         final Ignite ignite = startNodeAndLock("node1");
 
         new Thread(new Runnable() {
@@ -36,27 +35,24 @@ public class GridCacheRecreateLockTest extends GridCommonAbstractTest {
         startNodeAndLock("node2");
     }
 
-    private Ignite startNodeAndLock(String name) {
-        try {
-            IgniteConfiguration cfg = new IgniteConfiguration();
-            cfg.setGridName(name);
-
-            Ignite ignite = Ignition.start(cfg);
+    /**
+     * @param name Grid name.
+     * @return Started Ignite instance.
+     */
+    private Ignite startNodeAndLock(String name) throws Exception {
+        IgniteConfiguration cfg = new IgniteConfiguration();
+        cfg.setGridName(name);
 
-            IgniteLock lock = ignite.reentrantLock("lock", true, true, true);
+        Ignite ignite = Ignition.start(cfg);
 
-            System.out.println("acquiring lock");
+        IgniteLock lock = ignite.reentrantLock("lock", true, true, true);
 
-            lock.lock();
+        System.out.println("acquiring lock");
 
-            System.out.println("acquired lock");
+        lock.lock();
 
-            return ignite;
-        }
-        catch (Exception e) {
-            assertTrue(false);
-        }
+        System.out.println("acquired lock");
 
-        return null;
+        return ignite;
     }
 }
\ No newline at end of file


[04/23] ignite git commit: Minor cleanup.

Posted by vo...@apache.org.
Minor cleanup.


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

Branch: refs/heads/ignite-4565-ddl
Commit: 620235f3ca0fa32c1f4a4dc1a5e0bd5545f6e07e
Parents: 7292a53
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Mar 2 11:05:19 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Mar 2 11:05:19 2017 +0700

----------------------------------------------------------------------
 .../ignite/internal/visor/cache/VisorCacheTypeMetadata.java    | 6 ++++++
 .../ignite/internal/visor/node/VisorIgfsConfiguration.java     | 1 -
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/620235f3/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
index f17e588..c87ad05 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
@@ -32,6 +32,7 @@ import org.apache.ignite.cache.store.jdbc.JdbcType;
 import org.apache.ignite.cache.store.jdbc.JdbcTypeField;
 import org.apache.ignite.internal.LessNamingBean;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 
@@ -372,4 +373,9 @@ public class VisorCacheTypeMetadata implements Serializable, LessNamingBean {
     public Map<String, LinkedHashMap<String, IgniteBiTuple<String, Boolean>>> grps() {
         return grps;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCacheTypeMetadata.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/620235f3/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
index 9f7652b..cb10c1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
@@ -25,7 +25,6 @@ import java.util.Map;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
 import org.apache.ignite.igfs.IgfsMode;
-import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.internal.LessNamingBean;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;


[21/23] ignite git commit: WIP.

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


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

Branch: refs/heads/ignite-4565-ddl
Commit: b35568e46dd25adaf60ad9d89f55ddeef6648416
Parents: f1419d7
Author: devozerov <vo...@gridgain.com>
Authored: Wed Mar 22 12:53:43 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Mar 22 12:53:43 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java        |  4 ----
 .../query/ddl/IndexAbstractDiscoveryMessage.java    | 16 ++--------------
 .../query/ddl/IndexAcceptDiscoveryMessage.java      |  5 +++++
 .../query/ddl/IndexFinishDiscoveryMessage.java      |  5 +++++
 .../query/ddl/IndexProposeDiscoveryMessage.java     |  5 +++++
 5 files changed, 17 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b35568e4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index bde7ff9..a173177 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -2808,8 +2808,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             msg.onError("Cache was stopped concurrently.");
         else
             desc.tryAccept(msg);
-
-        msg.exchange(true);
     }
 
     /**
@@ -2826,8 +2824,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             return;
 
         desc.tryFinish(msg);
-
-        msg.exchange(true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b35568e4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAbstractDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAbstractDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAbstractDiscoveryMessage.java
index 3de525b..314202e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAbstractDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAbstractDiscoveryMessage.java
@@ -38,9 +38,6 @@ public abstract class IndexAbstractDiscoveryMessage implements DiscoveryCustomMe
     @GridToStringInclude
     protected final AbstractIndexOperation op;
 
-    /** Whether request must be propagated to exchange worker for final processing. */
-    private transient boolean exchange;
-
     /**
      * Constructor.
      *
@@ -63,18 +60,9 @@ public abstract class IndexAbstractDiscoveryMessage implements DiscoveryCustomMe
     }
 
     /**
-     * @return Whether request must be propagated to exchange worker for final processing.
-     */
-    public boolean exchange() {
-        return exchange;
-    }
-
-    /**
-     * @param exchange Whether request must be propagated to exchange worker for final processing.
+     * @return Whether request must be propagated to exchange thread.
      */
-    public void exchange(boolean exchange) {
-        this.exchange = exchange;
-    }
+    public abstract boolean exchange();
 
     /** {@inheritDoc} */
     @Override public String toString() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b35568e4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAcceptDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAcceptDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAcceptDiscoveryMessage.java
index a7c756f..935e683 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAcceptDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAcceptDiscoveryMessage.java
@@ -52,6 +52,11 @@ public class IndexAcceptDiscoveryMessage extends IndexAbstractDiscoveryMessage {
         return false;
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean exchange() {
+        return true;
+    }
+
     /**
      * @return Error message.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b35568e4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexFinishDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexFinishDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexFinishDiscoveryMessage.java
index 5a2d66c..c3ce40c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexFinishDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexFinishDiscoveryMessage.java
@@ -59,6 +59,11 @@ public class IndexFinishDiscoveryMessage extends IndexAbstractDiscoveryMessage {
         return false;
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean exchange() {
+        return true;
+    }
+
     /**
      * @return {@code True} if error was reported during init.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b35568e4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexProposeDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexProposeDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexProposeDiscoveryMessage.java
index 6d6c72e..f619fe8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexProposeDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexProposeDiscoveryMessage.java
@@ -66,6 +66,11 @@ public class IndexProposeDiscoveryMessage extends IndexAbstractDiscoveryMessage
     }
 
     /** {@inheritDoc} */
+    @Override public boolean exchange() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public void context(GridKernalContext ctx) {
         this.ctx = ctx;
     }


[08/23] ignite git commit: IGNITE-4717 VisorClearTask minor fix. (cherry picked from commit d4b87f4)

Posted by vo...@apache.org.
IGNITE-4717 VisorClearTask minor fix.
(cherry picked from commit d4b87f4)


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

Branch: refs/heads/ignite-4565-ddl
Commit: 590b82d817595b82a0706e332c545e4746fafbc2
Parents: 3310f10
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Mar 10 15:51:38 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Mar 10 16:37:40 2017 +0700

----------------------------------------------------------------------
 .../visor/cache/VisorCacheClearTask.java        | 57 +++++++++++++++++++-
 1 file changed, 56 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/590b82d8/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
index 0c8476f..ce74f17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.visor.VisorJob;
 import org.apache.ignite.internal.visor.VisorOneNodeTask;
 import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.resources.JobContextResource;
@@ -145,4 +146,58 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
             return S.toString(VisorCacheClearJob.class, this);
         }
     }
-}
\ No newline at end of file
+
+    /**
+     * Callable to get cache size.
+     *
+     * @deprecated This class needed only for compatibility.
+     */
+    @GridInternal @Deprecated
+    private static class VisorCacheSizeCallable implements IgniteCallable<Integer> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final IgniteCache cache;
+
+        /**
+         * @param cache Cache to take size from.
+         */
+        private VisorCacheSizeCallable(IgniteCache cache) {
+            this.cache = cache;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer call() throws Exception {
+            return cache.size(CachePeekMode.PRIMARY);
+        }
+    }
+
+    /**
+     * Callable to clear cache.
+     *
+     * @deprecated This class needed only for compatibility.
+     */
+    @GridInternal @Deprecated
+    private static class VisorCacheClearCallable implements IgniteCallable<Integer> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final IgniteCache cache;
+
+        /**
+         * @param cache Cache to clear.
+         */
+        private VisorCacheClearCallable(IgniteCache cache) {
+            this.cache = cache;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer call() throws Exception {
+            cache.clear();
+
+            return 0;
+        }
+    }
+}


[10/23] ignite git commit: Merge ignite-1.7.8 into ignite-1.8.4

Posted by vo...@apache.org.
Merge ignite-1.7.8 into ignite-1.8.4

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java


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

Branch: refs/heads/ignite-4565-ddl
Commit: 22632839bd59123167ed96255d6fbcd9c04f45a9
Parents: 6633da6 c0e2df2
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Mar 15 13:47:55 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Mar 15 13:47:55 2017 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   4 +
 .../distributed/dht/GridDhtCacheAdapter.java    |  19 +-
 .../cache/query/GridCacheQueryManager.java      |  97 +++++-----
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   5 +
 .../org/apache/ignite/spi/IgniteSpiContext.java |   6 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  32 +++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  19 ++
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   5 +
 .../tcp/internal/TcpDiscoveryNode.java          |   6 +-
 ...veryNodeAttributesUpdateOnReconnectTest.java | 110 +++++++++++
 .../TcpDiscoverySslSecuredUnsecuredTest.java    | 185 +++++++++++++++++++
 .../tcp/TestReconnectPluginProvider.java        | 111 +++++++++++
 .../discovery/tcp/TestReconnectProcessor.java   |  93 ++++++++++
 .../testframework/GridSpiTestContext.java       |   5 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   5 +
 .../org.apache.ignite.plugin.PluginProvider     |   1 +
 parent/pom.xml                                  |   1 +
 17 files changed, 648 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/ignite/blob/22632839/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/22632839/parent/pom.xml
----------------------------------------------------------------------


[17/23] ignite git commit: IGNITE-4686 Added ability to group registered users in admin panel.

Posted by vo...@apache.org.
IGNITE-4686 Added ability to group registered users in admin panel.


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

Branch: refs/heads/ignite-4565-ddl
Commit: 827befb7f368afd1a42ec52e559dee01ab57113a
Parents: 3b89a5c
Author: Dmitriy Shabalin <ds...@gridgain.com>
Authored: Wed Mar 22 10:37:05 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Wed Mar 22 10:37:05 2017 +0700

----------------------------------------------------------------------
 modules/web-console/frontend/app/app.js         |   1 +
 .../form-field-datepicker.pug                   |  12 +-
 .../list-of-registered-users/index.js           |   2 +
 .../list-of-registered-users.column-defs.js     |  48 ++---
 .../list-of-registered-users.controller.js      | 193 ++++++++++++++-----
 .../list-of-registered-users.scss               |  28 +++
 .../list-of-registered-users.tpl.pug            |  50 +++--
 .../ui-grid-header/ui-grid-header.scss          |   6 +
 .../ui-grid-header/ui-grid-header.tpl.pug       |   4 +-
 .../ui-grid-settings/ui-grid-settings.scss      |  10 +
 .../frontend/app/primitives/badge/index.scss    |  36 ++++
 .../frontend/app/primitives/index.js            |  19 ++
 .../frontend/app/primitives/tabs/index.scss     |  73 +++++++
 .../frontend/public/stylesheets/style.scss      |   2 +
 14 files changed, 389 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/app.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js
index 1e21d24..26d3ad5 100644
--- a/modules/web-console/frontend/app/app.js
+++ b/modules/web-console/frontend/app/app.js
@@ -16,6 +16,7 @@
  */
 
 import '../public/stylesheets/style.scss';
+import '../app/primitives';
 import './components/ui-grid-header/ui-grid-header.scss';
 import './components/ui-grid-settings/ui-grid-settings.scss';
 import './components/form-field-datepicker/form-field-datepicker.scss';

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/components/form-field-datepicker/form-field-datepicker.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/form-field-datepicker/form-field-datepicker.pug b/modules/web-console/frontend/app/components/form-field-datepicker/form-field-datepicker.pug
index c9d382c..d70476f 100644
--- a/modules/web-console/frontend/app/components/form-field-datepicker/form-field-datepicker.pug
+++ b/modules/web-console/frontend/app/components/form-field-datepicker/form-field-datepicker.pug
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-mixin ignite-form-field-datepicker(label, model, name, disabled, required, placeholder, tip)
+mixin ignite-form-field-datepicker(label, model, name, mindate, maxdate, disabled, required, placeholder, tip)
     mixin form-field-input()
         input.form-control(
             id=`{{ ${name} }}Input`
@@ -30,8 +30,10 @@ mixin ignite-form-field-datepicker(label, model, name, disabled, required, place
             bs-datepicker
             data-date-format='MMM yyyy'
             data-start-view='1'
-            data-min-view='1' 
-            data-max-date='today'
+            data-min-view='1'
+
+            data-min-date=mindate ? `{{ ${mindate} }}` : false
+            data-max-date=maxdate ? `{{ ${maxdate} }}` : `today`
 
             data-container='body > .wrapper'
 
@@ -43,7 +45,9 @@ mixin ignite-form-field-datepicker(label, model, name, disabled, required, place
         )&attributes(attributes.attributes)
 
     .ignite-form-field
-        +ignite-form-field__label(label, name, required)
+        if name
+            +ignite-form-field__label(label, name, required)
+
         .ignite-form-field__control
             if tip
                 i.tipField.icon-help(bs-tooltip='' data-title=tip)

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/components/list-of-registered-users/index.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/list-of-registered-users/index.js b/modules/web-console/frontend/app/components/list-of-registered-users/index.js
index 22a89da..4e5061f 100644
--- a/modules/web-console/frontend/app/components/list-of-registered-users/index.js
+++ b/modules/web-console/frontend/app/components/list-of-registered-users/index.js
@@ -15,6 +15,8 @@
  * limitations under the License.
  */
 
+import './list-of-registered-users.scss';
+
 import templateUrl from './list-of-registered-users.tpl.pug';
 import controller from './list-of-registered-users.controller';
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.column-defs.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.column-defs.js b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.column-defs.js
index e6ba842..e859acf 100644
--- a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.column-defs.js
+++ b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.column-defs.js
@@ -49,32 +49,32 @@ const ACTIONS_TEMPLATE = `
 const EMAIL_TEMPLATE = '<div class="ui-grid-cell-contents"><a ng-href="mailto:{{ COL_FIELD }}">{{ COL_FIELD }}</a></div>';
 
 export default [
-    {displayName: 'Actions', categoryDisplayName: 'Actions', cellTemplate: ACTIONS_TEMPLATE, field: 'actions', minWidth: 65, width: 65, enableFiltering: false, enableSorting: false, pinnedLeft: true},
-    {displayName: 'User', categoryDisplayName: 'User', field: 'userName', cellTemplate: USER_TEMPLATE, minWidth: 160, enableFiltering: true, filter: { placeholder: 'Filter by name...' }, pinnedLeft: true},
-    {displayName: 'Email', categoryDisplayName: 'Email', field: 'email', cellTemplate: EMAIL_TEMPLATE, minWidth: 160, enableFiltering: true, filter: { placeholder: 'Filter by email...' }},
-    {displayName: 'Company', categoryDisplayName: 'Company', field: 'company', minWidth: 160, enableFiltering: true},
-    {displayName: 'Country', categoryDisplayName: 'Country', field: 'countryCode', minWidth: 80, enableFiltering: true},
-    {displayName: 'Last login', categoryDisplayName: 'Last login', field: 'lastLogin', cellFilter: 'date:"M/d/yy HH:mm"', minWidth: 105, width: 105, enableFiltering: false, visible: false},
-    {displayName: 'Last activity', categoryDisplayName: 'Last activity', field: 'lastActivity', cellFilter: 'date:"M/d/yy HH:mm"', minWidth: 105, width: 105, enableFiltering: false, visible: true, sort: { direction: 'desc', priority: 0 }},
+    {name: 'actions', displayName: 'Actions', categoryDisplayName: 'Actions', cellTemplate: ACTIONS_TEMPLATE, field: 'actions', minWidth: 70, width: 70, enableFiltering: false, enableSorting: false},
+    {name: 'user', displayName: 'User', categoryDisplayName: 'User', field: 'userName', cellTemplate: USER_TEMPLATE, minWidth: 160, enableFiltering: true, filter: { placeholder: 'Filter by name...' }},
+    {name: 'email', displayName: 'Email', categoryDisplayName: 'Email', field: 'email', cellTemplate: EMAIL_TEMPLATE, minWidth: 160, enableFiltering: true, filter: { placeholder: 'Filter by email...' }},
+    {name: 'company', displayName: 'Company', categoryDisplayName: 'Company', field: 'company', minWidth: 160, enableFiltering: true, filter: { placeholder: 'Filter by company...' }},
+    {name: 'country', displayName: 'Country', categoryDisplayName: 'Country', field: 'countryCode', minWidth: 80, enableFiltering: true, filter: { placeholder: 'Filter by country...' }},
+    {name: 'lastlogin', displayName: 'Last login', categoryDisplayName: 'Last login', field: 'lastLogin', cellFilter: 'date:"M/d/yy HH:mm"', minWidth: 105, width: 105, enableFiltering: false, visible: false},
+    {name: 'lastactivity', displayName: 'Last activity', categoryDisplayName: 'Last activity', field: 'lastActivity', cellFilter: 'date:"M/d/yy HH:mm"', minWidth: 115, width: 115, enableFiltering: false, visible: true, sort: { direction: 'desc', priority: 0 }},
     // Configurations
-    {displayName: 'Clusters count', categoryDisplayName: 'Configurations', headerCellTemplate: CLUSTER_HEADER_TEMPLATE, field: 'counters.clusters', type: 'number', headerTooltip: 'Clusters count', minWidth: 50, width: 50, enableFiltering: false, visible: false},
-    {displayName: 'Models count', categoryDisplayName: 'Configurations', headerCellTemplate: MODEL_HEADER_TEMPLATE, field: 'counters.models', type: 'number', headerTooltip: 'Models count', minWidth: 50, width: 50, enableFiltering: false, visible: false},
-    {displayName: 'Caches count', categoryDisplayName: 'Configurations', headerCellTemplate: CACHE_HEADER_TEMPLATE, field: 'counters.caches', type: 'number', headerTooltip: 'Caches count', minWidth: 50, width: 50, enableFiltering: false, visible: false},
-    {displayName: 'IGFS count', categoryDisplayName: 'Configurations', headerCellTemplate: IGFS_HEADER_TEMPLATE, field: 'counters.igfs', type: 'number', headerTooltip: 'IGFS count', minWidth: 50, width: 50, enableFiltering: false, visible: false},
+    {name: 'cfg_clusters', displayName: 'Clusters count', categoryDisplayName: 'Configurations', headerCellTemplate: CLUSTER_HEADER_TEMPLATE, field: 'counters.clusters', type: 'number', headerTooltip: 'Clusters count', minWidth: 55, width: 55, enableFiltering: false, visible: false},
+    {name: 'cfg_models', displayName: 'Models count', categoryDisplayName: 'Configurations', headerCellTemplate: MODEL_HEADER_TEMPLATE, field: 'counters.models', type: 'number', headerTooltip: 'Models count', minWidth: 55, width: 55, enableFiltering: false, visible: false},
+    {name: 'cfg_caches', displayName: 'Caches count', categoryDisplayName: 'Configurations', headerCellTemplate: CACHE_HEADER_TEMPLATE, field: 'counters.caches', type: 'number', headerTooltip: 'Caches count', minWidth: 55, width: 55, enableFiltering: false, visible: false},
+    {name: 'cfg_igfs', displayName: 'IGFS count', categoryDisplayName: 'Configurations', headerCellTemplate: IGFS_HEADER_TEMPLATE, field: 'counters.igfs', type: 'number', headerTooltip: 'IGFS count', minWidth: 55, width: 55, enableFiltering: false, visible: false},
     // Activities Total
-    {displayName: 'Cfg', categoryDisplayName: 'Total activities', field: 'activitiesTotal["configuration"] || 0', type: 'number', headerTooltip: 'Total count of configuration usages', minWidth: 50, width: 50, enableFiltering: false},
-    {displayName: 'Qry', categoryDisplayName: 'Total activities', field: 'activitiesTotal["queries"] || 0', type: 'number', headerTooltip: 'Total count of queries usages', minWidth: 50, width: 50, enableFiltering: false},
-    {displayName: 'Demo', categoryDisplayName: 'Total activities', field: 'activitiesTotal["demo"] || 0', type: 'number', headerTooltip: 'Total count of demo startup', minWidth: 60, width: 60, enableFiltering: false},
-    {displayName: 'Dnld', categoryDisplayName: 'Total activities', field: 'activitiesDetail["/agent/download"] || 0', type: 'number', headerTooltip: 'Total count of agent downloads', minWidth: 55, width: 55, enableFiltering: false},
-    {displayName: 'Starts', categoryDisplayName: 'Total activities', field: 'activitiesDetail["/agent/start"] || 0', type: 'number', headerTooltip: 'Total count of agent startup', minWidth: 60, width: 60, enableFiltering: false},
+    {name: 'cfg', displayName: 'Cfg', categoryDisplayName: 'Total activities', field: 'activitiesTotal["configuration"] || 0', type: 'number', headerTooltip: 'Total count of configuration usages', minWidth: 55, width: 55, enableFiltering: false},
+    {name: 'qry', displayName: 'Qry', categoryDisplayName: 'Total activities', field: 'activitiesTotal["queries"] || 0', type: 'number', headerTooltip: 'Total count of queries usages', minWidth: 55, width: 55, enableFiltering: false},
+    {name: 'demo', displayName: 'Demo', categoryDisplayName: 'Total activities', field: 'activitiesTotal["demo"] || 0', type: 'number', headerTooltip: 'Total count of demo startup', minWidth: 65, width: 65, enableFiltering: false},
+    {name: 'dnld', displayName: 'Dnld', categoryDisplayName: 'Total activities', field: 'activitiesDetail["/agent/download"] || 0', type: 'number', headerTooltip: 'Total count of agent downloads', minWidth: 55, width: 55, enableFiltering: false},
+    {name: 'starts', displayName: 'Starts', categoryDisplayName: 'Total activities', field: 'activitiesDetail["/agent/start"] || 0', type: 'number', headerTooltip: 'Total count of agent startup', minWidth: 65, width: 65, enableFiltering: false},
     // Activities Configuration
-    {displayName: 'Clusters', categoryDisplayName: 'Configuration\'s activities', field: 'activitiesDetail["/configuration/clusters"] || 0', type: 'number', headerTooltip: 'Configuration clusters', minWidth: 50, width: 80, enableFiltering: false, visible: false},
-    {displayName: 'Model', categoryDisplayName: 'Configuration\'s activities', field: 'activitiesDetail["/configuration/domains"] || 0', type: 'number', headerTooltip: 'Configuration model', minWidth: 50, width: 80, enableFiltering: false, visible: false},
-    {displayName: 'Caches', categoryDisplayName: 'Configuration\'s activities', field: 'activitiesDetail["/configuration/caches"] || 0', type: 'number', headerTooltip: 'Configuration caches', minWidth: 50, width: 80, enableFiltering: false, visible: false},
-    {displayName: 'IGFS', categoryDisplayName: 'Configuration\'s activities', field: 'activitiesDetail["/configuration/igfs"] || 0', type: 'number', headerTooltip: 'Configuration IGFS', minWidth: 50, width: 80, enableFiltering: false, visible: false},
-    {displayName: 'Summary', categoryDisplayName: 'Configuration\'s activities', field: 'activitiesDetail["/configuration/summary"] || 0', type: 'number', headerTooltip: 'Configuration summary', minWidth: 50, width: 80, enableFiltering: false, visible: false},
+    {name: 'clusters', displayName: 'Clusters', categoryDisplayName: 'Configuration\'s activities', field: 'activitiesDetail["/configuration/clusters"] || 0', type: 'number', headerTooltip: 'Configuration clusters', minWidth: 55, width: 80, enableFiltering: false, visible: false},
+    {name: 'model', displayName: 'Model', categoryDisplayName: 'Configuration\'s activities', field: 'activitiesDetail["/configuration/domains"] || 0', type: 'number', headerTooltip: 'Configuration model', minWidth: 55, width: 80, enableFiltering: false, visible: false},
+    {name: 'caches', displayName: 'Caches', categoryDisplayName: 'Configuration\'s activities', field: 'activitiesDetail["/configuration/caches"] || 0', type: 'number', headerTooltip: 'Configuration caches', minWidth: 55, width: 80, enableFiltering: false, visible: false},
+    {name: 'igfs', displayName: 'IGFS', categoryDisplayName: 'Configuration\'s activities', field: 'activitiesDetail["/configuration/igfs"] || 0', type: 'number', headerTooltip: 'Configuration IGFS', minWidth: 55, width: 80, enableFiltering: false, visible: false},
+    {name: 'summary', displayName: 'Summary', categoryDisplayName: 'Configuration\'s activities', field: 'activitiesDetail["/configuration/summary"] || 0', type: 'number', headerTooltip: 'Configuration summary', minWidth: 55, width: 80, enableFiltering: false, visible: false},
     // Activities Queries
-    {displayName: 'Execute', categoryDisplayName: 'Queries\' activities', field: 'activitiesDetail["/queries/execute"] || 0', type: 'number', headerTooltip: 'Query executions', minWidth: 50, width: 80, enableFiltering: false, visible: false},
-    {displayName: 'Explain', categoryDisplayName: 'Queries\' activities', field: 'activitiesDetail["/queries/explain"] || 0', type: 'number', headerTooltip: 'Query explain executions', minWidth: 50, width: 80, enableFiltering: false, visible: false},
-    {displayName: 'Scan', categoryDisplayName: 'Queries\' activities', field: 'activitiesDetail["/queries/scan"] || 0', type: 'number', headerTooltip: 'Scan query executions', minWidth: 50, width: 80, enableFiltering: false, visible: false}
+    {name: 'execute', displayName: 'Execute', categoryDisplayName: 'Queries\' activities', field: 'activitiesDetail["/queries/execute"] || 0', type: 'number', headerTooltip: 'Query executions', minWidth: 55, width: 80, enableFiltering: false, visible: false},
+    {name: 'explain', displayName: 'Explain', categoryDisplayName: 'Queries\' activities', field: 'activitiesDetail["/queries/explain"] || 0', type: 'number', headerTooltip: 'Query explain executions', minWidth: 55, width: 80, enableFiltering: false, visible: false},
+    {name: 'scan', displayName: 'Scan', categoryDisplayName: 'Queries\' activities', field: 'activitiesDetail["/queries/scan"] || 0', type: 'number', headerTooltip: 'Scan query executions', minWidth: 55, width: 80, enableFiltering: false, visible: false}
 ];

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.controller.js b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.controller.js
index 54971b1..acf76fa 100644
--- a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.controller.js
+++ b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.controller.js
@@ -30,34 +30,21 @@ const rowTemplate = `<div
   ui-grid-cell/>`;
 
 export default class IgniteListOfRegisteredUsersCtrl {
-    static $inject = ['$scope', '$state', '$filter', 'User', 'uiGridConstants', 'IgniteAdminData', 'IgniteNotebookData', 'IgniteConfirm', 'IgniteActivitiesUserDialog'];
+    static $inject = ['$scope', '$state', '$filter', 'User', 'uiGridGroupingConstants', 'IgniteAdminData', 'IgniteNotebookData', 'IgniteConfirm', 'IgniteActivitiesUserDialog'];
 
-    constructor($scope, $state, $filter, User, uiGridConstants, AdminData, NotebookData, Confirm, ActivitiesUserDialog) {
+    constructor($scope, $state, $filter, User, uiGridGroupingConstants, AdminData, NotebookData, Confirm, ActivitiesUserDialog) {
         const $ctrl = this;
 
-        const companySelectOptions = [];
-        const countrySelectOptions = [];
-
         const dtFilter = $filter('date');
 
+        $ctrl.groupBy = 'user';
+
         $ctrl.params = {
-            startDate: new Date()
+            startDate: new Date(),
+            endDate: new Date()
         };
 
-        const columnCompany = _.find(columnDefs, { displayName: 'Company' });
-        const columnCountry = _.find(columnDefs, { displayName: 'Country' });
-
-        columnCompany.filter = {
-            selectOptions: companySelectOptions,
-            type: uiGridConstants.filter.SELECT,
-            condition: uiGridConstants.filter.EXACT
-        };
-
-        columnCountry.filter = {
-            selectOptions: countrySelectOptions,
-            type: uiGridConstants.filter.SELECT,
-            condition: uiGridConstants.filter.EXACT
-        };
+        $ctrl.uiGridGroupingConstants = uiGridGroupingConstants;
 
         const becomeUser = (user) => {
             AdminData.becomeUser(user._id)
@@ -105,6 +92,11 @@ export default class IgniteListOfRegisteredUsersCtrl {
             return renderableRows;
         };
 
+        $ctrl._userGridOptions = {
+            columnDefs,
+            categories
+        };
+
         $ctrl.gridOptions = {
             data: [],
             columnVirtualizationThreshold: 30,
@@ -134,19 +126,6 @@ export default class IgniteListOfRegisteredUsersCtrl {
             }
         };
 
-        const usersToFilterOptions = (column) => {
-            return _.sortBy(
-                _.map(
-                    _.groupBy($ctrl.gridOptions.data, (usr) => {
-                        const fld = usr[column];
-
-                        return _.isNil(fld) ? fld : fld.toUpperCase();
-                    }),
-                    (arr, value) => ({label: `${_.head(arr)[column] || 'Not set'} (${arr.length})`, value})
-                ),
-                'value');
-        };
-
         /**
          * @param {{startDate: number, endDate: number}} params
          */
@@ -154,31 +133,32 @@ export default class IgniteListOfRegisteredUsersCtrl {
             AdminData.loadUsers(params)
                 .then((data) => $ctrl.gridOptions.data = data)
                 .then((data) => {
-                    companySelectOptions.length = 0;
-                    countrySelectOptions.length = 0;
-
-                    companySelectOptions.push(...usersToFilterOptions('company'));
-                    countrySelectOptions.push(...usersToFilterOptions('countryCode'));
-
                     this.gridApi.grid.refresh();
 
+                    this.companies = _.values(_.groupBy(data, (b) => b.company.toLowerCase()));
+                    this.countries = _.values(_.groupBy(data, (b) => b.countryCode));
+
                     return data;
-                })
-                .then((data) => $ctrl.adjustHeight(data.length));
+                });
+        };
+
+        const fitlerDates = (sdt, edt) => {
+            $ctrl.gridOptions.exporterCsvFilename = `web_console_users_${dtFilter(sdt, 'yyyy_MM')}.csv`;
+
+            const startDate = Date.UTC(sdt.getFullYear(), sdt.getMonth(), 1);
+            const endDate = Date.UTC(edt.getFullYear(), edt.getMonth() + 1, 1);
+
+            reloadUsers({ startDate, endDate });
         };
 
         $scope.$watch(() => $ctrl.params.companiesExclude, () => {
             $ctrl.gridApi.grid.refreshRows();
         });
 
-        $scope.$watch(() => $ctrl.params.startDate, (dt) => {
-            $ctrl.gridOptions.exporterCsvFilename = `web_console_users_${dtFilter(dt, 'yyyy_MM')}.csv`;
+        $scope.$watch(() => $ctrl.params.startDate, (sdt) => fitlerDates(sdt, $ctrl.params.endDate));
+        $scope.$watch(() => $ctrl.params.endDate, (edt) => fitlerDates($ctrl.params.startDate, edt));
 
-            const startDate = Date.UTC(dt.getFullYear(), dt.getMonth(), 1);
-            const endDate = Date.UTC(dt.getFullYear(), dt.getMonth() + 1, 1);
-
-            reloadUsers({ startDate, endDate });
-        });
+        $scope.$watch(() => $ctrl.gridApi.grid.getVisibleRows().length, (length) => $ctrl.adjustHeight(length >= 20 ? 20 : length));
     }
 
     adjustHeight(rows) {
@@ -235,4 +215,121 @@ export default class IgniteListOfRegisteredUsersCtrl {
     exportCsv() {
         this.gridApi.exporter.csvExport('visible', 'visible');
     }
+
+    groupByUser() {
+        this.groupBy = 'user';
+
+        this.gridApi.grouping.clearGrouping();
+
+        this.gridOptions.categories = this._userGridOptions.categories;
+        this.gridOptions.columnDefs = this._userGridOptions.columnDefs;
+    }
+
+    groupByCompany() {
+        this.groupBy = 'company';
+
+        this.gridApi.grouping.clearGrouping();
+        this.gridApi.grouping.groupColumn('company');
+        this.gridApi.grouping.aggregateColumn('user', this.uiGridGroupingConstants.aggregation.COUNT);
+
+        if (this._companyGridOptions) {
+            this.gridOptions.categories = this._companyGridOptions.categories;
+            this.gridOptions.columnDefs = this._companyGridOptions.columnDefs;
+
+            return;
+        }
+
+        const _categories = _.cloneDeep(categories);
+        const _columnDefs = _.cloneDeep(columnDefs);
+
+        // Cut company category;
+        const company = _categories.splice(3, 1)[0];
+
+        // Hide Actions category;
+        _categories.splice(0, 1);
+
+        _.forEach(_.filter(_columnDefs, {displayName: 'Actions'}), (col) => {
+            col.visible = false;
+        });
+
+        // Add company as first column;
+        _categories.unshift(company);
+
+        _.forEach(_columnDefs, (col) => {
+            col.enableSorting = true;
+
+            if (col.type !== 'number')
+                return;
+
+            col.treeAggregationType = this.uiGridGroupingConstants.aggregation.SUM;
+            col.customTreeAggregationFinalizerFn = (agg) => agg.rendered = agg.value;
+        });
+
+        // Set grouping to last activity column
+        const lastactivity = _.find(_columnDefs, { name: 'lastactivity' });
+
+        if (_.nonNil(lastactivity)) {
+            lastactivity.treeAggregationType = this.uiGridGroupingConstants.aggregation.MAX;
+            lastactivity.customTreeAggregationFinalizerFn = (agg) => agg.rendered = agg.value;
+        }
+
+        this._companyGridOptions = {
+            categories: this.gridOptions.categories = _categories,
+            columnDefs: this.gridOptions.columnDefs = _columnDefs
+        };
+    }
+
+    groupByCountry() {
+        this.groupBy = 'country';
+
+        this.gridApi.grouping.clearGrouping();
+        this.gridApi.grouping.groupColumn('country');
+        this.gridApi.grouping.aggregateColumn('user', this.uiGridGroupingConstants.aggregation.COUNT);
+
+        if (this._countryGridOptions) {
+            this.gridOptions.categories = this._countryGridOptions.categories;
+            this.gridOptions.columnDefs = this._countryGridOptions.columnDefs;
+
+            return;
+        }
+
+        const _categories = _.cloneDeep(categories);
+        const _columnDefs = _.cloneDeep(columnDefs);
+
+        // Cut country category;
+        const country = _categories.splice(4, 1)[0];
+
+        // Hide Actions category;
+        _categories.splice(0, 1);
+
+        _.forEach(_.filter(_columnDefs, {displayName: 'Actions'}), (col) => {
+            col.visible = false;
+        });
+
+        // Add company as first column;
+        _categories.unshift(country);
+
+        _.forEach(_columnDefs, (col) => {
+            col.enableSorting = true;
+
+            if (col.type !== 'number')
+                return;
+
+            col.treeAggregationType = this.uiGridGroupingConstants.aggregation.SUM;
+            col.customTreeAggregationFinalizerFn = (agg) => agg.rendered = agg.value;
+        });
+
+        // Set grouping to last activity column
+        const lastactivity = _.find(_columnDefs, { name: 'lastactivity' });
+
+        if (_.nonNil(lastactivity)) {
+            lastactivity.treeAggregationType = this.uiGridGroupingConstants.aggregation.MAX;
+            lastactivity.customTreeAggregationFinalizerFn = (agg) => agg.rendered = agg.value;
+        }
+
+        this._countryGridOptions = {
+            categories: this.gridOptions.categories = _categories,
+            columnDefs: this.gridOptions.columnDefs = _columnDefs
+        };
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.scss b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.scss
new file mode 100644
index 0000000..8059d70
--- /dev/null
+++ b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.scss
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+.list-of-registered-users {
+  & > a {
+    display: inline-block;
+    margin: 10px;
+    margin-left: 0;
+
+    &.active {
+      font-weight: bold;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.tpl.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.tpl.pug b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.tpl.pug
index 52975b9..ec4b4fd 100644
--- a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.tpl.pug
+++ b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.tpl.pug
@@ -34,25 +34,39 @@ mixin grid-settings()
         li
             a(ng-click='$hide()') Close
 
-.panel.panel-default
-    .panel-heading.ui-grid-settings
-        +grid-settings
-        label Total users: 
-            strong {{ $ctrl.gridOptions.data.length }}&nbsp;&nbsp;&nbsp;
-        label Showing users:
-            strong {{ $ctrl.gridApi.grid.getVisibleRows().length }}
-            sub(ng-show='users.length === $ctrl.gridApi.grid.getVisibleRows().length') all
-
-        form.pull-right(ng-form=form novalidate)
-            -var form = 'admin'
 
-            button.btn.btn-primary(ng-click='$ctrl.exportCsv()' bs-tooltip data-title='Export table to csv') Export
+.list-of-registered-users
+    ul.tabs
+        li(role='presentation' ng-class='{ active: $ctrl.groupBy === "user" }') 
+            a(ng-click='$ctrl.groupByUser()') 
+                span Users
+                span.badge {{ $ctrl.gridOptions.data.length }}
+        li(role='presentation' ng-class='{ active: $ctrl.groupBy === "company" }')
+            a(ng-click='$ctrl.groupByCompany()') 
+                span Companies
+                span.badge {{ $ctrl.companies.length }}
+        li(role='presentation' ng-class='{ active: $ctrl.groupBy === "country" }')
+            a(ng-click='$ctrl.groupByCountry()')
+                span Countries
+                span.badge {{ $ctrl.countries.length }}
+
+    .panel.panel-default
+        .panel-heading.ui-grid-settings
+            +grid-settings
+            label(ng-show='$ctrl.groupBy === "user"') Showing users:&nbsp;
+                strong {{ $ctrl.gridApi.grid.getVisibleRows().length }}
+                sub(ng-show='users.length === $ctrl.gridApi.grid.getVisibleRows().length') all
+
+            -var form = 'admin'
+            form.pull-right(name=form novalidate)
+                button.btn.btn-primary(ng-click='$ctrl.exportCsv()' bs-tooltip data-title='Export table to csv') Export
 
-            .ui-grid-settings-dateperiod
-                +ignite-form-field-datepicker('Period:', '$ctrl.params.startDate', '"period"')
+                .ui-grid-settings-dateperiod
+                    +ignite-form-field-datepicker('Period:', '$ctrl.params.startDate', '"period"', null, '$ctrl.params.endDate')
+                    +ignite-form-field-datepicker('Period:', '$ctrl.params.endDate', null, '$ctrl.params.startDate', null)
 
-            .ui-grid-settings-filter
-                +ignite-form-field-text('Exclude:', '$ctrl.params.companiesExclude', '"exclude"', false, false, 'Exclude by company name...')
+                .ui-grid-settings-filter
+                    +ignite-form-field-text('Exclude:', '$ctrl.params.companiesExclude', '"exclude"', false, false, 'Exclude by company name...')
 
-    .panel-collapse
-        .grid.ui-grid--ignite(ui-grid='$ctrl.gridOptions' ui-grid-resize-columns ui-grid-selection ui-grid-exporter ui-grid-pinning)
+        .panel-collapse
+            .grid.ui-grid--ignite(ui-grid='$ctrl.gridOptions' ui-grid-resize-columns ui-grid-selection ui-grid-exporter ui-grid-pinning ui-grid-grouping)

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.scss b/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.scss
index c6e7bdf..4530c02 100644
--- a/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.scss
+++ b/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.scss
@@ -28,6 +28,12 @@
         height: 30px;
     }
 
+    .ui-grid-header-cell {
+        .ui-grid-cell-contents > span:not(.ui-grid-header-cell-label) {
+            right: 3px;
+        }
+    }
+
     .ui-grid-header-cell [role="columnheader"] {
         display: flex;
         

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.tpl.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.tpl.pug b/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.tpl.pug
index 7e44d94..9b14fca 100644
--- a/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.tpl.pug
+++ b/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.tpl.pug
@@ -20,8 +20,10 @@
             .ui-grid-header-canvas
                 .ui-grid-header-cell-wrapper(ng-style='colContainer.headerCellWrapperStyle()')
                     .ui-grid-header-cell-row(role='row')
-                        .ui-grid-header-span.ui-grid-header-cell.ui-grid-clearfix(ng-repeat='cat in grid.options.categories')
+                        .ui-grid-header-span.ui-grid-header-cell.ui-grid-clearfix.ui-grid-category(ng-repeat='cat in grid.options.categories', ng-if='cat.visible && \
+                        (colContainer.renderedColumns | uiGridSubcategories: cat.name).length > 0')
                             div(ng-show='(colContainer.renderedColumns|uiGridSubcategories:cat.name).length > 1')
                                 .ui-grid-cell-contents {{ cat.name }}
                             .ui-grid-header-cell-row
                                 .ui-grid-header-cell.ui-grid-clearfix(ng-repeat='col in (colContainer.renderedColumns|uiGridSubcategories:cat.name) track by col.uid' ui-grid-header-cell='' col='col' render-index='$index')
+                        .ui-grid-header-cell.ui-grid-clearfix(ng-if='col.colDef.name === "treeBaseRowHeaderCol"' ng-repeat='col in colContainer.renderedColumns track by col.uid' ui-grid-header-cell='' col='col' render-index='$index')

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/components/ui-grid-settings/ui-grid-settings.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/ui-grid-settings/ui-grid-settings.scss b/modules/web-console/frontend/app/components/ui-grid-settings/ui-grid-settings.scss
index 24f4d9b..d0a31f0 100644
--- a/modules/web-console/frontend/app/components/ui-grid-settings/ui-grid-settings.scss
+++ b/modules/web-console/frontend/app/components/ui-grid-settings/ui-grid-settings.scss
@@ -129,6 +129,16 @@
                     width: 60%;
                 }
             }
+
+            &:nth-child(2) {
+                float: left;
+
+                width: 100px;
+
+                .ignite-form-field__control {
+                    width: 100%;
+                }
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/primitives/badge/index.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/badge/index.scss b/modules/web-console/frontend/app/primitives/badge/index.scss
new file mode 100644
index 0000000..837ab5b
--- /dev/null
+++ b/modules/web-console/frontend/app/primitives/badge/index.scss
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+@import '../../../public/stylesheets/variables';
+
+.badge {
+  display: inline-block;
+  min-width: 26px;
+  height: 18px;
+
+  padding: 2px 9px;
+
+  border-radius: 9px;
+
+  color: white;
+  font-family: Roboto;
+  font-size: 12px;
+  text-align: center;
+  line-height: 12px;
+
+  background-color: $brand-primary;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/primitives/index.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/index.js b/modules/web-console/frontend/app/primitives/index.js
new file mode 100644
index 0000000..7940f7a
--- /dev/null
+++ b/modules/web-console/frontend/app/primitives/index.js
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import './badge/index.scss';
+import './tabs/index.scss';

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/app/primitives/tabs/index.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/tabs/index.scss b/modules/web-console/frontend/app/primitives/tabs/index.scss
new file mode 100644
index 0000000..eed88cb
--- /dev/null
+++ b/modules/web-console/frontend/app/primitives/tabs/index.scss
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+@import '../../../public/stylesheets/variables';
+
+ul.tabs {
+    $width: auto;
+    $height: 40px;
+    $offset-vertical: 11px;
+    $offset-horizontal: 25px;
+    $font-size: 14px;
+
+    list-style: none;
+
+    padding-left: 0;
+    border-bottom: 1px solid $nav-tabs-border-color;
+
+    li {
+        position: relative;
+        top: 1px;
+
+        display: inline-block;
+
+        border-bottom: 5px solid transparent;
+
+        a {
+            display: inline-block;
+            width: $width;
+            height: $height;
+
+            padding: $offset-vertical $offset-horizontal;
+
+            color: $text-color;
+            font-size: $font-size;
+            text-align: center;
+            line-height: $height - 2*$offset-vertical;
+
+            &:hover {
+              text-decoration: none;
+            }
+
+            .badge {
+              margin-left: $offset-vertical;
+            }
+        }
+
+        &.active {
+            border-color: $brand-primary;
+        }
+
+        &:not(.active):hover {
+            border-color: lighten($brand-primary, 25%);
+        }
+
+        & + li {
+            margin-left: 45px;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/827befb7/modules/web-console/frontend/public/stylesheets/style.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/stylesheets/style.scss b/modules/web-console/frontend/public/stylesheets/style.scss
index a07472e..2f4966f 100644
--- a/modules/web-console/frontend/public/stylesheets/style.scss
+++ b/modules/web-console/frontend/public/stylesheets/style.scss
@@ -2302,6 +2302,8 @@ html,body,.splash-screen {
 
 .admin-page {
     .panel-heading {
+        height: 38px;
+
         border-bottom: 0;
         padding-bottom: 0;
 


[22/23] ignite git commit: Cleaning up indexing module.

Posted by vo...@apache.org.
Cleaning up indexing module.


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

Branch: refs/heads/ignite-4565-ddl
Commit: f39de03c6d12ae01a77a162f83a6c95742cb8f27
Parents: b35568e
Author: devozerov <vo...@gridgain.com>
Authored: Wed Mar 22 13:04:18 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Mar 22 13:04:18 2017 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |   6 +-
 .../query/h2/ddl/DdlStatementsProcessor.java    | 466 ++-----------------
 .../h2/ddl/msg/DdlInitDiscoveryMessage.java     |   2 +-
 .../query/h2/ddl/GridDdlProtoTest.java          | 188 --------
 4 files changed, 30 insertions(+), 632 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f39de03c/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 07c0650..e61bfd6 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
@@ -1498,7 +1498,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
                     if (DdlStatementsProcessor.isDdlStatement(prepared)) {
                         try {
-                            return ddlProc.runDdlStatement(stmt);
+                            return ddlProc.runDdlStatement(cctx.name(), stmt);
                         }
                         catch (IgniteCheckedException e) {
                             throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + sqlQry + ']',
@@ -2200,10 +2200,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             log.debug("Stopping cache query index...");
 
 //        unregisterMBean(); TODO https://issues.apache.org/jira/browse/IGNITE-2139
-
-        if (ddlProc != null)
-            ddlProc.stop();
-
         for (Schema schema : schemas.values())
             schema.onDrop();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f39de03c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
index 8bb831b..32e6da5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
@@ -17,50 +17,27 @@
 
 package org.apache.ignite.internal.processors.query.h2.ddl;
 
-import java.sql.PreparedStatement;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.query.QueryCursor;
-import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.GridTopic;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.managers.communication.GridIoPolicy;
-import org.apache.ignite.internal.managers.communication.GridMessageListener;
-import org.apache.ignite.internal.managers.discovery.CustomEventListener;
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
-import org.apache.ignite.internal.processors.query.ddl.DdlOperationNodeResult;
-import org.apache.ignite.internal.processors.query.ddl.DdlOperationResult;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
-import org.apache.ignite.internal.processors.query.h2.ddl.msg.DdlAckDiscoveryMessage;
-import org.apache.ignite.internal.processors.query.h2.ddl.msg.DdlInitDiscoveryMessage;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlCreateIndex;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDropIndex;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.internal.util.worker.GridWorker;
-import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.thread.IgniteThread;
 import org.h2.command.Prepared;
 import org.h2.command.ddl.CreateIndex;
 import org.h2.command.ddl.DropIndex;
 import org.h2.jdbc.JdbcPreparedStatement;
-import org.jetbrains.annotations.Nullable;
+
+import java.sql.PreparedStatement;
+import java.util.Collections;
+import java.util.List;
 
 import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.UPDATE_RESULT_META;
 
@@ -75,15 +52,6 @@ public class DdlStatementsProcessor {
     /** Logger. */
     private IgniteLogger log;
 
-    /** State flag. */
-    private AtomicBoolean isStopped = new AtomicBoolean();
-
-    /** Running operations originating at this node as a client. */
-    private Map<IgniteUuid, GridFutureAdapter> operations = new ConcurrentHashMap<>();
-
-    /** Worker. */
-    private volatile DdlWorker worker;
-
     /**
      * Initialize message handlers and this' fields needed for further operation.
      *
@@ -94,378 +62,49 @@ public class DdlStatementsProcessor {
         this.ctx = ctx;
 
         log = ctx.log(DdlStatementsProcessor.class);
-
-        worker = new DdlWorker(ctx.igniteInstanceName(), log);
-
-        IgniteThread workerThread = new IgniteThread(worker);
-
-        workerThread.setDaemon(true);
-
-        workerThread.start();
-
-        ctx.discovery().setCustomEventListener(DdlInitDiscoveryMessage.class,
-            new CustomEventListener<DdlInitDiscoveryMessage>() {
-            /** {@inheritDoc} */
-            @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "unchecked"})
-            @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd,
-                DdlInitDiscoveryMessage msg) {
-                onInit(msg);
-            }
-        });
-
-        ctx.discovery().setCustomEventListener(DdlAckDiscoveryMessage.class,
-            new CustomEventListener<DdlAckDiscoveryMessage>() {
-            /** {@inheritDoc} */
-            @Override public void onCustomEvent(AffinityTopologyVersion topVer, final ClusterNode snd,
-                final DdlAckDiscoveryMessage msg) {
-                submitTask(new DdlTask() {
-                    @Override public void run() {
-                        onAck(snd, msg);
-                    }
-                });
-            }
-        });
-
-        ctx.io().addMessageListener(GridTopic.TOPIC_QUERY, new GridMessageListener() {
-            /** {@inheritDoc} */
-            @Override public void onMessage(UUID nodeId, Object msg) {
-                if (msg instanceof DdlOperationResult) {
-                    DdlOperationResult res = (DdlOperationResult) msg;
-
-                    onResult(res.getOperationId(), bytesToException(res.getError()));
-                }
-
-                if (msg instanceof DdlOperationNodeResult) {
-                    DdlOperationNodeResult res = (DdlOperationNodeResult) msg;
-
-                    onNodeResult(res.getOperationId(), bytesToException(res.getError()));
-                }
-            }
-        });
-    }
-
-    /**
-     * Submit a task to {@link #worker} for async execution.
-     *
-     * @param task Task.
-     */
-    private void submitTask(DdlTask task) {
-        DdlWorker worker0 = worker;
-
-        if (worker0 != null)
-            worker0.submit(task);
-        else
-            log.debug("Cannot submit DDL task because worker is null (node is stopping): " + task);
-    }
-
-    /**
-     * Handle {@code ACK} message on a <b>peer node</b> - do local portion of actual DDL job and notify
-     * <b>coordinator</b> about success or failure.
-     *
-     * @param snd Sender.
-     * @param msg Message.
-     */
-    @SuppressWarnings({"ThrowableInstanceNeverThrown", "unchecked"})
-    private void onAck(ClusterNode snd, DdlAckDiscoveryMessage msg) {
-        // Don't do anything if we didn't choose to participate.
-        if (!msg.nodeIds().contains(ctx.localNodeId()))
-            return;
-
-        IgniteCheckedException ex = null;
-
-        DdlAbstractOperation args = msg.operation();
-
-        try {
-            doAck(args);
-        }
-        catch (Throwable e) {
-            ex = wrapThrowableIfNeeded(e);
-        }
-
-        try {
-            DdlOperationNodeResult res = new DdlOperationNodeResult();
-
-            res.setOperationId(msg.operation().operationId());
-            res.setError(exceptionToBytes(ex));
-
-            ctx.io().sendToGridTopic(snd, GridTopic.TOPIC_QUERY, res, GridIoPolicy.IDX_POOL);
-        }
-        catch (Throwable e) {
-            U.error(log, "Failed to notify coordinator about local DLL operation completion [opId=" +
-                msg.operation().operationId() + ", clientNodeId=" + snd.id() + ']', e);
-        }
-    }
-
-    /**
-     * Perform local portion of DDL operation.
-     * Exists as a separate method to allow overriding it in tests to check behavior in case of errors.
-     *
-     * @param args Operation arguments.
-     * @throws IgniteCheckedException if failed.
-     */
-    @SuppressWarnings("unchecked")
-    void doAck(DdlAbstractOperation args) throws IgniteCheckedException {
-        if (args instanceof DdlCreateIndexOperation) {
-            // No-op.
-        }
-    }
-
-    /**
-     * Handle local DDL operation result from <b>a peer node</b> on <b>the coordinator</b>.
-     *
-     * @param opId DDL operation ID.
-     * @param err Exception that occurred on the <b>peer</b>, or null if the local operation has been successful.
-     */
-    @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "SynchronizationOnLocalVariableOrMethodParameter", "ForLoopReplaceableByForEach"})
-    private void onNodeResult(IgniteUuid opId, IgniteCheckedException err) {
-        // No-op.
-    }
-
-    /**
-     * Process result of executing {@link DdlInitDiscoveryMessage} and react accordingly.
-     * Called from {@link DdlInitDiscoveryMessage#ackMessage()}.
-     *
-     * @param msg {@link DdlInitDiscoveryMessage} message.
-     * @return {@link DiscoveryCustomMessage} to return from {@link DdlInitDiscoveryMessage#ackMessage()}.
-     */
-    @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "UnnecessaryInitCause"})
-    public DiscoveryCustomMessage onInitFinished(DdlInitDiscoveryMessage msg) {
-        Map<UUID, IgniteCheckedException> nodesState = msg.nodeState();
-
-        assert nodesState != null;
-
-        Map<UUID, IgniteCheckedException> errors = new HashMap<>();
-
-        for (Map.Entry<UUID, IgniteCheckedException> e : nodesState.entrySet())
-            if (e.getValue() != null)
-                errors.put(e.getKey(), e.getValue());
-
-        if (!errors.isEmpty()) {
-            IgniteCheckedException resEx = new IgniteCheckedException("DDL operation has been cancelled at INIT stage");
-
-            if (errors.size() > 1) {
-                for (IgniteCheckedException e : errors.values())
-                    resEx.addSuppressed(e);
-            }
-            else
-                resEx.initCause(errors.values().iterator().next());
-
-            sendResult(msg.operation(), resEx);
-
-            return null;
-        }
-        else
-            return new DdlAckDiscoveryMessage(msg.operation(), msg.nodeState().keySet());
-    }
-
-    /**
-     * Notify client about result.
-     *
-     * @param args Operation arguments.
-     * @param err Error, if any.
-     */
-    private void sendResult(DdlAbstractOperation args, IgniteCheckedException err) {
-        assert args != null;
-
-        DdlOperationResult res = new DdlOperationResult();
-
-        res.setError(exceptionToBytes(err));
-        res.setOperationId(args.operationId());
-
-        try {
-            ctx.io().sendToGridTopic(args.clientNodeId(), GridTopic.TOPIC_QUERY, res, GridIoPolicy.IDX_POOL);
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to notify client node about DDL operation failure " +
-                "[opId=" + args.operationId() + ", clientNodeId=" + args.clientNodeId() + ']', e);
-        }
-    }
-
-    /**
-     * Callback handling whole DDL operation result <b>on the client</b>.
-     *
-     * @param opId DDL operation ID.
-     * @param err Error, if any.
-     */
-    @SuppressWarnings("unchecked")
-    private void onResult(IgniteUuid opId, IgniteCheckedException err) {
-        GridFutureAdapter fut = operations.get(opId);
-
-        if (fut == null) {
-            U.warn(log, "DDL operation not found at its client [opId=" + opId + ", nodeId=" + ctx.localNodeId() + ']');
-
-            return;
-        }
-
-        fut.onDone(null, err);
-    }
-
-    /**
-     * Perform preliminary actions and checks for {@code INIT} stage of DDL statement execution <b>on a peer node</b>.
-     *
-     * @param msg {@code INIT} message.
-     */
-    @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
-    private void onInit(DdlInitDiscoveryMessage msg) {
-        try {
-            // Let's tell everyone that we're participating if our init is successful...
-            if (doInit(msg.operation()))
-                msg.nodeState().put(ctx.localNodeId(), null);
-        }
-        catch (Throwable e) {
-            // Or tell everyone about the error that occurred
-            msg.nodeState().put(ctx.localNodeId(), wrapThrowableIfNeeded(e));
-        }
-    }
-
-    /**
-     * Perform actual INIT actions.
-     * Exists as a separate method to allow overriding it in tests to check behavior in case of errors.
-     *
-     * @param args Operation arguments.
-     * @throws IgniteCheckedException if failed.
-     */
-    @SuppressWarnings("unchecked")
-    boolean doInit(DdlAbstractOperation args) throws IgniteCheckedException {
-        if (args instanceof DdlCreateIndexOperation) {
-            return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * Optionally wrap a {@link Throwable} into an {@link IgniteCheckedException}.
-     *
-     * @param e Throwable to wrap.
-     * @return {@code e} if it's an {@link IgniteCheckedException} or an {@link IgniteCheckedException} wrapping it.
-     */
-    private static IgniteCheckedException wrapThrowableIfNeeded(Throwable e) {
-        if (e instanceof IgniteCheckedException)
-            return (IgniteCheckedException) e;
-        else
-            return new IgniteCheckedException(e);
-    }
-
-    /**
-     * Do cleanup.
-     */
-    public void stop() throws IgniteCheckedException {
-        if (!isStopped.compareAndSet(false, true))
-            throw new IgniteCheckedException(new IllegalStateException("DDL processor has been stopped already"));
-
-        DdlWorker worker0 = worker;
-
-        if (worker0 != null) {
-            worker0.cancel();
-
-            worker = null;
-        }
-
-        for (Map.Entry<IgniteUuid, GridFutureAdapter> e : operations.entrySet())
-            e.getValue().onDone(new IgniteCheckedException("Operation has been cancelled [opId=" + e.getKey() +']'));
     }
 
     /**
      * Execute DDL statement.
      *
+     * @param cacheName Cache name.
      * @param stmt H2 statement to parse and execute.
      */
     @SuppressWarnings("unchecked")
-    public QueryCursor<List<?>> runDdlStatement(PreparedStatement stmt)
+    public QueryCursor<List<?>> runDdlStatement(String cacheName, PreparedStatement stmt)
         throws IgniteCheckedException {
-        if (isStopped.get())
-            throw new IgniteCheckedException(new IllegalStateException("DDL processor has been stopped"));
-
         assert stmt instanceof JdbcPreparedStatement;
 
-        GridSqlStatement gridStmt = new GridSqlQueryParser(false).parse(GridSqlQueryParser.prepared(stmt));
-
-        DdlAbstractOperation op;
-
-        if (gridStmt instanceof GridSqlCreateIndex) {
-            GridSqlCreateIndex createIdx = (GridSqlCreateIndex) gridStmt;
-
-            op = new DdlCreateIndexOperation(IgniteUuid.randomUuid(), ctx.localNodeId(), createIdx.index(),
-                createIdx.schemaName(), createIdx.tableName(), createIdx.ifNotExists());
-        }
-        else if (gridStmt instanceof GridSqlDropIndex)
-            throw new UnsupportedOperationException("DROP INDEX");
-        else
-            throw new IgniteSQLException("Unexpected DDL operation [type=" + gridStmt.getClass() + ']',
-                IgniteQueryErrorCode.UNEXPECTED_OPERATION);
-
-        GridFutureAdapter opFut = new GridFutureAdapter();
-
-        operations.put(op.operationId(), opFut);
+        IgniteInternalFuture fut;
 
         try {
-            ctx.discovery().sendCustomEvent(new DdlInitDiscoveryMessage(op));
+            GridSqlStatement gridStmt = new GridSqlQueryParser(false).parse(GridSqlQueryParser.prepared(stmt));
 
-            opFut.get();
-        }
-        finally {
-            operations.remove(op.operationId());
-        }
+            if (gridStmt instanceof GridSqlCreateIndex) {
+                GridSqlCreateIndex createIdx = (GridSqlCreateIndex) gridStmt;
 
-        QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
-            (Collections.singletonList(0L)), null, false);
-
-        resCur.fieldsMeta(UPDATE_RESULT_META);
-
-        return resCur;
-    }
-
-    /**
-     * Serialize exception or at least its message to bytes.
-     *
-     * @param ex Exception.
-     * @return Serialized exception.
-     */
-    private byte[] exceptionToBytes(IgniteCheckedException ex) {
-        if (ex == null)
-            return null;
-
-        try {
-            return U.marshal(ctx, ex);
-        }
-        catch (IgniteCheckedException e) {
-            IgniteCheckedException resEx;
-
-            // Let's try to serialize at least the message
-            try {
-                resEx = new IgniteCheckedException("Failed to serialize exception " +
-                    "[msg=" + ex.getMessage() + ']');
-            }
-            catch (Throwable ignored) {
-                resEx = new IgniteCheckedException("Failed to serialize exception");
+                // TODO: How to handle schema name properly?
+                fut = ctx.cache().dynamicIndexCreate(
+                    cacheName, createIdx.tableName(), createIdx.index(), createIdx.ifNotExists());
             }
+            else if (gridStmt instanceof GridSqlDropIndex)
+                throw new UnsupportedOperationException("DROP INDEX");
+            else
+                throw new IgniteSQLException("Unexpected DDL operation [type=" + gridStmt.getClass() + ']',
+                    IgniteQueryErrorCode.UNEXPECTED_OPERATION);
 
-            try {
-                return U.marshal(ctx, resEx);
-            }
-            catch (IgniteCheckedException exx) {
-                // Why would it fail? We've sanitized it...
-                throw new AssertionError(exx);
-            }
-        }
-    }
+            fut.get();
 
-    /**
-     * Deserialize exception from bytes.
-     *
-     * @param ex Exception.
-     * @return Serialized exception.
-     */
-    private IgniteCheckedException bytesToException(byte[] ex) {
-        if (ex == null)
-            return null;
+            QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
+                (Collections.singletonList(0L)), null, false);
 
-        try {
-            return U.unmarshal(ctx, ex, U.resolveClassLoader(ctx.config()));
+            resCur.fieldsMeta(UPDATE_RESULT_META);
+
+            return resCur;
         }
-        catch (Throwable e) {
-            return new IgniteCheckedException("Failed to deserialize exception", e);
+        catch (Exception e) {
+            // TODO: Proper error handling.
+            throw new IgniteSQLException("DLL operation failed.", e);
         }
     }
 
@@ -476,53 +115,4 @@ public class DdlStatementsProcessor {
     public static boolean isDdlStatement(Prepared cmd) {
         return cmd instanceof CreateIndex || cmd instanceof DropIndex;
     }
-
-    /**
-     * DDL worker.
-     */
-    private class DdlWorker extends GridWorker {
-        /** Worker queue. */
-        private final BlockingQueue<DdlTask> queue = new LinkedBlockingDeque<>();
-
-        /**
-         * Constructor.
-         *
-         * @param gridName Gird name.
-         * @param log Logger.
-         */
-        public DdlWorker(@Nullable String gridName, IgniteLogger log) {
-            super(gridName, "indexing-ddl-worker", log);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
-            while (!isCancelled()) {
-                DdlTask task = queue.take();
-
-                try {
-                    task.run();
-                }
-                catch (Exception e) {
-                    U.error(log, "Unexpected exception during DDL task processing [task=" + task + ']', e);
-                }
-                catch (Throwable t) {
-                    U.error(log, "Unexpected error during DDL task processing (worker will be stopped) [task=" +
-                        task + ']', t);
-
-                    throw t;
-                }
-            }
-        }
-
-        /**
-         * Submit task.
-         *
-         * @param task Task.
-         */
-        public void submit(DdlTask task) {
-            assert task != null;
-
-            queue.add(task);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f39de03c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/msg/DdlInitDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/msg/DdlInitDiscoveryMessage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/msg/DdlInitDiscoveryMessage.java
index 753eb0c..4e00d54 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/msg/DdlInitDiscoveryMessage.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/msg/DdlInitDiscoveryMessage.java
@@ -61,7 +61,7 @@ public class DdlInitDiscoveryMessage extends DdlAbstractDiscoveryMessage impleme
     /** {@inheritDoc} */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
     @Nullable @Override public DiscoveryCustomMessage ackMessage() {
-        return ((IgniteH2Indexing)ctx.query().getIndexing()).getDdlStatementsProcessor().onInitFinished(this);
+        return null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f39de03c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ddl/GridDdlProtoTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ddl/GridDdlProtoTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ddl/GridDdlProtoTest.java
deleted file mode 100644
index d507dff..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ddl/GridDdlProtoTest.java
+++ /dev/null
@@ -1,188 +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.ddl;
-
-import java.io.Serializable;
-import java.util.concurrent.Callable;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
-import org.apache.ignite.cache.query.annotations.QuerySqlField;
-import org.apache.ignite.configuration.CacheConfiguration;
-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.ddl.msg.DdlInitDiscoveryMessage;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- *
- */
-public class GridDdlProtoTest extends GridCommonAbstractTest {
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override protected void beforeTestsStarted() throws Exception {
-        IgniteH2Indexing.ddlProcCls = DdlProc.class;
-
-        startGridsMultiThreaded(3, true);
-
-        ignite(0).createCache(cacheConfig("S2P", true, false).setIndexedTypes(String.class, Person.class));
-
-        startGrid(getTestIgniteInstanceName(3), getConfiguration().setClientMode(true));
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        DdlProc.testName = null;
-
-        ignite(0).cache("S2P").clear();
-
-        ignite(0).cache("S2P").put("FirstKey", new Person(1, "John", "White"));
-        ignite(0).cache("S2P").put("SecondKey", new Person(2, "Joe", "Black"));
-        ignite(0).cache("S2P").put("k3", new Person(3, "Sylvia", "Green"));
-        ignite(0).cache("S2P").put("f0u4thk3y", new Person(4, "Jane", "Silver"));
-    }
-
-    /** Test behavior in case of INIT failure (cancel via {@link DdlInitDiscoveryMessage#ackMessage}). */
-    public void testInitFailure() {
-        DdlProc.testName = GridTestUtils.getGridTestName();
-
-        assertCreateIndexThrowsWithMessage("DDL operation has been cancelled at INIT stage", false);
-    }
-
-    /**
-     * Test error handling.
-     *
-     * @param msg Expected message.
-     * @param loc Run query locally on single node.
-     */
-    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    private void assertCreateIndexThrowsWithMessage(String msg, final boolean loc) {
-        final Throwable e = GridTestUtils.assertThrows(null, new Callable<Object>() {
-            /** {@inheritDoc} */
-            @Override public Object call() throws Exception {
-                ignite(3).cache("S2P").query(new SqlFieldsQuery("create index idx on Person(id desc)").setLocal(loc));
-                return null;
-            }
-        }, IgniteSQLException.class, "Failed to execute DDL statement");
-
-        GridTestUtils.assertThrows(null, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                throw (Exception) e.getCause();
-            }
-        }, IgniteCheckedException.class, msg);
-    }
-
-    /**
-     * @param name Cache name.
-     * @param partitioned Partition or replicated cache.
-     * @param escapeSql whether identifiers should be quoted - see {@link CacheConfiguration#setSqlEscapeAll}
-     * @return Cache configuration.
-     */
-    protected static CacheConfiguration cacheConfig(String name, boolean partitioned, boolean escapeSql) {
-        return new CacheConfiguration()
-            .setName(name)
-            .setCacheMode(partitioned ? CacheMode.PARTITIONED : CacheMode.REPLICATED)
-            .setAtomicityMode(CacheAtomicityMode.ATOMIC)
-            .setBackups(1)
-            .setSqlEscapeAll(escapeSql);
-    }
-    
-    /**
-     *
-     */
-    static class Person implements Serializable {
-        /** */
-        public Person(int id, String name, String secondName) {
-            this.id = id;
-            this.name = name;
-            this.secondName = secondName;
-        }
-
-        /** */
-        @QuerySqlField
-        protected int id;
-
-        /** */
-        @QuerySqlField(name = "firstName")
-        protected final String name;
-
-        /** */
-        @QuerySqlField
-        final String secondName;
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-
-            Person person = (Person) o;
-
-            return id == person.id && name.equals(person.name) && secondName.equals(person.secondName);
-
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            int res = id;
-            res = 31 * res + name.hashCode();
-            res = 31 * res + secondName.hashCode();
-            return res;
-        }
-    }
-
-    /**
-     * Custom implementation to test behavior on failure during various stages.
-     */
-    public final static class DdlProc extends DdlStatementsProcessor {
-        /** Name of current test. */
-        private static volatile String testName;
-
-        /** {@inheritDoc} */
-        @Override boolean doInit(DdlAbstractOperation args) {
-            // Let's throw an exception on a single node in the ring
-            if ("InitFailure".equals(testName) && ctx.igniteInstanceName().endsWith("2"))
-                throw new RuntimeException("Hello from DdlProc Init");
-            else
-                try {
-                    return super.doInit(args);
-                }
-                catch (IgniteCheckedException e) {
-                    throw new IgniteException(e);
-                }
-        }
-
-        /** {@inheritDoc}
-         * @param args*/
-        @Override void doAck(DdlAbstractOperation args) {
-            if ("AckFailure".equals(testName) && ctx.igniteInstanceName().endsWith("1"))
-                throw new RuntimeException("Hello from DdlProc Ack");
-            else
-                try {
-                    super.doInit(args);
-                }
-                catch (IgniteCheckedException e) {
-                    throw new IgniteException(e);
-                }
-        }
-    }
-}


[16/23] ignite git commit: IGNITE-4846 .NET: Support complex type dictionaries in app.config configuration

Posted by vo...@apache.org.
IGNITE-4846 .NET: Support complex type dictionaries in app.config configuration

This closes #1653


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

Branch: refs/heads/ignite-4565-ddl
Commit: 3b89a5ce78fadf4e39c8c927afd8ffef7b9fa186
Parents: b4cc8a7
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Mar 21 18:08:03 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Mar 21 18:08:03 2017 +0300

----------------------------------------------------------------------
 .../IgniteConfigurationSerializerTest.cs        |  44 +++-
 .../Common/IgniteConfigurationXmlSerializer.cs  | 250 ++++++++++++-------
 2 files changed, 204 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3b89a5ce/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 2f9366e..4335d11 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -126,7 +126,10 @@ namespace Apache.Ignite.Core.Tests
                                 <int>TaskFailed</int>
                                 <int>JobFinished</int>
                             </includedEventTypes>
-                            <userAttributes><pair key='myNode' value='true' /></userAttributes>
+                            <userAttributes>
+                                <pair key='myNode' value='true' />
+                                <pair key='foo'><value type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests'><bar>Baz</bar></value></pair>
+                            </userAttributes>
                             <atomicConfiguration backups='2' cacheMode='Local' atomicSequenceReserveSize='250' />
                             <transactionConfiguration defaultTransactionConcurrency='Optimistic' defaultTransactionIsolation='RepeatableRead' defaultTimeout='0:1:2' pessimisticTransactionLogSize='15' pessimisticTransactionLogLinger='0:0:33' />
                             <logger type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+TestLogger, Apache.Ignite.Core.Tests' />
@@ -202,7 +205,11 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(99, af.Partitions);
             Assert.IsTrue(af.ExcludeNeighbors);
 
-            Assert.AreEqual(new Dictionary<string, object> {{"myNode", "true"}}, cfg.UserAttributes);
+            Assert.AreEqual(new Dictionary<string, object>
+            {
+                {"myNode", "true"},
+                {"foo", new FooClass {Bar = "Baz"}}
+            }, cfg.UserAttributes);
 
             var atomicCfg = cfg.AtomicConfiguration;
             Assert.AreEqual(2, atomicCfg.Backups);
@@ -544,8 +551,9 @@ namespace Apache.Ignite.Core.Tests
                     Assert.IsNull(xVal);
                     Assert.IsNull(yVal);
                 }
-                else if (propType != typeof(string) && propType.IsGenericType 
-                    && propType.GetGenericTypeDefinition() == typeof (ICollection<>))
+                else if (propType != typeof(string) && propType.IsGenericType &&
+                         (propType.GetGenericTypeDefinition() == typeof(ICollection<>) ||
+                          propType.GetGenericTypeDefinition() == typeof(IDictionary<,>) ))
                 {
                     var xCol = ((IEnumerable) xVal).OfType<object>().ToList();
                     var yCol = ((IEnumerable) yVal).OfType<object>().ToList();
@@ -738,7 +746,8 @@ namespace Apache.Ignite.Core.Tests
                 SuppressWarnings = true,
                 WorkDirectory = @"c:\work",
                 IsDaemon = true,
-                UserAttributes = Enumerable.Range(1, 10).ToDictionary(x => x.ToString(), x => (object) x),
+                UserAttributes = Enumerable.Range(1, 10).ToDictionary(x => x.ToString(),
+                    x => x%2 == 0 ? (object) x : new FooClass {Bar = x.ToString()}),
                 AtomicConfiguration = new AtomicConfiguration
                 {
                     CacheMode = CacheMode.Replicated,
@@ -904,7 +913,30 @@ namespace Apache.Ignite.Core.Tests
         /// </summary>
         public class FooClass
         {
-            // No-op.
+            public string Bar { get; set; }
+
+            public override bool Equals(object obj)
+            {
+                if (ReferenceEquals(null, obj)) return false;
+                if (ReferenceEquals(this, obj)) return true;
+                if (obj.GetType() != GetType()) return false;
+                return string.Equals(Bar, ((FooClass) obj).Bar);
+            }
+
+            public override int GetHashCode()
+            {
+                return Bar != null ? Bar.GetHashCode() : 0;
+            }
+
+            public static bool operator ==(FooClass left, FooClass right)
+            {
+                return Equals(left, right);
+            }
+
+            public static bool operator !=(FooClass left, FooClass right)
+            {
+                return !Equals(left, right);
+            }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b89a5ce/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
index feb0f9e..6c5d620 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
@@ -153,9 +153,19 @@ namespace Apache.Ignite.Core.Impl.Common
         {
             var props = GetNonDefaultProperties(obj).OrderBy(x => x.Name).ToList();
 
-            // Specify type for interfaces and abstract classes
-            if (valueType.IsAbstract)
+            var realType = obj.GetType();
+
+            // Specify type when it differs from declared type.
+            if (valueType != realType)
+            {
                 writer.WriteAttributeString(TypNameAttribute, TypeStringConverter.Convert(obj.GetType()));
+            }
+
+            if (IsBasicType(obj.GetType()))
+            {
+                WriteBasicProperty(obj, writer, realType, null);
+                return;
+            }
 
             // Write attributes
             foreach (var prop in props.Where(p => IsBasicType(p.PropertyType) && !IsObsolete(p)))
@@ -181,10 +191,14 @@ namespace Apache.Ignite.Core.Impl.Common
             // Read attributes
             while (reader.MoveToNextAttribute())
             {
-                var name = reader.Name;
-                var val = reader.Value;
+                if (reader.Name == TypNameAttribute || reader.Name == XmlnsAttribute)
+                    continue;
+
+                var prop = GetPropertyOrThrow(reader.Name, reader.Value, target.GetType());
+
+                var value = ConvertBasicValue(reader.Value, prop, prop.PropertyType);
 
-                SetProperty(target, name, val);
+                prop.SetValue(target, value, null);
             }
 
             // Read content
@@ -195,64 +209,56 @@ namespace Apache.Ignite.Core.Impl.Common
                 if (reader.NodeType != XmlNodeType.Element)
                     continue;
 
-                var name = reader.Name;
-                var prop = GetPropertyOrThrow(name, reader.Value, targetType);
-                var propType = prop.PropertyType;
+                var prop = GetPropertyOrThrow(reader.Name, reader.Value, targetType);
 
-                if (IsBasicType(propType))
-                {
-                    // Regular property in xmlElement form
-                    SetProperty(target, name, reader.ReadString());
-                }
-                else if (propType.IsGenericType && propType.GetGenericTypeDefinition() == typeof (ICollection<>))
-                {
-                    // Collection
-                    ReadCollectionProperty(reader, prop, target, resolver);
-                }
-                else if (propType.IsGenericType && propType.GetGenericTypeDefinition() == typeof (IDictionary<,>))
-                {
-                    // Dictionary
-                    ReadDictionaryProperty(reader, prop, target);
-                }
-                else
-                {
-                    // Nested object (complex property)
-                    prop.SetValue(target, ReadComplexProperty(reader, propType, prop.Name, targetType, resolver), null);
-                }
+                var value = ReadPropertyValue(reader, resolver, prop, targetType);
+
+                prop.SetValue(target, value, null);
             }
         }
 
         /// <summary>
-        /// Reads the complex property (nested object).
+        /// Reads the property value.
         /// </summary>
-        private static object ReadComplexProperty(XmlReader reader, Type propType, string propName, Type targetType, 
-            TypeResolver resolver)
+        private static object ReadPropertyValue(XmlReader reader, TypeResolver resolver, 
+            PropertyInfo prop, Type targetType)
         {
-            if (propType.IsAbstract)
+            var propType = prop.PropertyType;
+
+            if (propType == typeof(object))
             {
-                var typeName = reader.GetAttribute(TypNameAttribute);
+                propType = ResolvePropertyType(reader, propType, prop.Name, targetType, resolver);
+            }
 
-                var derivedTypes = GetConcreteDerivedTypes(propType);
+            if (IsBasicType(propType))
+            {
+                // Regular property in xmlElement form.
+                return ConvertBasicValue(reader.ReadString(), prop, propType);
+            }
 
-                propType = typeName == null
-                    ? null
-                    : resolver.ResolveType(typeName) ?? derivedTypes.FirstOrDefault(x => x.Name == typeName);
+            if (propType.IsGenericType && propType.GetGenericTypeDefinition() == typeof(ICollection<>))
+            {
+                // Collection.
+                return ReadCollectionProperty(reader, prop, targetType, resolver);
+            }
 
-                if (propType == null)
-                {
-                    var message = string.Format("'type' attribute is required for '{0}.{1}' property", targetType.Name,
-                        propName);
+            if (propType.IsGenericType && propType.GetGenericTypeDefinition() == typeof(IDictionary<,>))
+            {
+                // Dictionary.
+                return ReadDictionaryProperty(reader, prop, resolver);
+            }
 
-                    if (typeName != null)
-                    {
-                        message += ", specified type cannot be resolved: " + typeName;
-                    }
-                    else if (derivedTypes.Any())
-                        message += ", possible values are: " + string.Join(", ", derivedTypes.Select(x => x.Name));
+            // Nested object (complex property).
+            return ReadComplexProperty(reader, propType, prop.Name, targetType, resolver);
+        }
 
-                    throw new ConfigurationErrorsException(message);
-                }
-            }
+        /// <summary>
+        /// Reads the complex property (nested object).
+        /// </summary>
+        private static object ReadComplexProperty(XmlReader reader, Type propType, string propName, Type targetType, 
+            TypeResolver resolver)
+        {
+            propType = ResolvePropertyType(reader, propType, propName, targetType, resolver);
 
             var nestedVal = Activator.CreateInstance(propType);
 
@@ -267,9 +273,46 @@ namespace Apache.Ignite.Core.Impl.Common
         }
 
         /// <summary>
+        /// Resolves the type of the property.
+        /// </summary>
+        private static Type ResolvePropertyType(XmlReader reader, Type propType, string propName, Type targetType,
+            TypeResolver resolver)
+        {
+            var typeName = reader.GetAttribute(TypNameAttribute);
+
+            if (!propType.IsAbstract && typeName == null)
+                return propType;
+
+            var res = typeName == null
+                ? null
+                : resolver.ResolveType(typeName) ??
+                  GetConcreteDerivedTypes(propType).FirstOrDefault(x => x.Name == typeName);
+
+            if (res != null)
+                return res;
+
+            var message = string.Format("'type' attribute is required for '{0}.{1}' property", targetType.Name,
+                propName);
+
+            var derivedTypes = GetConcreteDerivedTypes(propType);
+
+
+            if (typeName != null)
+            {
+                message += ", specified type cannot be resolved: " + typeName;
+            }
+            else if (derivedTypes.Any())
+            {
+                message += ", possible values are: " + string.Join(", ", derivedTypes.Select(x => x.Name));
+            }
+
+            throw new ConfigurationErrorsException(message);
+        }
+
+        /// <summary>
         /// Reads the collection.
         /// </summary>
-        private static void ReadCollectionProperty(XmlReader reader, PropertyInfo prop, object target, 
+        private static IList ReadCollectionProperty(XmlReader reader, PropertyInfo prop, Type targetType, 
             TypeResolver resolver)
         {
             var elementType = prop.PropertyType.GetGenericArguments().Single();
@@ -295,22 +338,24 @@ namespace Apache.Ignite.Core.Impl.Common
 
                     list.Add(converter != null
                         ? converter.ConvertFromInvariantString(subReader.ReadString())
-                        : ReadComplexProperty(subReader, elementType, prop.Name, target.GetType(), resolver));
+                        : ReadComplexProperty(subReader, elementType, prop.Name, targetType, resolver));
                 }
             }
 
-            prop.SetValue(target, list, null);
+            return list;
         }
         
         /// <summary>
         /// Reads the dictionary.
         /// </summary>
-        private static void ReadDictionaryProperty(XmlReader reader, PropertyInfo prop, object target)
+        private static IDictionary ReadDictionaryProperty(XmlReader reader, PropertyInfo prop, TypeResolver resolver)
         {
             var keyValTypes = prop.PropertyType.GetGenericArguments();
 
             var dictType = typeof (Dictionary<,>).MakeGenericType(keyValTypes);
 
+            var pairType = typeof(Pair<,>).MakeGenericType(keyValTypes);
+
             var dict = (IDictionary) Activator.CreateInstance(dictType);
 
             using (var subReader = reader.ReadSubtree())
@@ -326,42 +371,29 @@ namespace Apache.Ignite.Core.Impl.Common
                             string.Format("Invalid dictionary element in IgniteConfiguration: expected '{0}', " +
                                           "but was '{1}'", KeyValPairElement, subReader.Name));
 
-                    var key = subReader.GetAttribute("key");
+                    var pair = (IPair) Activator.CreateInstance(pairType);
 
-                    if (key == null)
-                        throw new ConfigurationErrorsException(
-                            "Invalid dictionary entry, key attribute is missing for property " + prop);
+                    var pairReader = subReader.ReadSubtree();
 
-                    dict[key] = subReader.GetAttribute("value");
+                    pairReader.Read();
+
+                    ReadElement(pairReader, pair, resolver);
+
+                    dict[pair.Key] = pair.Value;
                 }
             }
 
-            prop.SetValue(target, dict, null);
+            return dict;
         }
 
         /// <summary>
-        /// Sets the property.
+        /// Reads the basic value.
         /// </summary>
-        private static void SetProperty(object target, string propName, string propVal)
+        private static object ConvertBasicValue(string propVal, PropertyInfo property, Type propertyType)
         {
-            if (propName == TypNameAttribute || propName == XmlnsAttribute)
-                return;
+            var converter = GetConverter(property, propertyType);
 
-            var type = target.GetType();
-            var property = GetPropertyOrThrow(propName, propVal, type);
-
-            if (!property.CanWrite)
-            {
-                throw new ConfigurationErrorsException(string.Format(
-                        "Invalid IgniteConfiguration attribute '{0}={1}', property '{2}.{3}' is not writeable",
-                        propName, propVal, type, property.Name));
-            }
-
-            var converter = GetConverter(property, property.PropertyType);
-
-            var convertedVal = converter.ConvertFromInvariantString(propVal);
-
-            property.SetValue(target, convertedVal, null);
+            return converter.ConvertFromInvariantString(propVal);
         }
 
         /// <summary>
@@ -376,15 +408,24 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <summary>
         /// Gets specified property from a type or throws an exception.
         /// </summary>
-        private static PropertyInfo GetPropertyOrThrow(string propName, string propVal, Type type)
+        private static PropertyInfo GetPropertyOrThrow(string propName, object propVal, Type type)
         {
             var property = type.GetProperty(XmlNameToPropertyName(propName));
 
             if (property == null)
+            {
                 throw new ConfigurationErrorsException(
                     string.Format(
                         "Invalid IgniteConfiguration attribute '{0}={1}', there is no such property on '{2}'",
                         propName, propVal, type));
+            }
+
+            if (!property.CanWrite)
+            {
+                throw new ConfigurationErrorsException(string.Format(
+                        "Invalid IgniteConfiguration attribute '{0}={1}', property '{2}.{3}' is not writeable",
+                        propName, propVal, type, property.Name));
+            }
 
             return property;
         }
@@ -425,8 +466,7 @@ namespace Apache.Ignite.Core.Impl.Common
             if (IsKeyValuePair(propertyType))
                 return false;
 
-            return propertyType.IsValueType || propertyType == typeof (string) || propertyType == typeof (Type) ||
-                   propertyType == typeof (object);
+            return propertyType.IsValueType || propertyType == typeof (string) || propertyType == typeof (Type);
         }
 
         /// <summary>
@@ -444,7 +484,6 @@ namespace Apache.Ignite.Core.Impl.Common
         /// </summary>
         private static TypeConverter GetConverter(PropertyInfo property, Type propertyType)
         {
-            Debug.Assert(property != null);
             Debug.Assert(propertyType != null);
 
             if (propertyType.IsEnum)
@@ -456,7 +495,8 @@ namespace Apache.Ignite.Core.Impl.Common
             if (propertyType == typeof(bool))
                 return BooleanLowerCaseConverter.Instance;
 
-            if (property.DeclaringType == typeof (IgniteConfiguration) && property.Name == "IncludedEventTypes")
+            if (property != null &&
+                property.DeclaringType == typeof (IgniteConfiguration) && property.Name == "IncludedEventTypes")
                 return EventTypeConverter.Instance;
 
             if (propertyType == typeof (object))
@@ -508,5 +548,47 @@ namespace Apache.Ignite.Core.Impl.Common
 
             return property.GetCustomAttributes(typeof(ObsoleteAttribute), true).Any();
         }
+
+        /// <summary>
+        /// Non-generic Pair accessor.
+        /// </summary>
+        private interface IPair
+        {
+            /// <summary>
+            /// Gets the key.
+            /// </summary>
+            object Key { get; }
+
+            /// <summary>
+            /// Gets the value.
+            /// </summary>
+            object Value { get; }
+        }
+
+        /// <summary>
+        /// Surrogate dictionary entry to overcome immutable KeyValuePair.
+        /// </summary>
+        private class Pair<TK, TV> : IPair
+        {
+            // ReSharper disable once UnusedAutoPropertyAccessor.Local
+            // ReSharper disable once MemberCanBePrivate.Local
+            public TK Key { get; set; }
+
+            // ReSharper disable once UnusedAutoPropertyAccessor.Local
+            // ReSharper disable once MemberCanBePrivate.Local
+            public TV Value { get; set; }
+
+            /** <inheritdoc /> */
+            object IPair.Key
+            {
+                get { return Key; }
+            }
+
+            /** <inheritdoc /> */
+            object IPair.Value
+            {
+                get { return Value; }
+            }
+        }
     }
 }


[20/23] ignite git commit: Completion protocol works fine!

Posted by vo...@apache.org.
Completion protocol works fine!


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

Branch: refs/heads/ignite-4565-ddl
Commit: f1419d71a93c1f67eb1aefea40a3386509fa3708
Parents: edaf29a
Author: devozerov <vo...@gridgain.com>
Authored: Wed Mar 22 12:48:14 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Mar 22 12:48:14 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 32 +++++++++++---------
 .../query/QueryTypeDescriptorImpl.java          |  2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  2 +-
 3 files changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f1419d71/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 2e5ee50..c27b2f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -468,6 +468,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         if (!type0.fields().containsKey(idxField)) {
                             completed = true;
                             errMsg = "Field doesn't exist: " + idxField;
+
+                            break;
                         }
                     }
                 }
@@ -1385,6 +1387,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Process status response.
+     *
+     * @param resp Status response.
+     */
+    private void processStatusResponse(IndexOperationStatusResponse resp) {
+        IndexOperationState idxOpState = idxOpStates.get(resp.operationId());
+
+        if (idxOpState != null)
+            idxOpState.onNodeFinished(resp.senderNodeId(), resp.errorMessage());
+        else {
+            // TODO: Log!
+        }
+    }
+
+    /**
      * Send status response.
      *
      * @param destNodeId Destination node ID.
@@ -1405,21 +1422,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Process status response.
-     *
-     * @param resp Status response.
-     */
-    private void processStatusResponse(IndexOperationStatusResponse resp) {
-        IndexOperationState idxOpState = idxOpStates.get(resp.operationId());
-
-        if (idxOpState != null)
-            idxOpState.onNodeFinished(resp.senderNodeId(), resp.errorMessage());
-        else {
-            // TODO: Log!
-        }
-    }
-
-    /**
      * @param ver Version.
      */
     public static void setRequestAffinityTopologyVersion(AffinityTopologyVersion ver) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/f1419d71/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
index 8cc9c6c..287873d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -138,7 +138,7 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
      * @return Table name.
      */
     public String tableName() {
-        return tblName;
+        return tblName != null ? tblName : name;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f1419d71/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 dc66e5c..07c0650 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
@@ -2744,7 +2744,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             this.type = type;
             this.schema = schema;
 
-            String tblName = escapeName(type.tableName() != null ? type.tableName() : type.name(), schema.escapeAll());
+            String tblName = escapeName(type.tableName(), schema.escapeAll());
 
             fullTblName = schema.schemaName + "." + tblName;
         }