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

[01/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-991' into ignite-sprint-5

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-633 02780e419 -> 0b5b90ea8


Merge remote-tracking branch 'remotes/origin/ignite-991' into ignite-sprint-5


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

Branch: refs/heads/ignite-633
Commit: a03d111f26fd16faf0629bc11fe2ef201dee92f3
Parents: bf3203a b5ee09f
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 4 14:14:29 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 4 14:14:29 2015 +0300

----------------------------------------------------------------------
 .../dht/GridClientPartitionTopology.java        |  2 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  8 +-
 .../GridDhtPartitionsExchangeFuture.java        | 19 +++-
 ...niteDynamicCacheWithConfigStartSelfTest.java | 97 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |  1 +
 5 files changed, 118 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


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

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


[46/50] incubator-ignite git commit: # spring-5 - Removed confusing warning

Posted by sb...@apache.org.
# spring-5 - Removed confusing warning


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

Branch: refs/heads/ignite-633
Commit: 928be42cf309293a3cd9947b8614eb1bc045ddef
Parents: 91104a2
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Tue Jun 9 23:40:22 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Tue Jun 9 23:40:22 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/managers/indexing/GridIndexingManager.java   | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/928be42c/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
index 9a81cd1..f1561bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.indexing.*;
 
@@ -46,9 +45,6 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @Override public void start() throws IgniteCheckedException {
-        if (!enabled())
-            U.warn(log, "Indexing is disabled (to enable please configure GridIndexingSpi).");
-
         startSpi();
 
         if (log.isDebugEnabled())


[25/50] incubator-ignite git commit: # IGNITE-992 Review.

Posted by sb...@apache.org.
# IGNITE-992 Review.


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

Branch: refs/heads/ignite-633
Commit: 662f7337f7d03f94373f4ebd083d308823aff50c
Parents: a5b5ec7
Author: AKuznetsov <ak...@gridgain.com>
Authored: Mon Jun 8 16:44:25 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Mon Jun 8 16:44:25 2015 +0700

----------------------------------------------------------------------
 .../internal/util/IgniteExceptionRegistry.java  |  7 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |  9 ++-
 .../node/VisorNodeDataCollectorTaskResult.java  | 17 +++--
 .../internal/visor/query/VisorQueryJob.java     | 11 +--
 .../internal/visor/query/VisorQueryTask.java    |  3 +-
 .../visor/util/VisorExceptionWrapper.java       | 78 ++++++++++++++++++++
 .../internal/visor/util/VisorTaskUtils.java     | 10 +++
 7 files changed, 114 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/662f7337/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
index ab113d7..a56570a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.util;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.internal.visor.util.*;
 
 import java.io.*;
 import java.util.*;
@@ -160,7 +161,7 @@ public class IgniteExceptionRegistry {
 
         /** */
         @GridToStringExclude
-        private final Throwable error;
+        private final VisorExceptionWrapper error;
 
         /** */
         private final long threadId;
@@ -186,7 +187,7 @@ public class IgniteExceptionRegistry {
          */
         public ExceptionInfo(long order, Throwable error, String msg, long threadId, String threadName, long time) {
             this.order = order;
-            this.error = error;
+            this.error = VisorTaskUtils.wrap(error);
             this.threadId = threadId;
             this.threadName = threadName;
             this.time = time;
@@ -210,7 +211,7 @@ public class IgniteExceptionRegistry {
         /**
          * @return Suppressed error.
          */
-        public Throwable error() {
+        public VisorExceptionWrapper error() {
             return error;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/662f7337/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
index fde871b..7dbfd39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
@@ -23,6 +23,7 @@ import org.apache.ignite.compute.*;
 import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.visor.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -87,7 +88,7 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
                 else {
                     // Ignore nodes that left topology.
                     if (!(unhandledEx instanceof ClusterGroupEmptyException))
-                        taskRes.unhandledEx().put(nid, unhandledEx);
+                        taskRes.unhandledEx().put(nid, VisorTaskUtils.wrap(unhandledEx));
                 }
             }
         }
@@ -116,13 +117,13 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
             taskRes.events().addAll(jobRes.events());
 
         if (jobRes.eventsEx() != null)
-            taskRes.eventsEx().put(nid, jobRes.eventsEx());
+            taskRes.eventsEx().put(nid, VisorTaskUtils.wrap(jobRes.eventsEx()));
 
         if (!jobRes.caches().isEmpty())
             taskRes.caches().put(nid, jobRes.caches());
 
         if (jobRes.cachesEx() != null)
-            taskRes.cachesEx().put(nid, jobRes.cachesEx());
+            taskRes.cachesEx().put(nid, VisorTaskUtils.wrap(jobRes.cachesEx()));
 
         if (!jobRes.igfss().isEmpty())
             taskRes.igfss().put(nid, jobRes.igfss());
@@ -131,6 +132,6 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
             taskRes.igfsEndpoints().put(nid, jobRes.igfsEndpoints());
 
         if (jobRes.igfssEx() != null)
-            taskRes.igfssEx().put(nid, jobRes.igfssEx());
+            taskRes.igfssEx().put(nid, VisorTaskUtils.wrap(jobRes.igfssEx()));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/662f7337/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
index 6485978..1a4eb02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.visor.node;
 import org.apache.ignite.internal.visor.cache.*;
 import org.apache.ignite.internal.visor.event.*;
 import org.apache.ignite.internal.visor.igfs.*;
+import org.apache.ignite.internal.visor.util.*;
 
 import java.io.*;
 import java.util.*;
@@ -32,7 +33,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     private static final long serialVersionUID = 0L;
 
     /** Unhandled exceptions from nodes. */
-    private final Map<UUID, Throwable> unhandledEx = new HashMap<>();
+    private final Map<UUID, VisorExceptionWrapper> unhandledEx = new HashMap<>();
 
     /** Nodes grid names. */
     private final Map<UUID, String> gridNames = new HashMap<>();
@@ -50,13 +51,13 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     private final List<VisorGridEvent> evts = new ArrayList<>();
 
     /** Exceptions caught during collecting events from nodes. */
-    private final Map<UUID, Throwable> evtsEx = new HashMap<>();
+    private final Map<UUID, VisorExceptionWrapper> evtsEx = new HashMap<>();
 
     /** All caches collected from nodes. */
     private final Map<UUID, Collection<VisorCache>> caches = new HashMap<>();
 
     /** Exceptions caught during collecting caches from nodes. */
-    private final Map<UUID, Throwable> cachesEx = new HashMap<>();
+    private final Map<UUID, VisorExceptionWrapper> cachesEx = new HashMap<>();
 
     /** All IGFS collected from nodes. */
     private final Map<UUID, Collection<VisorIgfs>> igfss = new HashMap<>();
@@ -65,7 +66,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     private final Map<UUID, Collection<VisorIgfsEndpoint>> igfsEndpoints = new HashMap<>();
 
     /** Exceptions caught during collecting IGFS from nodes. */
-    private final Map<UUID, Throwable> igfssEx = new HashMap<>();
+    private final Map<UUID, VisorExceptionWrapper> igfssEx = new HashMap<>();
 
     /**
      * @return {@code true} If no data was collected.
@@ -88,7 +89,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     /**
      * @return Unhandled exceptions from nodes.
      */
-    public Map<UUID, Throwable> unhandledEx() {
+    public Map<UUID, VisorExceptionWrapper> unhandledEx() {
         return unhandledEx;
     }
 
@@ -123,7 +124,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     /**
      * @return Exceptions caught during collecting events from nodes.
      */
-    public Map<UUID, Throwable> eventsEx() {
+    public Map<UUID, VisorExceptionWrapper> eventsEx() {
         return evtsEx;
     }
 
@@ -137,7 +138,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     /**
      * @return Exceptions caught during collecting caches from nodes.
      */
-    public Map<UUID, Throwable> cachesEx() {
+    public Map<UUID, VisorExceptionWrapper> cachesEx() {
         return cachesEx;
     }
 
@@ -158,7 +159,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     /**
      * @return Exceptions caught during collecting IGFS from nodes.
      */
-    public Map<UUID, Throwable> igfssEx() {
+    public Map<UUID, VisorExceptionWrapper> igfssEx() {
         return igfssEx;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/662f7337/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 8915240..82555cc 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
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.visor.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 
 import javax.cache.*;
@@ -36,7 +37,7 @@ import static org.apache.ignite.internal.visor.query.VisorQueryUtils.*;
 /**
  * Job for execute SCAN or SQL query and get first page of results.
  */
-public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? extends Exception, VisorQueryResultEx>> {
+public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? extends VisorExceptionWrapper, VisorQueryResultEx>> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -61,7 +62,7 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteBiTuple<? extends Exception, VisorQueryResultEx> run(VisorQueryArg arg) {
+    @Override protected IgniteBiTuple<? extends VisorExceptionWrapper, VisorQueryResultEx> run(VisorQueryArg arg) {
         try {
             UUID nid = ignite.localNode().id();
 
@@ -110,8 +111,8 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
                 Collection<GridQueryFieldMetadata> meta = cur.fieldsMeta();
 
                 if (meta == null)
-                    return new IgniteBiTuple<Exception, VisorQueryResultEx>(
-                        new SQLException("Fail to execute query. No metadata available."), null);
+                    return new IgniteBiTuple<>(
+                        VisorTaskUtils.wrap(new SQLException("Fail to execute query. No metadata available.")), null);
                 else {
                     List<VisorQueryField> names = new ArrayList<>(meta.size());
 
@@ -138,7 +139,7 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
             }
         }
         catch (Exception e) {
-            return new IgniteBiTuple<>(e, null);
+            return new IgniteBiTuple<>(VisorTaskUtils.wrap(e), null);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/662f7337/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
index 4f2fda5..98c876a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
@@ -19,13 +19,14 @@ package org.apache.ignite.internal.visor.query;
 
 import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.visor.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 
 /**
  * Task for execute SCAN or SQL query and get first page of results.
  */
 @GridInternal
-public class VisorQueryTask extends VisorOneNodeTask<VisorQueryArg, IgniteBiTuple<? extends Exception, VisorQueryResultEx>> {
+public class VisorQueryTask extends VisorOneNodeTask<VisorQueryArg, IgniteBiTuple<? extends VisorExceptionWrapper, VisorQueryResultEx>> {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/662f7337/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
new file mode 100644
index 0000000..be6f63c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
@@ -0,0 +1,78 @@
+/*
+ * 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.util;
+
+/**
+ * Wrapper of exceptions for transferring to Visor with absent exception classes.
+ */
+public class VisorExceptionWrapper extends Throwable {
+    /** Detail message string of this throwable */
+    private String detailMsg;
+
+    /** Simple class name of original throwable */
+    private String originalName;
+
+    /** Full class name of original throwable */
+    private String fullName;
+
+    /**
+     * Wrap throwable by presented on Visor throwable object.
+     *
+     * @param cause Base throwable object.
+     */
+    public VisorExceptionWrapper(Throwable cause) {
+        assert cause != null;
+
+        originalName = cause.getClass().getSimpleName();
+        fullName = cause.getClass().getName();
+
+        detailMsg = cause.getMessage();
+
+        StackTraceElement[] stackTrace = cause.getStackTrace();
+
+        if (stackTrace != null)
+            setStackTrace(stackTrace);
+
+        if (cause.getCause() != null)
+            initCause(new VisorExceptionWrapper(cause.getCause()));
+    }
+
+    /**
+     * @return Simple name of base throwable object.
+     */
+    public String getOriginalName() {
+        return originalName;
+    }
+
+    /**
+     * @return Full name of base throwable object.
+     */
+    public String getFullName() {
+        return fullName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getMessage() {
+        return detailMsg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return (detailMsg != null) ? (fullName + ": " + detailMsg) : fullName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/662f7337/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
index e8ae76d..b0afbc9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
@@ -867,4 +867,14 @@ public class VisorTaskUtils {
 
         return bos.toByteArray();
     }
+
+    /**
+     * Wrap throwable object of any type to presented on Visor throwable object.
+     *
+     * @param e Base throwable object.
+     * @return Wrapped throwable object.
+     */
+    public static VisorExceptionWrapper wrap(Throwable e) {
+        return new VisorExceptionWrapper(e);
+    }
 }


[19/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-883_1

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-883_1


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

Branch: refs/heads/ignite-633
Commit: 0a4e7dd4c8321bb163cd8568494c97a1a4aef437
Parents: 50b0b49 59db4a5
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 10:10:59 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 10:10:59 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |  11 +
 .../apache/ignite/internal/IgniteKernal.java    |   2 +
 .../ignite/internal/IgniteNodeAttributes.java   |   5 +-
 .../cache/DynamicCacheDescriptor.java           |  17 +
 .../processors/cache/GridCacheAdapter.java      |  17 +-
 .../processors/cache/GridCacheContext.java      |  13 +
 .../processors/cache/GridCacheProcessor.java    |  37 ++-
 .../processors/cache/GridCacheProxyImpl.java    |  14 +-
 .../processors/cache/IgniteCacheProxy.java      |  23 ++
 .../processors/cache/IgniteInternalCache.java   |  11 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   4 +
 .../transactions/IgniteTxLocalAdapter.java      |  28 ++
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../IgniteTxRollbackCheckedException.java       |   9 +
 ...acheReadOnlyTransactionalClientSelfTest.java | 327 +++++++++++++++++++
 .../cache/GridCacheAbstractFullApiSelfTest.java |  83 +++++
 .../GridCacheExAbstractFullApiSelfTest.java     | 103 ------
 .../GridCacheExColocatedFullApiSelfTest.java    |  33 --
 .../near/GridCacheExNearFullApiSelfTest.java    |  39 ---
 .../GridCacheExReplicatedFullApiSelfTest.java   |  33 --
 .../local/GridCacheExLocalFullApiSelfTest.java  |  30 --
 .../loadtests/hashmap/GridCacheTestContext.java |   1 +
 .../IgniteCacheFullApiSelfTestSuite.java        |   6 -
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 24 files changed, 591 insertions(+), 259 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a4e7dd4/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a4e7dd4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a4e7dd4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------


[33/50] incubator-ignite git commit: # ignite-992 Review.

Posted by sb...@apache.org.
# ignite-992 Review.


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

Branch: refs/heads/ignite-633
Commit: 0eee664c0e9a2a0376aa840a5aaf52d6f82a38be
Parents: ea12580
Author: Andrey <an...@gridgain.com>
Authored: Tue Jun 9 11:11:32 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Tue Jun 9 11:11:32 2015 +0700

----------------------------------------------------------------------
 .../visor/util/VisorExceptionWrapper.java       | 26 ++++++++++----------
 1 file changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0eee664c/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
index a2965d7..e253dcf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
@@ -24,11 +24,11 @@ public class VisorExceptionWrapper extends Throwable {
     /** Detail message string of this throwable */
     private String detailMsg;
 
-    /** Simple class name of original throwable */
-    private String originalName;
+    /** Simple class name of base throwable object. */
+    private String classSimpleName;
 
-    /** Full class name of original throwable */
-    private String fullName;
+    /** Class name of base throwable object. */
+    private String className;
 
     /**
      * Wrap throwable by presented on Visor throwable object.
@@ -38,8 +38,8 @@ public class VisorExceptionWrapper extends Throwable {
     public VisorExceptionWrapper(Throwable cause) {
         assert cause != null;
 
-        originalName = cause.getClass().getSimpleName();
-        fullName = cause.getClass().getName();
+        classSimpleName = cause.getClass().getSimpleName();
+        className = cause.getClass().getName();
 
         detailMsg = cause.getMessage();
 
@@ -53,17 +53,17 @@ public class VisorExceptionWrapper extends Throwable {
     }
 
     /**
-     * @return Simple name of base throwable object.
+     * @return Class simple name of base throwable object.
      */
-    public String getOriginalName() {
-        return originalName;
+    public String getClassSimpleName() {
+        return classSimpleName;
     }
 
     /**
-     * @return Full name of base throwable object.
+     * @return Class name of base throwable object.
      */
-    public String getFullName() {
-        return fullName;
+    public String getClassName() {
+        return className;
     }
 
     /** {@inheritDoc} */
@@ -73,6 +73,6 @@ public class VisorExceptionWrapper extends Throwable {
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return (detailMsg != null) ? (fullName + ": " + detailMsg) : fullName;
+        return (detailMsg != null) ? (className + ": " + detailMsg) : className;
     }
 }


[41/50] incubator-ignite git commit: #sberb-23: Get caused exception for indexed entity in offheap mode. Add test for bug

Posted by sb...@apache.org.
#sberb-23: Get caused exception for indexed entity in offheap mode.
Add test for bug


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

Branch: refs/heads/ignite-633
Commit: 638dd313eded4946adcf3dc3d309db1c59ae8ce3
Parents: 410c1d7
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 9 12:13:51 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 9 12:13:51 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheOffheapIndexGetSelfTest.java | 62 +++++++++++++++++++-
 1 file changed, 61 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/638dd313/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
index 41eb45a..4e613ae 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
@@ -19,14 +19,17 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cache.query.annotations.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
 
 import javax.cache.*;
+import java.io.*;
 import java.util.*;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
@@ -71,7 +74,7 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest {
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
         cacheCfg.setMemoryMode(OFFHEAP_TIERED);
         cacheCfg.setEvictionPolicy(null);
-        cacheCfg.setIndexedTypes(Long.class, Long.class);
+        cacheCfg.setIndexedTypes(Long.class, Long.class, String.class, TestEntity.class);
 
         cfg.setCacheConfiguration(cacheCfg);
 
@@ -120,4 +123,61 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest {
             assertNotNull(e.getValue());
         }
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutGet() throws Exception {
+        IgniteCache<Object, Object> cache = grid(0).cache(null);
+
+        Map map = new HashMap();
+
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC,
+            TransactionIsolation.REPEATABLE_READ, 100000, 1000)) {
+
+            for (int i = 4; i < 400; i++) {
+                map.put("key" + i, new TestEntity("value"));
+                map.put(i, "value");
+            }
+
+            cache.putAll(map);
+
+            tx.commit();
+        }
+
+        for (int i = 0; i < 100; i++) {
+            cache.get("key" + i);
+            cache.get(i);
+        }
+    }
+
+    /**
+     * Test entry class.
+     */
+    private static class TestEntity implements Serializable {
+        /** Value. */
+        @QuerySqlField(index = true)
+        private String val;
+
+        /**
+         * @param value Value.
+         */
+        public TestEntity(String value) {
+            this.val = value;
+        }
+
+        /**
+         * @return Value.
+         */
+        public String getValue() {
+            return val;
+        }
+
+        /**
+         * @param val Value
+         */
+        public void setValue(String val) {
+            this.val = val;
+        }
+    }
 }


[34/50] incubator-ignite git commit: Merge branches 'ignite-992' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-992

Posted by sb...@apache.org.
Merge branches 'ignite-992' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-992


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

Branch: refs/heads/ignite-633
Commit: e934bcad4235494986a0dd30352210255beb927e
Parents: 0eee664 0fa2853
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Jun 9 12:10:23 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Jun 9 12:10:23 2015 +0700

----------------------------------------------------------------------
 DEVNOTES.txt                                    |  42 +-
 .../apache/ignite/internal/IgniteKernal.java    |  26 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   8 +-
 .../internal/MarshallerContextAdapter.java      |  36 +-
 .../internal/managers/GridManagerAdapter.java   |   9 +
 .../checkpoint/GridCheckpointManager.java       |  52 +-
 .../discovery/GridDiscoveryManager.java         |  28 +-
 .../processors/cache/GridCacheAdapter.java      |   4 +
 .../GridCachePartitionExchangeManager.java      |  26 +-
 .../processors/cache/GridCacheTtlManager.java   |   9 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  18 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |  40 +-
 .../GridDhtPartitionsExchangeFuture.java        |  50 +-
 .../cache/transactions/IgniteTxManager.java     |   3 -
 .../datastructures/DataStructuresProcessor.java | 107 +++-
 .../service/GridServiceProcessor.java           |   4 +-
 .../timeout/GridSpiTimeoutObject.java           |  73 +++
 .../timeout/GridTimeoutProcessor.java           | 105 +++-
 .../util/nio/GridCommunicationClient.java       |  30 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |  13 +-
 .../util/nio/GridTcpCommunicationClient.java    | 554 -------------------
 .../util/nio/GridTcpNioCommunicationClient.java |   8 -
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  27 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  10 +
 .../ignite/spi/IgniteSpiTimeoutObject.java      |  44 ++
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |   3 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 438 ++++-----------
 .../tcp/TcpCommunicationSpiMBean.java           |   2 -
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   3 -
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  10 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 156 +-----
 .../IgniteCountDownLatchAbstractSelfTest.java   | 102 ++++
 .../IgniteCacheClientNearCacheExpiryTest.java   | 103 ++++
 .../IgniteCacheExpiryPolicyTestSuite.java       |   2 +
 .../DataStreamerMultinodeCreateCacheTest.java   |  97 ++++
 .../internal/util/nio/GridNioSelfTest.java      |   2 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |   4 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   2 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |   2 -
 ...cpCommunicationSpiMultithreadedSelfTest.java |   2 +-
 .../discovery/AbstractDiscoverySelfTest.java    |  13 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  25 +
 .../testframework/GridSpiTestContext.java       |  10 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 +
 44 files changed, 1073 insertions(+), 1230 deletions(-)
----------------------------------------------------------------------



[37/50] incubator-ignite git commit: # ignite-883

Posted by sb...@apache.org.
# ignite-883


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

Branch: refs/heads/ignite-633
Commit: eb0e2db5718d36693952bad897fcd31bae74d37d
Parents: 14bb076
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 9 10:02:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 9 10:11:58 2015 +0300

----------------------------------------------------------------------
 .../processors/affinity/GridAffinityAssignment.java   | 12 ++++++++++++
 .../affinity/GridAffinityAssignmentCache.java         |  4 ++--
 .../dht/atomic/GridNearAtomicUpdateFuture.java        |  6 +++++-
 .../preloader/GridDhtPartitionsExchangeFuture.java    | 14 +++++++++-----
 4 files changed, 28 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb0e2db5/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
index e9df8b8..5373e46 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
@@ -68,6 +68,18 @@ class GridAffinityAssignment implements Serializable {
     }
 
     /**
+     * @param topVer Topology version.
+     * @param aff Assignment to copy from.
+     */
+    GridAffinityAssignment(AffinityTopologyVersion topVer, GridAffinityAssignment aff) {
+        this.topVer = topVer;
+
+        assignment = aff.assignment;
+        primary = aff.primary;
+        backup = aff.backup;
+    }
+
+    /**
      * @return Affinity assignment.
      */
     public List<List<ClusterNode>> assignment() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb0e2db5/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 0969a57..c46490e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -235,7 +235,7 @@ public class GridAffinityAssignmentCache {
         assert evt.type() == EVT_DISCOVERY_CUSTOM_EVT  || aff.primaryPartitions(evt.eventNode().id()).isEmpty() : evt;
         assert evt.type() == EVT_DISCOVERY_CUSTOM_EVT  || aff.backupPartitions(evt.eventNode().id()).isEmpty() : evt;
 
-        GridAffinityAssignment assignmentCpy = new GridAffinityAssignment(topVer, aff.assignment());
+        GridAffinityAssignment assignmentCpy = new GridAffinityAssignment(topVer, aff);
 
         affCache.put(topVer, assignmentCpy);
         head.set(assignmentCpy);
@@ -244,7 +244,7 @@ public class GridAffinityAssignmentCache {
             if (entry.getKey().compareTo(topVer) <= 0) {
                 if (log.isDebugEnabled())
                     log.debug("Completing topology ready future (use previous affinity) " +
-                            "[locNodeId=" + ctx.localNodeId() + ", futVer=" + entry.getKey() + ", topVer=" + topVer + ']');
+                        "[locNodeId=" + ctx.localNodeId() + ", futVer=" + entry.getKey() + ", topVer=" + topVer + ']');
 
                 entry.getValue().onDone(topVer);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb0e2db5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 55cc027..07f5ecf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -463,7 +463,11 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                 if (waitTopFut) {
                     fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                            mapOnTopology(keys, remap, oldNodeId, waitTopFut);
+                            cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                                @Override public void run() {
+                                    mapOnTopology(keys, remap, oldNodeId, waitTopFut);
+                                }
+                            });
                         }
                     });
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb0e2db5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 05f5eaf..9f18c98 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -976,14 +976,18 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
     /** {@inheritDoc} */
     @Override public boolean onDone(AffinityTopologyVersion res, Throwable err) {
-        Map<Integer, Boolean> m = new HashMap<>();
+        Map<Integer, Boolean> m = null;
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-            if (cacheCtx.config().getTopologyValidator() != null && !CU.isSystemCache(cacheCtx.name()))
+            if (cacheCtx.config().getTopologyValidator() != null && !CU.isSystemCache(cacheCtx.name())) {
+                if (m == null)
+                    m = new HashMap<>();
+
                 m.put(cacheCtx.cacheId(), cacheCtx.config().getTopologyValidator().validate(discoEvt.topologyNodes()));
+            }
         }
 
-        cacheValidRes = m;
+        cacheValidRes = m != null ? m : Collections.<Integer, Boolean>emptyMap();
 
         cctx.cache().onExchangeDone(exchId.topologyVersion(), reqs, err);
 
@@ -1001,8 +1005,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             if (timeoutObj != null)
                 cctx.kernalContext().timeout().removeTimeoutObject(timeoutObj);
 
-            for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-                if (exchId.event() == EventType.EVT_NODE_FAILED || exchId.event() == EventType.EVT_NODE_LEFT)
+            if (exchId.isLeft()) {
+                for (GridCacheContext cacheCtx : cctx.cacheContexts())
                     cacheCtx.config().getAffinity().removeNode(exchId.nodeId());
             }
 


[49/50] incubator-ignite git commit: Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-6

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-6


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

Branch: refs/heads/ignite-633
Commit: 8018af48d7510fc39124208d152aa26d441f2708
Parents: da27efd e0426f0
Author: iveselovskiy <iv...@gridgain.com>
Authored: Wed Jun 10 12:43:34 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Wed Jun 10 12:43:34 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   42 +-
 assembly/dependencies-visor-console.xml         |    3 +
 .../hibernate/CacheHibernatePersonStore.java    |  202 +-
 .../hibernate/CacheHibernateStoreExample.java   |   17 +
 .../store/jdbc/CacheJdbcPersonStore.java        |  180 +-
 .../store/jdbc/CacheJdbcStoreExample.java       |   13 +
 .../store/spring/CacheSpringPersonStore.java    |  128 +
 .../store/spring/CacheSpringStoreExample.java   |  143 +
 .../datagrid/store/spring/package-info.java     |   22 +
 .../java/org/apache/ignite/IgniteCache.java     |   16 +
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |  187 +-
 .../ignite/cache/eviction/EvictableEntry.java   |    7 +
 .../ignite/cache/eviction/EvictionPolicy.java   |    2 +
 .../cache/eviction/fifo/FifoEvictionPolicy.java |  117 +-
 .../eviction/fifo/FifoEvictionPolicyMBean.java  |   22 +
 .../cache/eviction/lru/LruEvictionPolicy.java   |  135 +-
 .../eviction/lru/LruEvictionPolicyMBean.java    |   38 +
 .../eviction/random/RandomEvictionPolicy.java   |   10 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |  141 +-
 .../sorted/SortedEvictionPolicyMBean.java       |   22 +
 .../apache/ignite/cache/query/QueryMetrics.java |    6 +-
 .../apache/ignite/cache/store/CacheStore.java   |    2 +
 .../ignite/cache/store/CacheStoreSession.java   |   22 +
 .../cache/store/CacheStoreSessionListener.java  |  133 +
 .../jdbc/CacheJdbcStoreSessionListener.java     |  141 +
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   66 +-
 .../configuration/IgniteConfiguration.java      |   48 +-
 .../org/apache/ignite/igfs/IgfsUserContext.java |  119 +
 .../igfs/secondary/IgfsSecondaryFileSystem.java |    7 +
 .../ignite/internal/ClusterMetricsSnapshot.java |   14 +
 .../internal/GridEventConsumeHandler.java       |  100 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../apache/ignite/internal/IgniteKernal.java    |   56 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    5 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   30 +-
 .../internal/MarshallerContextAdapter.java      |   36 +-
 .../ignite/internal/MarshallerContextImpl.java  |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |   18 +-
 .../internal/igfs/common/IgfsMarshaller.java    |   35 +-
 .../igfs/common/IgfsPathControlRequest.java     |   22 +
 .../internal/managers/GridManagerAdapter.java   |   68 +-
 .../checkpoint/GridCheckpointManager.java       |   52 +-
 .../managers/communication/GridIoManager.java   |  129 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   54 +
 .../discovery/GridDiscoveryManager.java         |  278 +-
 .../managers/indexing/GridIndexingManager.java  |    4 -
 .../affinity/GridAffinityAssignment.java        |   12 +
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../affinity/GridAffinityProcessor.java         |   23 +-
 .../cache/CacheEvictableEntryImpl.java          |   31 +
 .../processors/cache/CacheMetricsImpl.java      |  367 +-
 .../cache/CacheMetricsMXBeanImpl.java           |  100 +
 .../processors/cache/CacheMetricsSnapshot.java  |  380 +-
 .../cache/DynamicCacheChangeBatch.java          |   29 +-
 .../cache/DynamicCacheDescriptor.java           |   19 +
 .../processors/cache/GridCacheAdapter.java      |   50 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |   22 +-
 .../processors/cache/GridCacheEntryEx.java      |    6 +
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMapEntry.java     |   69 +-
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   97 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |  121 +-
 .../processors/cache/GridCacheProxyImpl.java    |   26 +-
 .../cache/GridCacheSharedContext.java           |   48 +-
 .../processors/cache/GridCacheSwapManager.java  |  118 +-
 .../processors/cache/GridCacheTtlManager.java   |    9 +-
 .../processors/cache/GridCacheUtils.java        |  288 +-
 .../processors/cache/IgniteCacheProxy.java      |   23 +
 .../processors/cache/IgniteInternalCache.java   |   16 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedCacheEntry.java  |    7 -
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../dht/GridClientPartitionTopology.java        |   10 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtCacheEntry.java      |    6 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |   11 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   38 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   53 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   84 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   44 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  442 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   39 +-
 .../preloader/GridDhtPreloaderAssignments.java  |    3 +-
 .../distributed/near/GridNearAtomicCache.java   |    5 +
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 +-
 .../distributed/near/GridNearLockFuture.java    |  271 +-
 .../distributed/near/GridNearLockRequest.java   |   68 +-
 .../distributed/near/GridNearLockResponse.java  |   48 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   83 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    5 +-
 .../near/GridNearTransactionalCache.java        |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |   43 +-
 .../near/GridNearTxPrepareRequest.java          |   72 +-
 .../near/GridNearTxPrepareResponse.java         |   70 +-
 .../processors/cache/local/GridLocalCache.java  |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |   31 +-
 .../cache/query/GridCacheQueryAdapter.java      |   24 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/store/CacheOsStoreManager.java        |    1 -
 .../cache/store/CacheStoreManager.java          |    7 +-
 .../store/GridCacheStoreManagerAdapter.java     |  202 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   48 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../transactions/IgniteTxLocalAdapter.java      |  170 +-
 .../cache/transactions/IgniteTxManager.java     |    3 -
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   63 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  838 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../datastreamer/DataStreamerImpl.java          |   94 +-
 .../datastructures/DataStructuresProcessor.java |  107 +-
 .../internal/processors/hadoop/HadoopJob.java   |    2 +-
 .../processors/hadoop/HadoopTaskContext.java    |   14 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |    8 +-
 .../internal/processors/igfs/IgfsImpl.java      |    8 +-
 .../processors/igfs/IgfsIpcHandler.java         |  184 +-
 .../processors/igfs/IgfsMetaManager.java        |    2 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |    9 +-
 .../internal/processors/igfs/IgfsServer.java    |    4 +-
 .../internal/processors/igfs/IgfsUtils.java     |   16 +
 .../processors/query/GridQueryProcessor.java    |  305 +-
 .../service/GridServiceProcessor.java           |  125 +-
 .../timeout/GridSpiTimeoutObject.java           |   73 +
 .../timeout/GridTimeoutProcessor.java           |  105 +-
 .../IgniteTxRollbackCheckedException.java       |    9 +
 .../ignite/internal/util/GridJavaProcess.java   |   30 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../util/nio/GridCommunicationClient.java       |   30 +-
 .../util/nio/GridNioDelimitedBuffer.java        |    2 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |   13 +-
 .../util/nio/GridTcpCommunicationClient.java    |  554 --
 .../util/nio/GridTcpNioCommunicationClient.java |    8 -
 .../ignite/internal/visor/cache/VisorCache.java |    2 +-
 .../VisorCacheConfigurationCollectorJob.java    |    6 +-
 .../internal/visor/cache/VisorCacheMetrics.java |   19 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |   10 +-
 .../cache/VisorCacheStoreConfiguration.java     |    5 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |    9 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |   17 +-
 .../node/VisorNodeSuppressedErrorsTask.java     |   12 +-
 .../internal/visor/query/VisorQueryJob.java     |   13 +-
 .../internal/visor/query/VisorQueryTask.java    |    3 +-
 .../visor/util/VisorExceptionWrapper.java       |   81 +
 .../internal/visor/util/VisorTaskUtils.java     |   10 +
 .../ignite/mxbean/CacheMetricsMXBean.java       |   80 +
 .../apache/ignite/plugin/PluginProvider.java    |   26 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   70 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   57 +-
 .../ignite/spi/IgniteSpiTimeoutObject.java      |   44 +
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |    3 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  438 +-
 .../tcp/TcpCommunicationSpiMBean.java           |    2 -
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../discovery/DiscoverySpiCustomMessage.java    |   40 +
 .../spi/discovery/DiscoverySpiListener.java     |    5 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 1478 +++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 4733 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  212 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5771 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../tcp/ipfinder/TcpDiscoveryIpFinder.java      |   10 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   57 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   24 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   67 +
 .../messages/TcpDiscoveryClientPingRequest.java |   56 +
 .../TcpDiscoveryClientPingResponse.java         |   67 +
 .../TcpDiscoveryCustomEventMessage.java         |   41 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   28 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   43 +
 .../messages/TcpDiscoveryNodeAddedMessage.java  |    2 +-
 .../tcp/messages/TcpDiscoveryPingRequest.java   |    6 +
 .../tcp/messages/TcpDiscoveryPingResponse.java  |   15 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    8 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 ...cheStoreSessionListenerAbstractSelfTest.java |  315 +
 ...heStoreSessionListenerLifecycleSelfTest.java |  395 ++
 .../CacheJdbcStoreSessionListenerSelfTest.java  |  175 +
 .../ignite/internal/GridAffinitySelfTest.java   |    1 +
 .../internal/GridDiscoveryEventSelfTest.java    |    7 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    7 +-
 .../internal/GridProjectionAbstractTest.java    |   16 +
 .../GridProjectionForCachesSelfTest.java        |   11 +-
 .../internal/GridReleaseTypeSelfTest.java       |   77 +-
 .../apache/ignite/internal/GridSelfTest.java    |    4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   62 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |  122 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   46 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    1 +
 ...acheReadOnlyTransactionalClientSelfTest.java |  327 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  113 +
 .../cache/GridCacheAbstractMetricsSelfTest.java |   48 +-
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 .../GridCacheConcurrentTxMultiNodeTest.java     |    8 +-
 ...idCacheConfigurationConsistencySelfTest.java |   14 +-
 .../GridCacheExAbstractFullApiSelfTest.java     |  103 -
 .../cache/GridCacheMemoryModeSelfTest.java      |   23 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../processors/cache/GridCacheOffHeapTest.java  |    5 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../cache/GridCacheReloadSelfTest.java          |    6 +-
 .../GridCacheReturnValueTransferSelfTest.java   |    3 +
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  190 +
 .../processors/cache/GridCacheTestEntryEx.java  |    4 +
 .../GridCacheVariableTopologySelfTest.java      |   12 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |    2 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    3 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   29 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |   13 +-
 .../IgniteCachePartitionMapUpdateTest.java      |  226 +
 .../cache/IgniteCachePeekModesAbstractTest.java |    5 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   62 +
 ...niteDynamicCacheWithConfigStartSelfTest.java |   97 +
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../IgniteCountDownLatchAbstractSelfTest.java   |  102 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../IgniteCrossCacheTxStoreSelfTest.java        |  147 +-
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   11 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |   11 +-
 .../GridCacheExColocatedFullApiSelfTest.java    |   33 -
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   53 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 .../near/GridCacheExNearFullApiSelfTest.java    |   39 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 .../GridCachePartitionedEvictionSelfTest.java   |   11 +-
 .../GridCachePartitionedFullApiSelfTest.java    |   32 +
 ...ePartitionedMultiThreadedPutGetSelfTest.java |    6 +-
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |    2 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheExReplicatedFullApiSelfTest.java   |   33 -
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 .../cache/eviction/EvictionAbstractTest.java    | 1056 ++++
 .../GridCacheBatchEvictUnswapSelfTest.java      |    5 +-
 ...heConcurrentEvictionConsistencySelfTest.java |   82 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |   29 +-
 .../GridCacheDistributedEvictionsSelfTest.java  |    5 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |   11 +-
 .../eviction/GridCacheEvictionAbstractTest.java |  484 --
 .../GridCacheEvictionTouchSelfTest.java         |   22 +-
 .../cache/eviction/GridCacheMockEntry.java      |    5 +
 .../fifo/FifoEvictionPolicySelfTest.java        |  262 +
 ...ridCacheFifoBatchEvictionPolicySelfTest.java |  384 --
 .../GridCacheFifoEvictionPolicySelfTest.java    |  372 --
 .../lru/GridCacheLruEvictionPolicySelfTest.java |  417 --
 .../GridCacheLruNearEvictionPolicySelfTest.java |  136 -
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |  171 -
 .../eviction/lru/LruEvictionPolicySelfTest.java |  353 ++
 .../lru/LruNearEvictionPolicySelfTest.java      |  140 +
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |  172 +
 .../GridCacheRandomEvictionPolicySelfTest.java  |  258 -
 .../RandomEvictionPolicyCacheSizeSelfTest.java  |    6 +
 .../random/RandomEvictionPolicySelfTest.java    |  357 ++
 ...dCacheSortedBatchEvictionPolicySelfTest.java |  385 --
 ...acheSortedEvictionPolicyPerformanceTest.java |  135 -
 .../GridCacheSortedEvictionPolicySelfTest.java  |  373 --
 .../SortedEvictionPolicyPerformanceTest.java    |  134 +
 .../sorted/SortedEvictionPolicySelfTest.java    |  266 +
 .../IgniteCacheClientNearCacheExpiryTest.java   |  103 +
 .../IgniteCacheExpiryPolicyTestSuite.java       |    2 +
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java |  412 ++
 .../local/GridCacheExLocalFullApiSelfTest.java  |   30 -
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   96 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../DataStreamerMultiThreadedSelfTest.java      |   59 +-
 .../DataStreamerMultinodeCreateCacheTest.java   |   97 +
 .../igfs/IgfsClientCacheSelfTest.java           |   12 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |   16 +-
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../nio/GridNioDelimitedBufferSelfTest.java     |  112 +
 .../util/nio/GridNioDelimitedBufferTest.java    |  112 -
 .../internal/util/nio/GridNioSelfTest.java      |    2 +-
 .../loadtests/GridCacheMultiNodeLoadTest.java   |    5 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |    6 +-
 .../loadtests/hashmap/GridCacheTestContext.java |    4 +-
 .../swap/GridSwapEvictAllBenchmark.java         |    6 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../IgniteMessagingWithClientTest.java          |  166 +
 .../GridTcpCommunicationSpiAbstractTest.java    |    4 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |    2 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    2 -
 ...cpCommunicationSpiMultithreadedSelfTest.java |    2 +-
 .../discovery/AbstractDiscoverySelfTest.java    |   21 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiMulticastTest.java |  129 +
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1196 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../stream/socket/SocketStreamerSelfTest.java   |   29 +-
 .../testframework/GridSpiTestContext.java       |   35 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/cache/TestCacheSession.java          |   18 +
 .../cache/TestThreadLocalCacheSession.java      |   15 +
 .../junits/common/GridCommonAbstractTest.java   |   83 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    4 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheEvictionSelfTestSuite.java       |   14 +-
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |    6 -
 .../IgniteCacheMetricsSelfTestSuite.java        |    1 +
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |   41 -
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    5 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    8 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    4 +-
 .../testsuites/IgniteStreamSelfTestSuite.java   |   39 +
 .../testsuites/IgniteStreamTestSuite.java       |   39 -
 .../testsuites/IgniteUtilSelfTestSuite.java     |    2 +-
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |   43 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   14 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  165 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  137 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   32 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |    2 +-
 .../internal/processors/hadoop/HadoopUtils.java |  288 +-
 .../hadoop/SecondaryFileSystemProvider.java     |   57 +-
 .../hadoop/fs/HadoopDistributedFileSystem.java  |   91 -
 .../hadoop/fs/HadoopFileSystemsUtils.java       |   17 -
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |  204 +
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |    6 +
 .../hadoop/igfs/HadoopIgfsInProc.java           |  170 +-
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |    2 +-
 .../hadoop/igfs/HadoopIgfsOutProc.java          |   33 +-
 .../hadoop/igfs/HadoopIgfsWrapper.java          |   19 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |   20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |   31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |   26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |   52 +-
 .../hadoop/HadoopClientProtocolSelfTest.java    |    6 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   56 +-
 ...oopSecondaryFileSystemConfigurationTest.java |    4 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   63 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |    2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |    2 +
 .../hadoop/HadoopAbstractSelfTest.java          |   14 +-
 .../hadoop/HadoopCommandLineTest.java           |   14 +-
 .../hadoop/HadoopFileSystemsTest.java           |   23 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |  176 +-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |    2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |   15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |    5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |    5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |    6 +-
 .../collections/HadoopAbstractMapTest.java      |   12 +
 .../collections/HadoopSkipListSelfTest.java     |    4 +-
 .../CacheHibernateStoreSessionListener.java     |  216 +
 ...heHibernateStoreSessionListenerSelfTest.java |  228 +
 .../testsuites/IgniteHibernateTestSuite.java    |    2 +
 .../GridCacheOffheapIndexEntryEvictTest.java    |  200 +
 .../cache/GridCacheOffheapIndexGetSelfTest.java |   80 +-
 .../cache/GridCacheQueryMetricsSelfTest.java    |   84 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    6 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...acheConfigurationPrimitiveTypesSelfTest.java |  104 +
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 ...QueryMultiThreadedOffHeapTieredSelfTest.java |   37 +
 ...eQueryMultiThreadedOffHeapTiredSelfTest.java |   37 -
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   11 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |    6 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    4 +-
 .../IgniteCacheWithIndexingTestSuite.java       |    3 +
 modules/mesos/pom.xml                           |    8 +-
 modules/spring/pom.xml                          |   14 +
 .../spring/CacheSpringStoreSessionListener.java |  207 +
 ...CacheSpringStoreSessionListenerSelfTest.java |  197 +
 .../testsuites/IgniteSpringTestSuite.java       |    3 +
 .../commands/cache/VisorCacheScanCommand.scala  |    2 +-
 scripts/git-apply-patch.sh                      |    8 +-
 scripts/git-format-patch.sh                     |   16 +-
 scripts/git-patch-functions.sh                  |   36 +-
 449 files changed, 30911 insertions(+), 16888 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8018af48/DEVNOTES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8018af48/modules/spring/pom.xml
----------------------------------------------------------------------


[02/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-981' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-981' into ignite-sprint-5


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

Branch: refs/heads/ignite-633
Commit: 20e567773b5b0674754a348b6d41d29370f3bd87
Parents: a03d111 a6ea325
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 4 15:06:44 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 4 15:06:44 2015 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java   | 117 +++++++------
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../igfs/IgfsClientCacheSelfTest.java           |   9 +-
 .../IgniteMessagingWithClientTest.java          | 164 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 5 files changed, 238 insertions(+), 55 deletions(-)
----------------------------------------------------------------------



[30/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5


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

Branch: refs/heads/ignite-633
Commit: ff7827e479b1fbcb79d934d8f240ae0a2ff9fc3d
Parents: f1cfd29 2a8e2ab
Author: avinogradov <av...@gridgain.com>
Authored: Mon Jun 8 15:18:21 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Jun 8 15:18:21 2015 +0300

----------------------------------------------------------------------
 .../DataStreamerMultinodeCreateCacheTest.java   | 97 ++++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |  1 +
 2 files changed, 98 insertions(+)
----------------------------------------------------------------------



[03/50] incubator-ignite git commit: [IGNITE-218]: Wrong staging permissions while running MR job under hadoop accelerator

Posted by sb...@apache.org.
[IGNITE-218]: Wrong staging permissions while running MR job under hadoop accelerator


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

Branch: refs/heads/ignite-633
Commit: c9f72917843092d596044197cf7cb05c56a13fca
Parents: 20e5677
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu Jun 4 18:20:24 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Thu Jun 4 18:20:24 2015 +0300

----------------------------------------------------------------------
 .../processors/hadoop/HadoopTaskContext.java    |  14 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   2 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |  14 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  70 ++---
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   2 +-
 .../internal/processors/hadoop/HadoopUtils.java | 282 ++++++++++++++++++-
 .../hadoop/SecondaryFileSystemProvider.java     |   4 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  48 +++-
 .../hadoop/HadoopClientProtocolSelfTest.java    |   6 +-
 .../hadoop/HadoopAbstractSelfTest.java          |  14 +-
 .../hadoop/HadoopCommandLineTest.java           |  14 +-
 .../processors/hadoop/HadoopMapReduceTest.java  | 176 +++++++++++-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |   2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |  15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |  12 +
 22 files changed, 643 insertions(+), 127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
index 371fd81..3d2ee17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
@@ -21,13 +21,14 @@ import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 
 import java.util.*;
+import java.util.concurrent.*;
 
 /**
  * Task context.
  */
 public abstract class HadoopTaskContext {
     /** */
-    private final HadoopJob job;
+    protected final HadoopJob job;
 
     /** */
     private HadoopTaskInput input;
@@ -187,4 +188,15 @@ public abstract class HadoopTaskContext {
      * @throws IgniteCheckedException If failed.
      */
     public abstract void cleanupTaskEnvironment() throws IgniteCheckedException;
+
+    /**
+     * Executes a callable on behalf of the job owner.
+     * In case of embedded task execution the implementation of this method
+     * will use classes loaded by the ClassLoader this HadoopTaskContext loaded with.
+     * @param c The callable.
+     * @param <T> The return type of the Callable.
+     * @return The result of the callable.
+     * @throws IgniteCheckedException On any error in callable.
+     */
+    public abstract <T> T runAsJobOwner(Callable<T> c) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
index b8095b8..44ee90f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
@@ -121,6 +121,6 @@ class IgfsSecondaryFileSystemImpl implements IgfsSecondaryFileSystem {
 
     /** {@inheritDoc} */
     @Override public void close() throws IgniteException {
-        igfs.stop(true);
+        // No-op.
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
index 66e9761..d910507 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
@@ -20,10 +20,12 @@ package org.apache.ignite.hadoop.fs;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.*;
+import org.apache.hadoop.mapreduce.*;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
+import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
 
 import java.io.*;
@@ -37,9 +39,6 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
     public static final String PERFORMANCE_COUNTER_FILE_NAME = "performance";
 
     /** */
-    private static final String DEFAULT_USER_NAME = "anonymous";
-
-    /** */
     public static final String COUNTER_WRITER_DIR_PROPERTY = "ignite.counters.fswriter.directory";
 
     /** */
@@ -52,15 +51,14 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
     @Override public void write(HadoopJobInfo jobInfo, HadoopJobId jobId, HadoopCounters cntrs)
         throws IgniteCheckedException {
 
-        Configuration hadoopCfg = new Configuration();
+        Configuration hadoopCfg = HadoopUtils.safeCreateConfiguration();
 
         for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)jobInfo).properties().entrySet())
             hadoopCfg.set(e.getKey(), e.getValue());
 
         String user = jobInfo.user();
 
-        if (F.isEmpty(user))
-            user = DEFAULT_USER_NAME;
+        user = IgfsUtils.fixUserName(user);
 
         String dir = jobInfo.property(COUNTER_WRITER_DIR_PROPERTY);
 
@@ -72,7 +70,9 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
         HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(cntrs, null);
 
         try {
-            FileSystem fs = jobStatPath.getFileSystem(hadoopCfg);
+            hadoopCfg.set(MRJobConfig.USER_NAME, user);
+
+            FileSystem fs = HadoopUtils.fileSystemForMrUser(jobStatPath.toUri(), hadoopCfg, true);
 
             fs.mkdirs(jobStatPath);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
index c0a9ade..9d94e5b 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.hdfs.*;
-import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.security.*;
 import org.apache.hadoop.util.*;
 import org.apache.ignite.*;
@@ -144,9 +143,6 @@ public class IgniteHadoopFileSystem extends FileSystem {
     /** Custom-provided sequential reads before prefetch. */
     private int seqReadsBeforePrefetch;
 
-    /** The cache was disabled when the instance was creating. */
-    private boolean cacheEnabled;
-
     /** {@inheritDoc} */
     @Override public URI getUri() {
         if (uri == null)
@@ -173,27 +169,13 @@ public class IgniteHadoopFileSystem extends FileSystem {
     }
 
     /**
-     * Gets non-null and interned user name as per the Hadoop file system viewpoint.
+     * Gets non-null user name as per the Hadoop file system viewpoint.
      * @return the user name, never null.
      */
-    public static String getFsHadoopUser(Configuration cfg) throws IOException {
-        String user = null;
-
-        // -------------------------------------------
-        // TODO: Temporary workaround, see https://issues.apache.org/jira/browse/IGNITE-761
-        // We have an issue there: sometimes FileSystem created from MR jobs gets incorrect
-        // UserGroupInformation.getCurrentUser() despite of the fact that it is invoked in correct
-        // ugi.doAs() closure.
-        if (cfg != null)
-            user = cfg.get(MRJobConfig.USER_NAME);
-        // -------------------------------------------
-
-        if (user == null) {
-            UserGroupInformation currUgi = UserGroupInformation.getCurrentUser();
-
-            if (currUgi != null)
-                user = currUgi.getShortUserName();
-        }
+    public static String getFsHadoopUser() throws IOException {
+        UserGroupInformation currUgi = UserGroupInformation.getCurrentUser();
+
+        String user = currUgi.getShortUserName();
 
         user = IgfsUtils.fixUserName(user);
 
@@ -228,10 +210,6 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             setConf(cfg);
 
-            String disableCacheName = String.format("fs.%s.impl.disable.cache", name.getScheme());
-
-            cacheEnabled = !cfg.getBoolean(disableCacheName, false);
-
             mgmt = cfg.getBoolean(IGFS_MANAGEMENT, false);
 
             if (!IGFS_SCHEME.equals(name.getScheme()))
@@ -242,7 +220,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             uriAuthority = uri.getAuthority();
 
-            user = getFsHadoopUser(cfg);
+            user = getFsHadoopUser();
 
             // Override sequential reads before prefetch if needed.
             seqReadsBeforePrefetch = parameter(cfg, PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH, uriAuthority, 0);
@@ -360,15 +338,13 @@ public class IgniteHadoopFileSystem extends FileSystem {
     @Override protected void finalize() throws Throwable {
         super.finalize();
 
-        close0();
+        close();
     }
 
     /** {@inheritDoc} */
     @Override public void close() throws IOException {
-        if (cacheEnabled && get(getUri(), getConf()) == this)
-            return;
-
-        close0();
+        if (closeGuard.compareAndSet(false, true))
+            close0();
     }
 
     /**
@@ -377,27 +353,25 @@ public class IgniteHadoopFileSystem extends FileSystem {
      * @throws IOException If failed.
      */
     private void close0() throws IOException {
-        if (closeGuard.compareAndSet(false, true)) {
-            if (LOG.isDebugEnabled())
-                LOG.debug("File system closed [uri=" + uri + ", endpoint=" + uriAuthority + ']');
+        if (LOG.isDebugEnabled())
+            LOG.debug("File system closed [uri=" + uri + ", endpoint=" + uriAuthority + ']');
 
-            if (rmtClient == null)
-                return;
+        if (rmtClient == null)
+            return;
 
-            super.close();
+        super.close();
 
-            rmtClient.close(false);
+        rmtClient.close(false);
 
-            if (clientLog.isLogEnabled())
-                clientLog.close();
+        if (clientLog.isLogEnabled())
+            clientLog.close();
 
-            if (secondaryFs != null)
-                U.closeQuiet(secondaryFs);
+        if (secondaryFs != null)
+            U.closeQuiet(secondaryFs);
 
-            // Reset initialized resources.
-            uri = null;
-            rmtClient = null;
-        }
+        // Reset initialized resources.
+        uri = null;
+        rmtClient = null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
index f3fbe9c..8330143 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
@@ -144,7 +144,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
         uri = name;
 
-        user = getFsHadoopUser(cfg);
+        user = getFsHadoopUser();
 
         try {
             initialize(name, cfg);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
index d0a327e..2e855d0 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
@@ -89,7 +89,7 @@ public class HadoopDefaultJobInfo implements HadoopJobInfo, Externalizable {
             if (jobCls0 == null) { // It is enough to have only one class loader with only Hadoop classes.
                 synchronized (HadoopDefaultJobInfo.class) {
                     if ((jobCls0 = jobCls) == null) {
-                        HadoopClassLoader ldr = new HadoopClassLoader(null, "hadoop-main");
+                        HadoopClassLoader ldr = new HadoopClassLoader(null, "hadoop-job");
 
                         jobCls = jobCls0 = ldr.loadClass(HadoopV2Job.class.getName());
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
index d493bd4..68a9ef6 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
@@ -26,10 +26,16 @@ import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.mapreduce.*;
 import org.apache.ignite.*;
+import org.apache.ignite.hadoop.fs.v1.*;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
 import org.apache.ignite.internal.processors.hadoop.v2.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
 
 import java.io.*;
+import java.net.*;
 import java.util.*;
 
 /**
@@ -57,6 +63,41 @@ public class HadoopUtils {
     /** Old reducer class attribute. */
     private static final String OLD_REDUCE_CLASS_ATTR = "mapred.reducer.class";
 
+    /** Lazy per-user cache for the file systems. It is cleared and nulled in #close() method. */
+    private static final HadoopLazyConcurrentMap<FsCacheKey, FileSystem> fileSysLazyMap = new HadoopLazyConcurrentMap<>(
+        new HadoopLazyConcurrentMap.ValueFactory<FsCacheKey, FileSystem>() {
+            @Override public FileSystem createValue(FsCacheKey key) {
+                try {
+                    assert key != null;
+
+                    // Explicitly disable FileSystem caching:
+                    URI uri = key.uri();
+
+                    String scheme = uri.getScheme();
+
+                    // Copy the configuration to avoid altering the external object.
+                    Configuration cfg = new Configuration(key.configuration());
+
+                    String prop = HadoopUtils.disableFsCachePropertyName(scheme);
+
+                    cfg.setBoolean(prop, true);
+
+                    return FileSystem.get(uri, cfg, key.user());
+                }
+                catch (IOException | InterruptedException ioe) {
+                    throw new IgniteException(ioe);
+                }
+            }
+        }
+    );
+
+    /**
+     * Constructor.
+     */
+    private HadoopUtils() {
+        // No-op.
+    }
+
     /**
      * Wraps native split.
      *
@@ -126,8 +167,6 @@ public class HadoopUtils {
                 break;
 
             case PHASE_REDUCE:
-                // TODO: temporary fixed, but why PHASE_REDUCE could have 0 reducers?
-                // See https://issues.apache.org/jira/browse/IGNITE-764
                 setupProgress = 1;
                 mapProgress = 1;
 
@@ -304,9 +343,242 @@ public class HadoopUtils {
     }
 
     /**
-     * Constructor.
+     * Creates {@link Configuration} in a correct class loader context to avoid caching
+     * of inappropriate class loader in the Configuration object.
+     * @return New instance of {@link Configuration}.
      */
-    private HadoopUtils() {
-        // No-op.
+    public static Configuration safeCreateConfiguration() {
+        final ClassLoader cl0 = Thread.currentThread().getContextClassLoader();
+
+        Thread.currentThread().setContextClassLoader(Configuration.class.getClassLoader());
+
+        try {
+            return new Configuration();
+        }
+        finally {
+            Thread.currentThread().setContextClassLoader(cl0);
+        }
+    }
+
+    /**
+     * Creates {@link JobConf} in a correct class loader context to avoid caching
+     * of inappropriate class loader in the Configuration object.
+     * @return New instance of {@link JobConf}.
+     */
+    public static JobConf safeCreateJobConf() {
+        final ClassLoader cl0 = Thread.currentThread().getContextClassLoader();
+
+        Thread.currentThread().setContextClassLoader(JobConf.class.getClassLoader());
+
+        try {
+            return new JobConf();
+        }
+        finally {
+            Thread.currentThread().setContextClassLoader(cl0);
+        }
+    }
+
+    /**
+     * Gets non-null user name as per the Hadoop viewpoint.
+     * @param cfg the Hadoop job configuration, may be null.
+     * @return the user name, never null.
+     */
+    private static String getMrHadoopUser(Configuration cfg) throws IOException {
+        String user = cfg.get(MRJobConfig.USER_NAME);
+
+        if (user == null)
+            user = IgniteHadoopFileSystem.getFsHadoopUser();
+
+        return user;
+    }
+
+    /**
+     * Common method to get the V1 file system in MapRed engine.
+     * It creates the filesystem for the user specified in the
+     * configuration with {@link MRJobConfig#USER_NAME} property.
+     * @param uri the file system uri.
+     * @param cfg the configuration.
+     * @return the file system
+     * @throws IOException
+     */
+    public static FileSystem fileSystemForMrUser(@Nullable URI uri, Configuration cfg, boolean doCacheFs) throws IOException {
+        final String usr = getMrHadoopUser(cfg);
+
+        assert usr != null;
+
+        if (uri == null)
+            uri = FileSystem.getDefaultUri(cfg);
+
+        final FileSystem fs;
+
+        if (doCacheFs) {
+            try {
+                fs = getWithCaching(uri, cfg, usr);
+            }
+            catch (IgniteException ie) {
+                throw new IOException(ie);
+            }
+        }
+        else {
+            try {
+                fs = FileSystem.get(uri, cfg, usr);
+            }
+            catch (InterruptedException ie) {
+                Thread.currentThread().interrupt();
+
+                throw new IOException(ie);
+            }
+        }
+
+        assert fs != null;
+        assert !(fs instanceof IgniteHadoopFileSystem) || F.eq(usr, ((IgniteHadoopFileSystem)fs).user());
+
+        return fs;
+    }
+
+    /**
+     * Note that configuration is not a part of the key.
+     * It is used solely to initialize the first instance
+     * that is created for the key.
+     */
+    public static final class FsCacheKey {
+        /** */
+        private final URI uri;
+
+        /** */
+        private final String usr;
+
+        /** */
+        private final String equalityKey;
+
+        /** */
+        private final Configuration cfg;
+
+        /**
+         * Constructor
+         */
+        public FsCacheKey(URI uri, String usr, Configuration cfg) {
+            assert uri != null;
+            assert usr != null;
+            assert cfg != null;
+
+            this.uri = fixUri(uri, cfg);
+            this.usr = usr;
+            this.cfg = cfg;
+
+            this.equalityKey = createEqualityKey();
+        }
+
+        /**
+         * Creates String key used for equality and hashing.
+         */
+        private String createEqualityKey() {
+            GridStringBuilder sb = new GridStringBuilder("(").a(usr).a(")@");
+
+            if (uri.getScheme() != null)
+                sb.a(uri.getScheme().toLowerCase());
+
+            sb.a("://");
+
+            if (uri.getAuthority() != null)
+                sb.a(uri.getAuthority().toLowerCase());
+
+            return sb.toString();
+        }
+
+        /**
+         * The URI.
+         */
+        public URI uri() {
+            return uri;
+        }
+
+        /**
+         * The User.
+         */
+        public String user() {
+            return usr;
+        }
+
+        /**
+         * The Configuration.
+         */
+        public Configuration configuration() {
+            return cfg;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("SimplifiableIfStatement")
+        @Override public boolean equals(Object obj) {
+            if (obj == this)
+                return true;
+
+            if (obj == null || getClass() != obj.getClass())
+                return false;
+
+            return equalityKey.equals(((FsCacheKey)obj).equalityKey);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return equalityKey.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return equalityKey;
+        }
+    }
+
+    /**
+     * Gets FileSystem caching it in static Ignite cache. The cache is a singleton
+     * for each class loader.
+     *
+     * <p/>Note that the file systems in the cache are keyed by a triplet {scheme, authority, user}.
+     * The Configuration is not a part of the key. This means that for the given key file system is
+     * initialized only once with the Configuration passed in upon the file system creation.
+     *
+     * @param uri The file system URI.
+     * @param cfg The configuration.
+     * @param usr The user to create file system for.
+     * @return The file system: either created, or taken from the cache.
+     */
+    private static FileSystem getWithCaching(URI uri, Configuration cfg, String usr) {
+        FsCacheKey key = new FsCacheKey(uri, usr, cfg);
+
+        return fileSysLazyMap.getOrCreate(key);
+    }
+
+    /**
+     * Gets the property name to disable file system cache.
+     * @param scheme The file system URI scheme.
+     * @return The property name. If scheme is null,
+     * returns "fs.null.impl.disable.cache".
+     */
+    public static String disableFsCachePropertyName(@Nullable String scheme) {
+        return String.format("fs.%s.impl.disable.cache", scheme);
+    }
+
+    /**
+     * Takes Fs URI using logic similar to that used in FileSystem#get(1,2,3).
+     * @param uri0 The uri.
+     * @param cfg The cfg.
+     * @return Correct URI.
+     */
+    public static URI fixUri(URI uri0, Configuration cfg) {
+        if (uri0 == null)
+            return FileSystem.getDefaultUri(cfg);
+
+        String scheme = uri0.getScheme();
+        String authority = uri0.getAuthority();
+
+        if (authority == null) {
+            URI dfltUri = FileSystem.getDefaultUri(cfg);
+
+            if (scheme == null || (scheme.equals(dfltUri.getScheme()) && dfltUri.getAuthority() != null))
+                return dfltUri;
+        }
+
+        return uri0;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
index b1a057c..dd679de 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
@@ -34,7 +34,7 @@ import java.security.*;
  */
 public class SecondaryFileSystemProvider {
     /** Configuration of the secondary filesystem, never null. */
-    private final Configuration cfg = new Configuration();
+    private final Configuration cfg = HadoopUtils.safeCreateConfiguration();
 
     /** The secondary filesystem URI, never null. */
     private final URI uri;
@@ -76,7 +76,7 @@ public class SecondaryFileSystemProvider {
         }
 
         // Disable caching:
-        String prop = String.format("fs.%s.impl.disable.cache", uri.getScheme());
+        String prop = HadoopUtils.disableFsCachePropertyName(uri.getScheme());
 
         cfg.setBoolean(prop, true);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
index 2e04ac1..b170125 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
@@ -99,6 +99,22 @@ public abstract class HadoopRunnableTask implements Callable<Void> {
 
     /** {@inheritDoc} */
     @Override public Void call() throws IgniteCheckedException {
+        ctx = job.getTaskContext(info);
+
+        return ctx.runAsJobOwner(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                call0();
+
+                return null;
+            }
+        });
+    }
+
+    /**
+     * Implements actual task running.
+     * @throws IgniteCheckedException
+     */
+    void call0() throws IgniteCheckedException {
         execStartTs = U.currentTimeMillis();
 
         Throwable err = null;
@@ -108,8 +124,6 @@ public abstract class HadoopRunnableTask implements Callable<Void> {
         HadoopPerformanceCounter perfCntr = null;
 
         try {
-            ctx = job.getTaskContext(info);
-
             perfCntr = HadoopPerformanceCounter.getCounter(ctx.counters(), nodeId);
 
             perfCntr.onTaskSubmit(info, submitTs);
@@ -156,8 +170,6 @@ public abstract class HadoopRunnableTask implements Callable<Void> {
             if (ctx != null)
                 ctx.cleanupTaskEnvironment();
         }
-
-        return null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
index d265ca8..d754039 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.hadoop.v2;
 
 import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.mapred.JobID;
@@ -68,7 +67,7 @@ public class HadoopV2Job implements HadoopJob {
         new ConcurrentHashMap8<>();
 
     /** Pooling task context class and thus class loading environment. */
-    private final Queue<Class<?>> taskCtxClsPool = new ConcurrentLinkedQueue<>();
+    private final Queue<Class<? extends HadoopTaskContext>> taskCtxClsPool = new ConcurrentLinkedQueue<>();
 
     /** All created contexts. */
     private final Queue<Class<?>> fullCtxClsQueue = new ConcurrentLinkedDeque<>();
@@ -93,12 +92,7 @@ public class HadoopV2Job implements HadoopJob {
 
         hadoopJobID = new JobID(jobId.globalId().toString(), jobId.localId());
 
-        HadoopClassLoader clsLdr = (HadoopClassLoader)getClass().getClassLoader();
-
-        // Before create JobConf instance we should set new context class loader.
-        Thread.currentThread().setContextClassLoader(clsLdr);
-
-        jobConf = new JobConf();
+        jobConf = HadoopUtils.safeCreateJobConf();
 
         HadoopFileSystemsUtils.setupFileSystems(jobConf);
 
@@ -139,7 +133,9 @@ public class HadoopV2Job implements HadoopJob {
 
             Path jobDir = new Path(jobDirPath);
 
-            try (FileSystem fs = FileSystem.get(jobDir.toUri(), jobConf)) {
+            try {
+                FileSystem fs = fileSystemForMrUser(jobDir.toUri(), jobConf, true);
+
                 JobSplit.TaskSplitMetaInfo[] metaInfos = SplitMetaInfoReader.readSplitMetaInfo(hadoopJobID, fs, jobConf,
                     jobDir);
 
@@ -197,7 +193,7 @@ public class HadoopV2Job implements HadoopJob {
         if (old != null)
             return old.get();
 
-        Class<?> cls = taskCtxClsPool.poll();
+        Class<? extends HadoopTaskContext> cls = taskCtxClsPool.poll();
 
         try {
             if (cls == null) {
@@ -205,9 +201,9 @@ public class HadoopV2Job implements HadoopJob {
                 // Note that the classloader identified by the task it was initially created for,
                 // but later it may be reused for other tasks.
                 HadoopClassLoader ldr = new HadoopClassLoader(rsrcMgr.classPath(),
-                    "hadoop-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber());
+                    "hadoop-task-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber());
 
-                cls = ldr.loadClass(HadoopV2TaskContext.class.getName());
+                cls = (Class<? extends HadoopTaskContext>)ldr.loadClass(HadoopV2TaskContext.class.getName());
 
                 fullCtxClsQueue.add(cls);
             }
@@ -325,7 +321,14 @@ public class HadoopV2Job implements HadoopJob {
 
     /** {@inheritDoc} */
     @Override public void cleanupStagingDirectory() {
-        if (rsrcMgr != null)
-            rsrcMgr.cleanupStagingDirectory();
+        rsrcMgr.cleanupStagingDirectory();
+    }
+
+    /**
+     * Getter for job configuration.
+     * @return The job configuration.
+     */
+    public JobConf jobConf() {
+        return jobConf;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
index 6f6bfa1..2f64e77 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
@@ -40,6 +40,9 @@ import java.util.*;
  * files are needed to be placed on local files system.
  */
 public class HadoopV2JobResourceManager {
+    /** File type Fs disable caching property name. */
+    private static final String FILE_DISABLE_CACHING_PROPERTY_NAME = HadoopUtils.disableFsCachePropertyName("file");
+
     /** Hadoop job context. */
     private final JobContextImpl ctx;
 
@@ -84,7 +87,7 @@ public class HadoopV2JobResourceManager {
         try {
             cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, dir.getAbsolutePath());
 
-            if(!cfg.getBoolean("fs.file.impl.disable.cache", false))
+            if (!cfg.getBoolean(FILE_DISABLE_CACHING_PROPERTY_NAME, false))
                 FileSystem.getLocal(cfg).setWorkingDirectory(new Path(dir.getAbsolutePath()));
         }
         finally {
@@ -112,15 +115,17 @@ public class HadoopV2JobResourceManager {
                 stagingDir = new Path(new URI(mrDir));
 
                 if (download) {
-                    FileSystem fs = FileSystem.get(stagingDir.toUri(), cfg);
+                    FileSystem fs = HadoopUtils.fileSystemForMrUser(stagingDir.toUri(), cfg, true);
 
                     if (!fs.exists(stagingDir))
-                        throw new IgniteCheckedException("Failed to find map-reduce submission directory (does not exist): " +
-                            stagingDir);
+                        throw new IgniteCheckedException("Failed to find map-reduce submission " +
+                            "directory (does not exist): " + stagingDir);
 
                     if (!FileUtil.copy(fs, stagingDir, jobLocDir, false, cfg))
-                        throw new IgniteCheckedException("Failed to copy job submission directory contents to local file system " +
-                            "[path=" + stagingDir + ", locDir=" + jobLocDir.getAbsolutePath() + ", jobId=" + jobId + ']');
+                        throw new IgniteCheckedException("Failed to copy job submission directory "
+                            + "contents to local file system "
+                            + "[path=" + stagingDir + ", locDir=" + jobLocDir.getAbsolutePath()
+                            + ", jobId=" + jobId + ']');
                 }
 
                 File jarJobFile = new File(jobLocDir, "job.jar");
@@ -144,7 +149,8 @@ public class HadoopV2JobResourceManager {
                 }
             }
             else if (!jobLocDir.mkdirs())
-                throw new IgniteCheckedException("Failed to create local job directory: " + jobLocDir.getAbsolutePath());
+                throw new IgniteCheckedException("Failed to create local job directory: "
+                    + jobLocDir.getAbsolutePath());
 
             setLocalFSWorkingDirectory(jobLocDir);
         }
@@ -204,14 +210,14 @@ public class HadoopV2JobResourceManager {
 
             FileSystem dstFs = FileSystem.getLocal(cfg);
 
-            FileSystem srcFs = srcPath.getFileSystem(cfg);
+            FileSystem srcFs = HadoopUtils.fileSystemForMrUser(srcPath.toUri(), cfg, true);
 
             if (extract) {
                 File archivesPath = new File(jobLocDir.getAbsolutePath(), ".cached-archives");
 
                 if (!archivesPath.exists() && !archivesPath.mkdir())
                     throw new IOException("Failed to create directory " +
-                         "[path=" + archivesPath + ", jobId=" + jobId + ']');
+                        "[path=" + archivesPath + ", jobId=" + jobId + ']');
 
                 File archiveFile = new File(archivesPath, locName);
 
@@ -287,7 +293,7 @@ public class HadoopV2JobResourceManager {
     public void cleanupStagingDirectory() {
         try {
             if (stagingDir != null)
-                stagingDir.getFileSystem(ctx.getJobConf()).delete(stagingDir, true);
+                HadoopUtils.fileSystemForMrUser(stagingDir.toUri(), ctx.getJobConf(), true).delete(stagingDir, true);
         }
         catch (Exception e) {
             log.error("Failed to remove job staging directory [path=" + stagingDir + ", jobId=" + jobId + ']' , e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
index dd18c66..e89feba 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
@@ -28,17 +28,21 @@ import org.apache.hadoop.mapred.TaskAttemptID;
 import org.apache.hadoop.mapred.TaskID;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.security.*;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
-import org.apache.ignite.internal.processors.hadoop.fs.*;
 import org.apache.ignite.internal.processors.hadoop.v1.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
+import java.security.*;
 import java.util.*;
+import java.util.concurrent.*;
 
 import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
 import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
@@ -419,7 +423,7 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     private Object readExternalSplit(HadoopExternalSplit split) throws IgniteCheckedException {
         Path jobDir = new Path(jobConf().get(MRJobConfig.MAPREDUCE_JOB_DIR));
 
-        try (FileSystem fs = FileSystem.get(jobDir.toUri(), jobConf());
+        try (FileSystem fs = fileSystemForMrUser(jobDir.toUri(), jobConf(), false);
             FSDataInputStream in = fs.open(JobSubmissionFiles.getJobSplitFile(jobDir))) {
 
             in.seek(split.offset());
@@ -448,4 +452,44 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
             throw new IgniteCheckedException(e);
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public <T> T runAsJobOwner(final Callable<T> c) throws IgniteCheckedException {
+        String user = job.info().user();
+
+        user = IgfsUtils.fixUserName(user);
+
+        assert user != null;
+
+        String ugiUser;
+
+        try {
+            UserGroupInformation currUser = UserGroupInformation.getCurrentUser();
+
+            assert currUser != null;
+
+            ugiUser = currUser.getShortUserName();
+        }
+        catch (IOException ioe) {
+            throw new IgniteCheckedException(ioe);
+        }
+
+        try {
+            if (F.eq(user, ugiUser))
+                // if current UGI context user is the same, do direct call:
+                return c.call();
+            else {
+                UserGroupInformation ugi = UserGroupInformation.getBestUGI(null, user);
+
+                return ugi.doAs(new PrivilegedExceptionAction<T>() {
+                    @Override public T run() throws Exception {
+                        return c.call();
+                    }
+                });
+            }
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
index b94d9d1..b9f8179 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
@@ -28,7 +28,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.hadoop.mapreduce.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.proto.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -449,7 +448,7 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
      * @return Configuration.
      */
     private Configuration config(int port) {
-        Configuration conf = new Configuration();
+        Configuration conf = HadoopUtils.safeCreateConfiguration();
 
         setupFileSystems(conf);
 
@@ -521,9 +520,8 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
             ctx.getCounter(TestCounter.COUNTER2).increment(1);
 
             int sum = 0;
-            for (IntWritable value : values) {
+            for (IntWritable value : values)
                 sum += value.get();
-            }
 
             ctx.write(key, new IntWritable(sum));
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
index af1a1e1..e8a0a6f 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
@@ -22,7 +22,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem;
 import org.apache.ignite.internal.processors.hadoop.fs.*;
-import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -62,6 +61,17 @@ public abstract class HadoopAbstractSelfTest extends GridCommonAbstractTest {
     /** Initial REST port. */
     private int restPort = REST_PORT;
 
+    /** Secondary file system REST endpoint configuration. */
+    protected static final IgfsIpcEndpointConfiguration SECONDARY_REST_CFG;
+
+    static {
+        SECONDARY_REST_CFG = new IgfsIpcEndpointConfiguration();
+
+        SECONDARY_REST_CFG.setType(IgfsIpcEndpointType.TCP);
+        SECONDARY_REST_CFG.setPort(11500);
+    }
+
+
     /** Initial classpath. */
     private static String initCp;
 
@@ -133,7 +143,7 @@ public abstract class HadoopAbstractSelfTest extends GridCommonAbstractTest {
     /**
      * @return IGFS configuration.
      */
-    public FileSystemConfiguration igfsConfiguration() {
+    public FileSystemConfiguration igfsConfiguration() throws Exception {
         FileSystemConfiguration cfg = new FileSystemConfiguration();
 
         cfg.setName(igfsName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java
index d10ee5c..c66cdf3 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java
@@ -19,12 +19,16 @@ package org.apache.ignite.internal.processors.hadoop;
 
 import com.google.common.base.*;
 import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.hadoop.fs.*;
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
+import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jsr166.*;
 
@@ -205,7 +209,15 @@ public class HadoopCommandLineTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        igfs = (IgfsEx) Ignition.start("config/hadoop/default-config.xml").fileSystem(igfsName);
+        String cfgPath = "config/hadoop/default-config.xml";
+
+        IgniteBiTuple<IgniteConfiguration, GridSpringResourceContext> tup = IgnitionEx.loadConfiguration(cfgPath);
+
+        IgniteConfiguration cfg = tup.get1();
+
+        cfg.setLocalHost("127.0.0.1"); // Avoid connecting to other nodes.
+
+        igfs = (IgfsEx) Ignition.start(cfg).fileSystem(igfsName);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
index 8a3a0ac..a1ef7ba 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
@@ -24,31 +24,104 @@ import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.hadoop.fs.*;
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
+import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.*;
+import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.util.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.igfs.IgfsMode.*;
 import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Test of whole cycle of map-reduce processing via Job tracker.
  */
 public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
+    /** IGFS block size. */
+    protected static final int IGFS_BLOCK_SIZE = 512 * 1024;
+
+    /** Amount of blocks to prefetch. */
+    protected static final int PREFETCH_BLOCKS = 1;
+
+    /** Amount of sequential block reads before prefetch is triggered. */
+    protected static final int SEQ_READS_BEFORE_PREFETCH = 2;
+
+    /** Secondary file system URI. */
+    protected static final String SECONDARY_URI = "igfs://igfs-secondary:grid-secondary@127.0.0.1:11500/";
+
+    /** Secondary file system configuration path. */
+    protected static final String SECONDARY_CFG = "modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml";
+
+    /** The user to run Hadoop job on behalf of. */
+    protected static final String USER = "vasya";
+
+    /** Secondary IGFS name. */
+    protected static final String SECONDARY_IGFS_NAME = "igfs-secondary";
+
+    /** The secondary Ignite node. */
+    protected Ignite igniteSecondary;
+
+    /** The secondary Fs. */
+    protected IgfsSecondaryFileSystem secondaryFs;
+
     /** {@inheritDoc} */
     @Override protected int gridCount() {
         return 3;
     }
 
     /**
+     * Gets owner of a IgfsEx path.
+     * @param p The path.
+     * @return The owner.
+     */
+    private static String getOwner(IgfsEx i, IgfsPath p) {
+        return i.info(p).property(IgfsEx.PROP_USER_NAME);
+    }
+
+    /**
+     * Gets owner of a secondary Fs path.
+     * @param secFs The sec Fs.
+     * @param p The path.
+     * @return The owner.
+     */
+    private static String getOwnerSecondary(final IgfsSecondaryFileSystem secFs, final IgfsPath p) {
+        return IgfsUserContext.doAs(USER, new IgniteOutClosure<String>() {
+            @Override public String apply() {
+                return secFs.info(p).property(IgfsEx.PROP_USER_NAME);
+            }
+        });
+    }
+
+    /**
+     * Checks owner of the path.
+     * @param p The path.
+     */
+    private void checkOwner(IgfsPath p) {
+        String ownerPrim = getOwner(igfs, p);
+        assertEquals(USER, ownerPrim);
+
+        String ownerSec = getOwnerSecondary(secondaryFs, p);
+        assertEquals(USER, ownerSec);
+    }
+
+    /**
      * Tests whole job execution with all phases in all combination of new and old versions of API.
      * @throws Exception If fails.
      */
@@ -71,7 +144,7 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
             JobConf jobConf = new JobConf();
 
             jobConf.set(JOB_COUNTER_WRITER_PROPERTY, IgniteHadoopFileSystemCounterWriter.class.getName());
-            jobConf.setUser("yyy");
+            jobConf.setUser(USER);
             jobConf.set(IgniteHadoopFileSystemCounterWriter.COUNTER_WRITER_DIR_PROPERTY, "/xxx/${USER}/zzz");
 
             //To split into about 40 items for v2
@@ -105,14 +178,20 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
 
             checkJobStatistics(jobId);
 
+            final String outFile = PATH_OUTPUT + "/" + (useNewReducer ? "part-r-" : "part-") + "00000";
+
+            checkOwner(new IgfsPath(PATH_OUTPUT + "/" + "_SUCCESS"));
+
+            checkOwner(new IgfsPath(outFile));
+
             assertEquals("Use new mapper: " + useNewMapper + ", new combiner: " + useNewCombiner + ", new reducer: " +
-                useNewReducer,
+                    useNewReducer,
                 "blue\t200000\n" +
-                "green\t150000\n" +
-                "red\t100000\n" +
-                "yellow\t70000\n",
-                readAndSortFile(PATH_OUTPUT + "/" + (useNewReducer ? "part-r-" : "part-") + "00000")
-            );
+                    "green\t150000\n" +
+                    "red\t100000\n" +
+                    "yellow\t70000\n",
+                readAndSortFile(outFile)
+                );
         }
     }
 
@@ -182,7 +261,7 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
             }
         }
 
-        final IgfsPath statPath = new IgfsPath("/xxx/yyy/zzz/" + jobId + "/performance");
+        final IgfsPath statPath = new IgfsPath("/xxx/" + USER + "/zzz/" + jobId + "/performance");
 
         assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
@@ -212,4 +291,85 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
                 ", actual=" + HadoopTestUtils.simpleCheckJobStatFile(reader) + ']';
         }
     }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        igniteSecondary = startGridWithIgfs("grid-secondary", SECONDARY_IGFS_NAME, PRIMARY, null, SECONDARY_REST_CFG);
+
+        super.beforeTest();
+    }
+
+    /**
+     * Start grid with IGFS.
+     *
+     * @param gridName Grid name.
+     * @param igfsName IGFS name
+     * @param mode IGFS mode.
+     * @param secondaryFs Secondary file system (optional).
+     * @param restCfg Rest configuration string (optional).
+     * @return Started grid instance.
+     * @throws Exception If failed.
+     */
+    protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
+        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg) throws Exception {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("dataCache");
+        igfsCfg.setMetaCacheName("metaCache");
+        igfsCfg.setName(igfsName);
+        igfsCfg.setBlockSize(IGFS_BLOCK_SIZE);
+        igfsCfg.setDefaultMode(mode);
+        igfsCfg.setIpcEndpointConfiguration(restCfg);
+        igfsCfg.setSecondaryFileSystem(secondaryFs);
+        igfsCfg.setPrefetchBlocks(PREFETCH_BLOCKS);
+        igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH);
+
+        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
+
+        dataCacheCfg.setName("dataCache");
+        dataCacheCfg.setCacheMode(PARTITIONED);
+        dataCacheCfg.setNearConfiguration(null);
+        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(2));
+        dataCacheCfg.setBackups(0);
+        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
+        dataCacheCfg.setOffHeapMaxMemory(0);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("metaCache");
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setGridName(gridName);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setConnectorConfiguration(null);
+
+        return G.start(cfg);
+    }
+
+    /**
+     * @return IGFS configuration.
+     */
+    @Override public FileSystemConfiguration igfsConfiguration() throws Exception {
+        FileSystemConfiguration fsCfg = super.igfsConfiguration();
+
+        secondaryFs = new IgniteHadoopIgfsSecondaryFileSystem(SECONDARY_URI, SECONDARY_CFG);
+
+        fsCfg.setSecondaryFileSystem(secondaryFs);
+
+        return fsCfg;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
index 8dc9830..eee5c8b 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
@@ -72,7 +72,7 @@ public class HadoopTaskExecutionSelfTest extends HadoopAbstractSelfTest {
 
 
     /** {@inheritDoc} */
-    @Override public FileSystemConfiguration igfsConfiguration() {
+    @Override public FileSystemConfiguration igfsConfiguration() throws Exception {
         FileSystemConfiguration cfg = super.igfsConfiguration();
 
         cfg.setFragmentizerEnabled(false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
index aaf0f92..6930020 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.io.*;
 import org.apache.ignite.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
 
 import java.io.*;
 import java.net.*;
@@ -43,7 +42,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @return Hadoop job.
      * @throws IOException If fails.
      */
-    public abstract HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception;
+    public abstract HadoopJob getHadoopJob(String inFile, String outFile) throws Exception;
 
     /**
      * @return prefix of reducer output file name. It's "part-" for v1 and "part-r-" for v2 API
@@ -79,7 +78,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
         HadoopFileBlock fileBlock2 = new HadoopFileBlock(HOSTS, inFileUri, fileBlock1.length(),
                 igfs.info(inFile).length() - fileBlock1.length());
 
-        HadoopV2Job gridJob = getHadoopJob(igfsScheme() + inFile.toString(), igfsScheme() + PATH_OUTPUT);
+        HadoopJob gridJob = getHadoopJob(igfsScheme() + inFile.toString(), igfsScheme() + PATH_OUTPUT);
 
         HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock1);
 
@@ -110,7 +109,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @return Context with mock output.
      * @throws IgniteCheckedException If fails.
      */
-    private HadoopTestTaskContext runTaskWithInput(HadoopV2Job gridJob, HadoopTaskType taskType,
+    private HadoopTestTaskContext runTaskWithInput(HadoopJob gridJob, HadoopTaskType taskType,
         int taskNum, String... words) throws IgniteCheckedException {
         HadoopTaskInfo taskInfo = new HadoopTaskInfo(taskType, gridJob.id(), taskNum, 0, null);
 
@@ -136,7 +135,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @throws Exception If fails.
      */
     public void testReduceTask() throws Exception {
-        HadoopV2Job gridJob = getHadoopJob(igfsScheme() + PATH_INPUT, igfsScheme() + PATH_OUTPUT);
+        HadoopJob gridJob = getHadoopJob(igfsScheme() + PATH_INPUT, igfsScheme() + PATH_OUTPUT);
 
         runTaskWithInput(gridJob, HadoopTaskType.REDUCE, 0, "word1", "5", "word2", "10");
         runTaskWithInput(gridJob, HadoopTaskType.REDUCE, 1, "word3", "7", "word4", "15");
@@ -162,7 +161,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @throws Exception If fails.
      */
     public void testCombinerTask() throws Exception {
-        HadoopV2Job gridJob = getHadoopJob("/", "/");
+        HadoopJob gridJob = getHadoopJob("/", "/");
 
         HadoopTestTaskContext ctx =
             runTaskWithInput(gridJob, HadoopTaskType.COMBINE, 0, "word1", "5", "word2", "10");
@@ -182,7 +181,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @return Context of combine task with mock output.
      * @throws IgniteCheckedException If fails.
      */
-    private HadoopTestTaskContext runMapCombineTask(HadoopFileBlock fileBlock, HadoopV2Job gridJob)
+    private HadoopTestTaskContext runMapCombineTask(HadoopFileBlock fileBlock, HadoopJob gridJob)
         throws IgniteCheckedException {
         HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock);
 
@@ -228,7 +227,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
         HadoopFileBlock fileBlock1 = new HadoopFileBlock(HOSTS, inFileUri, 0, l);
         HadoopFileBlock fileBlock2 = new HadoopFileBlock(HOSTS, inFileUri, l, fileLen - l);
 
-        HadoopV2Job gridJob = getHadoopJob(inFileUri.toString(), igfsScheme() + PATH_OUTPUT);
+        HadoopJob gridJob = getHadoopJob(inFileUri.toString(), igfsScheme() + PATH_OUTPUT);
 
         HadoopTestTaskContext combine1Ctx = runMapCombineTask(fileBlock1, gridJob);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
index b41a260..48e83cc 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.hadoop.mapred.*;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
 
 import java.io.*;
 import java.util.*;
@@ -38,7 +37,7 @@ public class HadoopTasksV1Test extends HadoopTasksAllVersionsTest {
      * @return Hadoop job.
      * @throws IOException If fails.
      */
-    @Override public HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception {
+    @Override public HadoopJob getHadoopJob(String inFile, String outFile) throws Exception {
         JobConf jobConf = HadoopWordCount1.getJob(inFile, outFile);
 
         setupFileSystems(jobConf);
@@ -47,7 +46,7 @@ public class HadoopTasksV1Test extends HadoopTasksAllVersionsTest {
 
         HadoopJobId jobId = new HadoopJobId(new UUID(0, 0), 0);
 
-        return new HadoopV2Job(jobId, jobInfo, log);
+        return jobInfo.createJob(jobId, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
index b677c63..e73fae3 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.lib.input.*;
 import org.apache.hadoop.mapreduce.lib.output.*;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
 
 import java.util.*;
 
@@ -42,7 +41,7 @@ public class HadoopTasksV2Test extends HadoopTasksAllVersionsTest {
      * @return Hadoop job.
      * @throws Exception if fails.
      */
-    @Override public HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception {
+    @Override public HadoopJob getHadoopJob(String inFile, String outFile) throws Exception {
         Job job = Job.getInstance();
 
         job.setOutputKeyClass(Text.class);
@@ -65,7 +64,7 @@ public class HadoopTasksV2Test extends HadoopTasksAllVersionsTest {
 
         HadoopJobId jobId = new HadoopJobId(new UUID(0, 0), 0);
 
-        return new HadoopV2Job(jobId, jobInfo, log);
+        return jobInfo.createJob(jobId, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
index ebc89f4..f3b9307 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
@@ -66,7 +66,11 @@ public class HadoopV2JobSelfTest extends HadoopAbstractSelfTest {
         cfg.setMapOutputValueClass(Text.class);
         cfg.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, CustomSerialization.class.getName());
 
-        HadoopJob job = new HadoopV2Job(new HadoopJobId(UUID.randomUUID(), 1), createJobInfo(cfg), log);
+        HadoopDefaultJobInfo info = createJobInfo(cfg);
+
+        HadoopJobId id = new HadoopJobId(UUID.randomUUID(), 1);
+
+        HadoopJob job = info.createJob(id, log);
 
         HadoopTaskContext taskCtx = job.getTaskContext(new HadoopTaskInfo(HadoopTaskType.MAP, null, 0, 0,
             null));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9f72917/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
index b4ed5e1..9395c5e 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
+import java.util.concurrent.*;
 
 /**
  * Abstract class for maps test.
@@ -95,9 +96,20 @@ public abstract class HadoopAbstractMapTest extends GridCommonAbstractTest {
             assert false;
         }
 
+        /** {@inheritDoc} */
         @Override public void cleanupTaskEnvironment() throws IgniteCheckedException {
             assert false;
         }
+
+        /** {@inheritDoc} */
+        @Override public <T> T runAsJobOwner(Callable<T> c) throws IgniteCheckedException {
+            try {
+                return c.call();
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException(e);
+            }
+        }
     }
 
     /**


[21/50] incubator-ignite git commit: # ignite-883 fix ConcurrentModificationException during DiscoCache initialization from the GridDiscoveryManager.start

Posted by sb...@apache.org.
# ignite-883 fix ConcurrentModificationException during DiscoCache initialization from the GridDiscoveryManager.start


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

Branch: refs/heads/ignite-633
Commit: f4da46cd25b61b3d7b55b6fba3884c8fa05f5cb8
Parents: 730ef10
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 10:27:04 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 10:27:04 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/managers/discovery/GridDiscoveryManager.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f4da46cd/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 142dbaa..5983553 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -173,7 +173,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         new ConcurrentHashMap8<>();
 
     /** Map of dynamic cache filters. */
-    private Map<String, CachePredicate> registeredCaches = new HashMap<>();
+    private Map<String, CachePredicate> registeredCaches = new ConcurrentHashMap<>();
 
     /** */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();


[39/50] incubator-ignite git commit: # ignite-sprint-5 added clientMode flag in node start message

Posted by sb...@apache.org.
# ignite-sprint-5 added clientMode flag in node start message


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

Branch: refs/heads/ignite-633
Commit: e7d8b5abf922a69edd7a4a17d15a6d4ded8075b8
Parents: 14bb076
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 9 11:35:46 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 9 11:35:46 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/IgniteKernal.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e7d8b5ab/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 1c12402..4f5e365 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -1550,7 +1550,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     ">>> Grid name: " + gridName + NL +
                     ">>> Local node [" +
                     "ID=" + locNode.id().toString().toUpperCase() +
-                    ", order=" + locNode.order() +
+                    ", order=" + locNode.order() + ", clientMode=" + ctx.clientNode() +
                     "]" + NL +
                     ">>> Local node addresses: " + U.addressesAsString(locNode) + NL +
                     ">>> Local ports: " + sb + NL;


[05/50] incubator-ignite git commit: Merge branches 'ignite-941' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-941

Posted by sb...@apache.org.
Merge branches 'ignite-941' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-941


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

Branch: refs/heads/ignite-633
Commit: b58c892e149567b3ca8d6746cb87cc11a4808dec
Parents: e9cea3a c9f7291
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 4 16:28:19 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 4 16:28:19 2015 -0700

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  13 +-
 .../apache/ignite/internal/IgniteKernal.java    |   3 +
 .../managers/communication/GridIoManager.java   | 117 ++++----
 .../dht/GridClientPartitionTopology.java        |   2 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  16 +-
 .../GridDhtPartitionsExchangeFuture.java        |  29 +-
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../processors/hadoop/HadoopTaskContext.java    |  14 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   2 +-
 .../internal/visor/query/VisorQueryJob.java     |   2 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   3 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  31 --
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  56 +++-
 .../tcp/ipfinder/TcpDiscoveryIpFinder.java      |  10 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  47 +++-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  62 ++++
 ...niteDynamicCacheWithConfigStartSelfTest.java |  97 +++++++
 .../igfs/IgfsClientCacheSelfTest.java           |   9 +-
 .../IgniteMessagingWithClientTest.java          | 164 +++++++++++
 .../tcp/TcpClientDiscoverySpiMulticastTest.java | 129 +++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   1 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   1 +
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |  43 +--
 .../fs/IgniteHadoopFileSystemCounterWriter.java |  14 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  70 ++---
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   2 +-
 .../internal/processors/hadoop/HadoopUtils.java | 282 ++++++++++++++++++-
 .../hadoop/SecondaryFileSystemProvider.java     |   4 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  48 +++-
 .../hadoop/HadoopClientProtocolSelfTest.java    |   6 +-
 .../hadoop/HadoopAbstractSelfTest.java          |  14 +-
 .../hadoop/HadoopCommandLineTest.java           |  14 +-
 .../processors/hadoop/HadoopMapReduceTest.java  | 176 +++++++++++-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |   2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |  15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |  12 +
 ...acheConfigurationPrimitiveTypesSelfTest.java | 104 +++++++
 .../IgniteCacheWithIndexingTestSuite.java       |   2 +
 .../commands/cache/VisorCacheScanCommand.scala  |   2 +-
 47 files changed, 1443 insertions(+), 273 deletions(-)
----------------------------------------------------------------------


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


[12/50] incubator-ignite git commit: # ignite-883

Posted by sb...@apache.org.
# ignite-883


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

Branch: refs/heads/ignite-633
Commit: fb827a7784614343ae639ea8b856d2f9f88d46db
Parents: db57652
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 5 11:41:46 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 5 15:10:00 2015 +0300

----------------------------------------------------------------------
 .../datastructures/DataStructuresProcessor.java |  31 +-
 .../timeout/GridSpiTimeoutObject.java           |  14 +
 .../util/nio/GridCommunicationClient.java       |   6 -
 .../util/nio/GridTcpCommunicationClient.java    | 554 -------------------
 .../util/nio/GridTcpNioCommunicationClient.java |   8 -
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   4 +
 .../communication/tcp/TcpCommunicationSpi.java  |  97 +---
 .../tcp/TcpCommunicationSpiMBean.java           |   2 -
 .../internal/util/nio/GridNioSelfTest.java      |   2 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |   4 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   2 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |   2 -
 ...cpCommunicationSpiMultithreadedSelfTest.java |   2 +-
 .../discovery/AbstractDiscoverySelfTest.java    |  13 +-
 14 files changed, 61 insertions(+), 680 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/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 2138639..aa3bfe2 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
@@ -101,7 +101,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
     private IgniteInternalCache<CacheDataStructuresCacheKey, List<CacheCollectionInfo>> utilityDataCache;
 
     /** */
-    private UUID qryId;
+    private volatile UUID qryId;
 
     /**
      * @param ctx Context.
@@ -144,11 +144,22 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
             seqView = atomicsCache;
 
             dsCacheCtx = atomicsCache.context();
+        }
+    }
 
-            qryId = dsCacheCtx.continuousQueries().executeInternalQuery(new DataStructuresEntryListener(),
-                new DataStructuresEntryFilter(),
-                dsCacheCtx.isReplicated() && dsCacheCtx.affinityNode(),
-                false);
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startQuery() throws IgniteCheckedException {
+        if (qryId == null) {
+            synchronized (this) {
+                if (qryId == null) {
+                    qryId = dsCacheCtx.continuousQueries().executeInternalQuery(new DataStructuresEntryListener(),
+                        new DataStructuresEntryFilter(),
+                        dsCacheCtx.isReplicated() && dsCacheCtx.affinityNode(),
+                        false);
+                }
+            }
         }
     }
 
@@ -178,6 +189,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
+        startQuery();
+
         return getAtomic(new IgniteOutClosureX<IgniteAtomicSequence>() {
             @Override public IgniteAtomicSequence applyx() throws IgniteCheckedException {
                 GridCacheInternalKey key = new GridCacheInternalKeyImpl(name);
@@ -304,6 +317,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
+        startQuery();
+
         return getAtomic(new IgniteOutClosureX<IgniteAtomicLong>() {
             @Override public IgniteAtomicLong applyx() throws IgniteCheckedException {
                 final GridCacheInternalKey key = new GridCacheInternalKeyImpl(name);
@@ -507,6 +522,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
+        startQuery();
+
         return getAtomic(new IgniteOutClosureX<IgniteAtomicReference>() {
             @Override public IgniteAtomicReference<T> applyx() throws IgniteCheckedException {
                 GridCacheInternalKey key = new GridCacheInternalKeyImpl(name);
@@ -608,6 +625,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
+        startQuery();
+
         return getAtomic(new IgniteOutClosureX<IgniteAtomicStamped>() {
             @Override public IgniteAtomicStamped<T, S> applyx() throws IgniteCheckedException {
                 GridCacheInternalKeyImpl key = new GridCacheInternalKeyImpl(name);
@@ -916,6 +935,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
+        startQuery();
+
         return getAtomic(new IgniteOutClosureX<IgniteCountDownLatch>() {
             @Override public IgniteCountDownLatch applyx() throws IgniteCheckedException {
                 GridCacheInternalKey key = new GridCacheInternalKeyImpl(name);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridSpiTimeoutObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridSpiTimeoutObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridSpiTimeoutObject.java
index 82267a2..a0fd9b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridSpiTimeoutObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridSpiTimeoutObject.java
@@ -53,6 +53,20 @@ public class GridSpiTimeoutObject implements GridTimeoutObject {
     }
 
     /** {@inheritDoc} */
+    @Override public int hashCode() {
+        assert false;
+
+        return super.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        assert false;
+
+        return super.equals(obj);
+    }
+
+    /** {@inheritDoc} */
     @Override public final String toString() {
         return S.toString(GridSpiTimeoutObject.class, this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
index 2f7fd88..693a5a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
@@ -100,12 +100,6 @@ public interface GridCommunicationClient {
     public boolean sendMessage(@Nullable UUID nodeId, Message msg) throws IgniteCheckedException;
 
     /**
-     * @param timeout Timeout.
-     * @throws IOException If failed.
-     */
-    public void flushIfNeeded(long timeout) throws IOException;
-
-    /**
      * @return {@code True} if send is asynchronous.
      */
     public boolean async();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpCommunicationClient.java
deleted file mode 100644
index 72c20f8..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpCommunicationClient.java
+++ /dev/null
@@ -1,554 +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.util.nio;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.plugin.extensions.communication.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.net.*;
-import java.nio.*;
-import java.util.*;
-import java.util.concurrent.locks.*;
-
-/**
- * Grid client for NIO server.
- */
-public class GridTcpCommunicationClient extends GridAbstractCommunicationClient {
-    /** Socket. */
-    private final Socket sock;
-
-    /** Output stream. */
-    private final UnsafeBufferedOutputStream out;
-
-    /** Minimum buffered message count. */
-    private final int minBufferedMsgCnt;
-
-    /** Communication buffer size ratio. */
-    private final double bufSizeRatio;
-
-    /** */
-    private final ByteBuffer writeBuf;
-
-    /** */
-    private final MessageFormatter formatter;
-
-    /**
-     * @param metricsLsnr Metrics listener.
-     * @param addr Address.
-     * @param locHost Local address.
-     * @param connTimeout Connect timeout.
-     * @param tcpNoDelay Value for {@code TCP_NODELAY} socket option.
-     * @param sockRcvBuf Socket receive buffer.
-     * @param sockSndBuf Socket send buffer.
-     * @param bufSize Buffer size (or {@code 0} to disable buffer).
-     * @param minBufferedMsgCnt Minimum buffered message count.
-     * @param bufSizeRatio Communication buffer size ratio.
-     * @param formatter Message formatter.
-     * @throws IgniteCheckedException If failed.
-     */
-    public GridTcpCommunicationClient(
-        GridNioMetricsListener metricsLsnr,
-        InetSocketAddress addr,
-        InetAddress locHost,
-        long connTimeout,
-        boolean tcpNoDelay,
-        int sockRcvBuf,
-        int sockSndBuf,
-        int bufSize,
-        int minBufferedMsgCnt,
-        double bufSizeRatio,
-        MessageFormatter formatter
-    ) throws IgniteCheckedException {
-        super(metricsLsnr);
-
-        assert metricsLsnr != null;
-        assert addr != null;
-        assert locHost != null;
-        assert connTimeout >= 0;
-        assert bufSize >= 0;
-
-        A.ensure(minBufferedMsgCnt >= 0,
-            "Value of minBufferedMessageCount property cannot be less than zero.");
-        A.ensure(bufSizeRatio > 0 && bufSizeRatio < 1,
-            "Value of bufSizeRatio property must be between 0 and 1 (exclusive).");
-
-        this.minBufferedMsgCnt = minBufferedMsgCnt;
-        this.bufSizeRatio = bufSizeRatio;
-        this.formatter = formatter;
-
-        writeBuf = ByteBuffer.allocate(8 << 10);
-
-        writeBuf.order(ByteOrder.nativeOrder());
-
-        sock = new Socket();
-
-        boolean success = false;
-
-        try {
-            sock.bind(new InetSocketAddress(locHost, 0));
-
-            sock.setTcpNoDelay(tcpNoDelay);
-
-            if (sockRcvBuf > 0)
-                sock.setReceiveBufferSize(sockRcvBuf);
-
-            if (sockSndBuf > 0)
-                sock.setSendBufferSize(sockSndBuf);
-
-            sock.connect(addr, (int)connTimeout);
-
-            out = new UnsafeBufferedOutputStream(sock.getOutputStream(), bufSize);
-
-            success = true;
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to connect to remote host " +
-                "[addr=" + addr + ", localHost=" + locHost + ']', e);
-        }
-        finally {
-            if (!success)
-                U.closeQuiet(sock);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void doHandshake(IgniteInClosure2X<InputStream, OutputStream> handshakeC) throws IgniteCheckedException {
-        try {
-            handshakeC.applyx(sock.getInputStream(), sock.getOutputStream());
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to access IO streams when executing handshake with remote node: " +
-                sock.getRemoteSocketAddress(), e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean close() {
-        boolean res = super.close();
-
-        if (res) {
-            U.closeQuiet(out);
-            U.closeQuiet(sock);
-        }
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void forceClose() {
-        super.forceClose();
-
-        try {
-            out.flush();
-        }
-        catch (IOException ignored) {
-            // No-op.
-        }
-
-        // Do not call (directly or indirectly) out.close() here
-        // since it may cause a deadlock.
-        out.forceClose();
-
-        U.closeQuiet(sock);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendMessage(byte[] data, int len) throws IgniteCheckedException {
-        if (closed())
-            throw new IgniteCheckedException("Client was closed: " + this);
-
-        try {
-            out.write(data, 0, len);
-
-            metricsLsnr.onBytesSent(len);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to send message to remote node: " + sock.getRemoteSocketAddress(), e);
-        }
-
-        markUsed();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean sendMessage(@Nullable UUID nodeId, Message msg)
-        throws IgniteCheckedException {
-        if (closed())
-            throw new IgniteCheckedException("Client was closed: " + this);
-
-        assert writeBuf.hasArray();
-
-        try {
-            int cnt = U.writeMessageFully(msg, out, writeBuf, formatter.writer());
-
-            metricsLsnr.onBytesSent(cnt);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to send message to remote node: " + sock.getRemoteSocketAddress(), e);
-        }
-
-        markUsed();
-
-        return false;
-    }
-
-    /**
-     * @param timeout Timeout.
-     * @throws IOException If failed.
-     */
-    @Override public void flushIfNeeded(long timeout) throws IOException {
-        assert timeout > 0;
-
-        out.flushOnTimeout(timeout);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendMessage(ByteBuffer data) throws IgniteCheckedException {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridTcpCommunicationClient.class, this, super.toString());
-    }
-
-    /**
-     *
-     */
-    private class UnsafeBufferedOutputStream extends FilterOutputStream {
-        /** The internal buffer where data is stored. */
-        private final byte buf[];
-
-        /** Current size. */
-        private int size;
-
-        /** Count. */
-        private int cnt;
-
-        /** Message count. */
-        private int msgCnt;
-
-        /** Total messages size. */
-        private int totalCnt;
-
-        /** Lock. */
-        private final ReentrantLock lock = new ReentrantLock();
-
-        /** Last flushed timestamp. */
-        private volatile long lastFlushed = U.currentTimeMillis();
-
-        /** Cached flush timeout. */
-        private volatile long flushTimeout;
-
-        /** Buffer adjusted timestamp. */
-        private long lastAdjusted = U.currentTimeMillis();
-
-        /**
-         * Creates a new buffered output stream to write data to the
-         * specified underlying output stream.
-         *
-         * @param out The underlying output stream.
-         */
-        UnsafeBufferedOutputStream(OutputStream out) {
-            this(out, 8192);
-        }
-
-        /**
-         * Creates a new buffered output stream to write data to the
-         * specified underlying output stream with the specified buffer
-         * size.
-         *
-         * @param out The underlying output stream.
-         * @param size The buffer size.
-         */
-        UnsafeBufferedOutputStream(OutputStream out, int size) {
-            super(out);
-
-            assert size >= 0;
-
-            this.size = size;
-            buf = size > 0 ? new byte[size] : null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(int b) throws IOException {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(byte[] b, int off, int len) throws IOException {
-            assert b != null;
-            assert off == 0;
-
-            // No buffering.
-            if (buf == null) {
-                lock.lock();
-
-                try {
-                    out.write(b, 0, len);
-                }
-                finally {
-                    lock.unlock();
-                }
-
-                return;
-            }
-
-            // Buffering is enabled.
-            lock.lock();
-
-            try {
-                msgCnt++;
-                totalCnt += len;
-
-                if (len >= size) {
-                    flushLocked();
-
-                    out.write(b, 0, len);
-
-                    lastFlushed = U.currentTimeMillis();
-
-                    adjustBufferIfNeeded();
-
-                    return;
-                }
-
-                if (cnt + len > size) {
-                    flushLocked();
-
-                    messageToBuffer0(b, off, len, buf, 0);
-
-                    cnt = len;
-
-                    assert cnt < size;
-
-                    adjustBufferIfNeeded();
-
-                    return;
-                }
-
-                messageToBuffer0(b, 0, len, buf, cnt);
-
-                cnt += len;
-
-                if (cnt == size)
-                    flushLocked();
-                else
-                    flushIfNeeded();
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-
-        /**
-         * @throws IOException If failed.
-         */
-        private void flushIfNeeded() throws IOException {
-            assert lock.isHeldByCurrentThread();
-            assert buf != null;
-
-            long flushTimeout0 = flushTimeout;
-
-            if (flushTimeout0 > 0)
-                flushOnTimeoutLocked(flushTimeout0);
-        }
-
-        /**
-         *
-         */
-        private void adjustBufferIfNeeded() {
-            assert lock.isHeldByCurrentThread();
-            assert buf != null;
-
-            long flushTimeout0 = flushTimeout;
-
-            if (flushTimeout0 > 0)
-                adjustBufferLocked(flushTimeout0);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void flush() throws IOException {
-            lock.lock();
-
-            try {
-                flushLocked();
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-
-        /**
-         * @param timeout Timeout.
-         * @throws IOException If failed.
-         */
-        public void flushOnTimeout(long timeout) throws IOException {
-            assert buf != null;
-            assert timeout > 0;
-
-            // Overwrite cached value.
-            flushTimeout = timeout;
-
-            if (lastFlushed + timeout > U.currentTimeMillis() || !lock.tryLock())
-                return;
-
-            try {
-                flushOnTimeoutLocked(timeout);
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-
-        /**
-         * @param timeout Timeout.
-         * @throws IOException If failed.
-         */
-        private void flushOnTimeoutLocked(long timeout) throws IOException {
-            assert lock.isHeldByCurrentThread();
-            assert timeout > 0;
-
-            // Double check.
-            if (cnt == 0 || lastFlushed + timeout > U.currentTimeMillis())
-                return;
-
-            flushLocked();
-
-            adjustBufferLocked(timeout);
-        }
-
-        /**
-         * @param timeout Timeout.
-         */
-        private void adjustBufferLocked(long timeout) {
-            assert lock.isHeldByCurrentThread();
-            assert timeout > 0;
-
-            long time = U.currentTimeMillis();
-
-            if (lastAdjusted + timeout < time) {
-                if (msgCnt <= minBufferedMsgCnt)
-                    size = 0;
-                else {
-                    size = (int)(totalCnt * bufSizeRatio);
-
-                    if (size > buf.length)
-                        size = buf.length;
-                }
-
-                msgCnt = 0;
-                totalCnt = 0;
-
-                lastAdjusted = time;
-            }
-        }
-
-        /**
-         * @throws IOException If failed.
-         */
-        private void flushLocked() throws IOException {
-            assert lock.isHeldByCurrentThread();
-
-            if (buf != null && cnt > 0) {
-                out.write(buf, 0, cnt);
-
-                cnt = 0;
-            }
-
-            out.flush();
-
-            lastFlushed = U.currentTimeMillis();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() throws IOException {
-            lock.lock();
-
-            try {
-                flushLocked();
-            }
-            finally {
-                try {
-                    out.close();
-                }
-                finally {
-                    lock.unlock();
-                }
-            }
-        }
-
-        /**
-         * Forcibly closes underlying stream ignoring any possible exception.
-         */
-        public void forceClose() {
-            try {
-                out.close();
-            }
-            catch (IOException ignored) {
-                // No-op.
-            }
-        }
-
-        /**
-         * @param b Buffer to copy from.
-         * @param off Offset in source buffer.
-         * @param len Length.
-         * @param resBuf Result buffer.
-         * @param resOff Result offset.
-         */
-        private void messageToBuffer(byte[] b, int off, int len, byte[] resBuf, int resOff) {
-            assert b.length == len;
-            assert off == 0;
-            assert resBuf.length >= resOff + len + 4;
-
-            U.intToBytes(len, resBuf, resOff);
-
-            U.arrayCopy(b, off, resBuf, resOff + 4, len);
-        }
-
-        /**
-         * @param b Buffer to copy from (length included).
-         * @param off Offset in source buffer.
-         * @param len Length.
-         * @param resBuf Result buffer.
-         * @param resOff Result offset.
-         */
-        private void messageToBuffer0(byte[] b, int off, int len, byte[] resBuf, int resOff) {
-            assert off == 0;
-            assert resBuf.length >= resOff + len;
-
-            U.arrayCopy(b, off, resBuf, resOff, len);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            lock.lock();
-
-            try {
-                return S.toString(UnsafeBufferedOutputStream.class, this);
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
index 788a8e6..abad875 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
@@ -122,14 +122,6 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
         return false;
     }
 
-    /**
-     * @param timeout Timeout.
-     * @throws IOException If failed.
-     */
-    @Override public void flushIfNeeded(long timeout) throws IOException {
-        // No-op.
-    }
-
     /** {@inheritDoc} */
     @Override public boolean async() {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index c9c633f..d095491 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -730,11 +730,15 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
 
         /** {@inheritDoc} */
         @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
+            assert ignite instanceof IgniteKernal : ignite;
+
             ((IgniteKernal)ignite).context().timeout().addTimeoutObject(new GridSpiTimeoutObject(obj));
         }
 
         /** {@inheritDoc} */
         @Override public void removeTimeoutObject(IgniteSpiTimeoutObject obj) {
+            assert ignite instanceof IgniteKernal : ignite;
+
             ((IgniteKernal)ignite).context().timeout().removeTimeoutObject(new GridSpiTimeoutObject(obj));
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index b324ab2..359de1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -157,12 +157,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Default idle connection timeout (value is <tt>30000</tt>ms). */
     public static final long DFLT_IDLE_CONN_TIMEOUT = 30000;
 
-    /** Default value for connection buffer flush frequency (value is <tt>100</tt> ms). */
-    public static final long DFLT_CONN_BUF_FLUSH_FREQ = 100;
-
-    /** Default value for connection buffer size (value is <tt>0</tt>). */
-    public static final int DFLT_CONN_BUF_SIZE = 0;
-
     /** Default socket send and receive buffer size. */
     public static final int DFLT_SOCK_BUF_SIZE = 32 * 1024;
 
@@ -603,13 +597,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Idle connection timeout. */
     private long idleConnTimeout = DFLT_IDLE_CONN_TIMEOUT;
 
-    /** Connection buffer flush frequency. */
-    private volatile long connBufFlushFreq = DFLT_CONN_BUF_FLUSH_FREQ;
-
-    /** Connection buffer size. */
-    @SuppressWarnings("RedundantFieldInitialization")
-    private int connBufSize = DFLT_CONN_BUF_SIZE;
-
     /** Connect timeout. */
     private long connTimeout = DFLT_CONN_TIMEOUT;
 
@@ -647,9 +634,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Socket write timeout. */
     private long sockWriteTimeout = DFLT_SOCK_WRITE_TIMEOUT;
 
-    /** Flush client worker. */
-    private ClientFlushWorker clientFlushWorker;
-
     /** Recovery and idle clients handler. */
     private CommunicationWorker commWorker;
 
@@ -876,31 +860,29 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
     /**
      * Sets connection buffer size. If set to {@code 0} connection buffer is disabled.
-     * <p>
-     * If not provided, default value is {@link #DFLT_CONN_BUF_SIZE}.
      *
      * @param connBufSize Connection buffer size.
      * @see #setConnectionBufferFlushFrequency(long)
      */
     @IgniteSpiConfiguration(optional = true)
     public void setConnectionBufferSize(int connBufSize) {
-        this.connBufSize = connBufSize;
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public int getConnectionBufferSize() {
-        return connBufSize;
+        return 0;
     }
 
     /** {@inheritDoc} */
     @IgniteSpiConfiguration(optional = true)
     @Override public void setConnectionBufferFlushFrequency(long connBufFlushFreq) {
-        this.connBufFlushFreq = connBufFlushFreq;
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public long getConnectionBufferFlushFrequency() {
-        return connBufFlushFreq;
+        return 0;
     }
 
     /**
@@ -1168,8 +1150,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         assertParameter(locPort <= 0xffff, "locPort < 0xffff");
         assertParameter(locPortRange >= 0, "locPortRange >= 0");
         assertParameter(idleConnTimeout > 0, "idleConnTimeout > 0");
-        assertParameter(connBufFlushFreq > 0, "connBufFlushFreq > 0");
-        assertParameter(connBufSize >= 0, "connBufSize >= 0");
         assertParameter(sockRcvBuf >= 0, "sockRcvBuf >= 0");
         assertParameter(sockSndBuf >= 0, "sockSndBuf >= 0");
         assertParameter(msgQueueLimit >= 0, "msgQueueLimit >= 0");
@@ -1239,8 +1219,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             log.debug(configInfo("idleConnTimeout", idleConnTimeout));
             log.debug(configInfo("directBuf", directBuf));
             log.debug(configInfo("directSendBuf", directSndBuf));
-            log.debug(configInfo("connBufSize", connBufSize));
-            log.debug(configInfo("connBufFlushFreq", connBufFlushFreq));
             log.debug(configInfo("selectorsCnt", selectorsCnt));
             log.debug(configInfo("tcpNoDelay", tcpNoDelay));
             log.debug(configInfo("sockSndBuf", sockSndBuf));
@@ -1255,11 +1233,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             log.debug(configInfo("unackedMsgsBufSize", unackedMsgsBufSize));
         }
 
-        if (connBufSize > 8192)
-            U.warn(log, "Specified communication IO buffer size is larger than recommended (ignore if done " +
-                "intentionally) [specified=" + connBufSize + ", recommended=8192]",
-                "Specified communication IO buffer size is larger than recommended (ignore if done intentionally).");
-
         if (!tcpNoDelay)
             U.quietAndWarn(log, "'TCP_NO_DELAY' for communication is off, which should be used with caution " +
                 "since may produce significant delays with some scenarios.");
@@ -1272,12 +1245,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
         commWorker.start();
 
-        if (connBufSize > 0) {
-            clientFlushWorker = new ClientFlushWorker();
-
-            clientFlushWorker.start();
-        }
-
         // Ack start.
         if (log.isDebugEnabled())
             log.debug(startInfo());
@@ -1431,10 +1398,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         if (nioSrvr != null)
             nioSrvr.stop();
 
-        U.interrupt(clientFlushWorker);
         U.interrupt(commWorker);
 
-        U.join(clientFlushWorker, log);
         U.join(commWorker, log);
 
         // Force closing on stop (safety).
@@ -2023,10 +1988,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         if (nioSrvr != null)
             nioSrvr.stop();
 
-        U.interrupt(clientFlushWorker);
         U.interrupt(commWorker);
 
-        U.join(clientFlushWorker, log);
         U.join(commWorker, log);
 
         for (GridCommunicationClient client : clients.values())
@@ -2134,58 +2097,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /**
      *
      */
-    private class ClientFlushWorker extends IgniteSpiThread {
-        /**
-         *
-         */
-        ClientFlushWorker() {
-            super(gridName, "nio-client-flusher", log);
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings({"BusyWait"})
-        @Override protected void body() throws InterruptedException {
-            while (!isInterrupted()) {
-                long connBufFlushFreq0 = connBufFlushFreq;
-
-                for (Map.Entry<UUID, GridCommunicationClient> entry : clients.entrySet()) {
-                    GridCommunicationClient client = entry.getValue();
-
-                    if (client.reserve()) {
-                        boolean err = true;
-
-                        try {
-                            client.flushIfNeeded(connBufFlushFreq0);
-
-                            err = false;
-                        }
-                        catch (IOException e) {
-                            if (getSpiContext().pingNode(entry.getKey()))
-                                U.error(log, "Failed to flush client: " + client, e);
-                            else {
-                                if (log.isDebugEnabled())
-                                    log.debug("Failed to flush client (node left): " + client);
-
-                                onException("Failed to flush client (node left): " + client, e);
-                            }
-                        }
-                        finally {
-                            if (err)
-                                client.forceClose();
-                            else
-                                client.release();
-                        }
-                    }
-                }
-
-                Thread.sleep(connBufFlushFreq0);
-            }
-        }
-    }
-
-    /**
-     *
-     */
     private class CommunicationWorker extends IgniteSpiThread {
         /** */
         private final BlockingQueue<GridNioRecoveryDescriptor> q = new LinkedBlockingQueue<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index 5c80e6e..6f5a738 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -171,8 +171,6 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      * This frequency defines how often system will advice to flush
      * connection buffer.
      * <p>
-     * If not provided, default value is {@link TcpCommunicationSpi#DFLT_CONN_BUF_FLUSH_FREQ}.
-     * <p>
      * This property is used only if {@link #getConnectionBufferSize()} is greater than {@code 0}.
      *
      * @param connBufFlushFreq Flush frequency.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
index e3baeb0..bdf9929 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
@@ -1286,7 +1286,7 @@ public class GridNioSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Test client to use instead of {@link GridTcpCommunicationClient}
+     * Test client to use instead of {@link GridTcpNioCommunicationClient}
      */
     private static class TestClient implements AutoCloseable {
         /** Socket implementation to use. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
index ea51aff..8d27485 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
@@ -64,7 +64,7 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
 
         // Test idle clients remove.
         for (CommunicationSpi spi : spis.values()) {
-            ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
+            ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
 
             assertEquals(2, clients.size());
 
@@ -77,7 +77,7 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
         super.afterTest();
 
         for (CommunicationSpi spi : spis.values()) {
-            ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
+            ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
 
             for (int i = 0; i < 20 && !clients.isEmpty(); i++) {
                 info("Check failed for SPI [grid=" +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
index c038ee7..2d175f5 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
@@ -256,7 +256,7 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
                     assertTrue(latch.await(10, TimeUnit.SECONDS));
 
                     for (CommunicationSpi spi : spis) {
-                        ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
+                        ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
 
                         assertEquals(1, clients.size());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
index 34fa610..c4a0916 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
@@ -32,8 +32,6 @@ public class GridTcpCommunicationSpiConfigSelfTest extends GridSpiAbstractConfig
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "localPort", 65636);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "localPortRange", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "idleConnectionTimeout", 0);
-        checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionBufferSize", -1);
-        checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionBufferFlushFrequency", 0);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "socketReceiveBuffer", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "socketSendBuffer", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "messageQueueLimit", -1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index e7ae957..3916f02 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -501,7 +501,7 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
         }
 
         for (CommunicationSpi spi : spis.values()) {
-            final ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
+            final ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
 
             assert GridTestUtils.waitForCondition(new PA() {
                 @Override public boolean apply() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb827a77/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
index 9c6fbb4..61bb944 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.spi.*;
+import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.config.*;
 import org.apache.ignite.testframework.junits.*;
 import org.apache.ignite.testframework.junits.spi.*;
@@ -267,9 +268,8 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
 
             Collection<UUID> nodeIds = new HashSet<>();
 
-            for (IgniteTestResources rsrc : spiRsrcs) {
+            for (IgniteTestResources rsrc : spiRsrcs)
                 nodeIds.add(rsrc.getNodeId());
-            }
 
             for (ClusterNode node : spi.getRemoteNodes()) {
                 if (nodeIds.contains(node.id())) {
@@ -390,6 +390,10 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
                     }
                 });
 
+                GridSpiTestContext ctx = initSpiContext();
+
+                GridTestUtils.setFieldValue(spi, IgniteSpiAdapter.class, "spiCtx", ctx);
+
                 spi.spiStart(getTestGridName() + i);
 
                 spis.add(spi);
@@ -397,7 +401,7 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
                 spiRsrcs.add(rsrcMgr);
 
                 // Force to use test context instead of default dummy context.
-                spi.onContextInitialized(initSpiContext());
+                spi.onContextInitialized(ctx);
             }
         }
         catch (Throwable e) {
@@ -438,9 +442,8 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
             spi.spiStop();
         }
 
-        for (IgniteTestResources rscrs : spiRsrcs) {
+        for (IgniteTestResources rscrs : spiRsrcs)
             rscrs.stopThreads();
-        }
 
         // Clear.
         spis.clear();



[22/50] incubator-ignite git commit: ignite-883

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


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

Branch: refs/heads/ignite-633
Commit: c7bc598a32c06cd1acc6f3e1b162d059d80a53f2
Parents: f4da46c
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 11:16:57 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 11:16:57 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/managers/discovery/GridDiscoveryManager.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c7bc598a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 5983553..142dbaa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -173,7 +173,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         new ConcurrentHashMap8<>();
 
     /** Map of dynamic cache filters. */
-    private Map<String, CachePredicate> registeredCaches = new ConcurrentHashMap<>();
+    private Map<String, CachePredicate> registeredCaches = new HashMap<>();
 
     /** */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();


[50/50] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-633

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-633

Conflicts:
	modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java


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

Branch: refs/heads/ignite-633
Commit: 0b5b90ea808e13e16825a5d9f28d8bcd588b8081
Parents: 02780e4 8018af4
Author: iveselovskiy <iv...@gridgain.com>
Authored: Wed Jun 10 13:35:19 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Wed Jun 10 13:35:19 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   42 +-
 assembly/dependencies-visor-console.xml         |    3 +
 .../hibernate/CacheHibernatePersonStore.java    |  202 +-
 .../hibernate/CacheHibernateStoreExample.java   |   17 +
 .../store/jdbc/CacheJdbcPersonStore.java        |  180 +-
 .../store/jdbc/CacheJdbcStoreExample.java       |   13 +
 .../store/spring/CacheSpringPersonStore.java    |  128 +
 .../store/spring/CacheSpringStoreExample.java   |  143 +
 .../datagrid/store/spring/package-info.java     |   22 +
 .../java/org/apache/ignite/IgniteCache.java     |   16 +
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |  187 +-
 .../ignite/cache/eviction/EvictableEntry.java   |    7 +
 .../ignite/cache/eviction/EvictionPolicy.java   |    2 +
 .../cache/eviction/fifo/FifoEvictionPolicy.java |  117 +-
 .../eviction/fifo/FifoEvictionPolicyMBean.java  |   22 +
 .../cache/eviction/lru/LruEvictionPolicy.java   |  135 +-
 .../eviction/lru/LruEvictionPolicyMBean.java    |   38 +
 .../eviction/random/RandomEvictionPolicy.java   |   10 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |  141 +-
 .../sorted/SortedEvictionPolicyMBean.java       |   22 +
 .../apache/ignite/cache/query/QueryMetrics.java |    6 +-
 .../apache/ignite/cache/store/CacheStore.java   |    2 +
 .../ignite/cache/store/CacheStoreSession.java   |   22 +
 .../cache/store/CacheStoreSessionListener.java  |  133 +
 .../jdbc/CacheJdbcStoreSessionListener.java     |  141 +
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   66 +-
 .../configuration/IgniteConfiguration.java      |   48 +-
 .../org/apache/ignite/igfs/IgfsUserContext.java |  119 +
 .../igfs/secondary/IgfsSecondaryFileSystem.java |    7 +
 .../ignite/internal/ClusterMetricsSnapshot.java |   14 +
 .../internal/GridEventConsumeHandler.java       |  100 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../apache/ignite/internal/IgniteKernal.java    |   56 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    5 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   30 +-
 .../internal/MarshallerContextAdapter.java      |   36 +-
 .../ignite/internal/MarshallerContextImpl.java  |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |   18 +-
 .../internal/igfs/common/IgfsMarshaller.java    |   35 +-
 .../igfs/common/IgfsPathControlRequest.java     |   22 +
 .../internal/managers/GridManagerAdapter.java   |   68 +-
 .../checkpoint/GridCheckpointManager.java       |   52 +-
 .../managers/communication/GridIoManager.java   |  129 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   54 +
 .../discovery/GridDiscoveryManager.java         |  278 +-
 .../managers/indexing/GridIndexingManager.java  |    4 -
 .../affinity/GridAffinityAssignment.java        |   12 +
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../affinity/GridAffinityProcessor.java         |   23 +-
 .../cache/CacheEvictableEntryImpl.java          |   31 +
 .../processors/cache/CacheMetricsImpl.java      |  367 +-
 .../cache/CacheMetricsMXBeanImpl.java           |  100 +
 .../processors/cache/CacheMetricsSnapshot.java  |  380 +-
 .../cache/DynamicCacheChangeBatch.java          |   29 +-
 .../cache/DynamicCacheDescriptor.java           |   19 +
 .../processors/cache/GridCacheAdapter.java      |   50 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |   22 +-
 .../processors/cache/GridCacheEntryEx.java      |    6 +
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMapEntry.java     |   69 +-
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   97 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |  121 +-
 .../processors/cache/GridCacheProxyImpl.java    |   26 +-
 .../cache/GridCacheSharedContext.java           |   48 +-
 .../processors/cache/GridCacheSwapManager.java  |  118 +-
 .../processors/cache/GridCacheTtlManager.java   |    9 +-
 .../processors/cache/GridCacheUtils.java        |  288 +-
 .../processors/cache/IgniteCacheProxy.java      |   23 +
 .../processors/cache/IgniteInternalCache.java   |   16 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedCacheEntry.java  |    7 -
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../dht/GridClientPartitionTopology.java        |   10 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtCacheEntry.java      |    6 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |   11 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   38 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   53 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   84 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   44 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  442 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   39 +-
 .../preloader/GridDhtPreloaderAssignments.java  |    3 +-
 .../distributed/near/GridNearAtomicCache.java   |    5 +
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 +-
 .../distributed/near/GridNearLockFuture.java    |  271 +-
 .../distributed/near/GridNearLockRequest.java   |   68 +-
 .../distributed/near/GridNearLockResponse.java  |   48 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   83 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    5 +-
 .../near/GridNearTransactionalCache.java        |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |   43 +-
 .../near/GridNearTxPrepareRequest.java          |   72 +-
 .../near/GridNearTxPrepareResponse.java         |   70 +-
 .../processors/cache/local/GridLocalCache.java  |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |   31 +-
 .../cache/query/GridCacheQueryAdapter.java      |   24 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/store/CacheOsStoreManager.java        |    1 -
 .../cache/store/CacheStoreManager.java          |    7 +-
 .../store/GridCacheStoreManagerAdapter.java     |  202 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   48 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../transactions/IgniteTxLocalAdapter.java      |  170 +-
 .../cache/transactions/IgniteTxManager.java     |    3 -
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   63 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  838 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../datastreamer/DataStreamerImpl.java          |   94 +-
 .../datastructures/DataStructuresProcessor.java |  107 +-
 .../internal/processors/hadoop/HadoopJob.java   |    2 +-
 .../processors/hadoop/HadoopTaskContext.java    |   14 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |    8 +-
 .../internal/processors/igfs/IgfsImpl.java      |    8 +-
 .../processors/igfs/IgfsIpcHandler.java         |  184 +-
 .../processors/igfs/IgfsMetaManager.java        |    2 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |    9 +-
 .../internal/processors/igfs/IgfsServer.java    |    4 +-
 .../internal/processors/igfs/IgfsUtils.java     |   16 +
 .../processors/query/GridQueryProcessor.java    |  305 +-
 .../service/GridServiceProcessor.java           |  125 +-
 .../timeout/GridSpiTimeoutObject.java           |   73 +
 .../timeout/GridTimeoutProcessor.java           |  105 +-
 .../IgniteTxRollbackCheckedException.java       |    9 +
 .../ignite/internal/util/GridJavaProcess.java   |   30 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../util/nio/GridCommunicationClient.java       |   30 +-
 .../util/nio/GridNioDelimitedBuffer.java        |    2 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |   13 +-
 .../util/nio/GridTcpCommunicationClient.java    |  554 --
 .../util/nio/GridTcpNioCommunicationClient.java |    8 -
 .../ignite/internal/visor/cache/VisorCache.java |    2 +-
 .../VisorCacheConfigurationCollectorJob.java    |    6 +-
 .../internal/visor/cache/VisorCacheMetrics.java |   19 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |   10 +-
 .../cache/VisorCacheStoreConfiguration.java     |    5 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |    9 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |   17 +-
 .../node/VisorNodeSuppressedErrorsTask.java     |   12 +-
 .../internal/visor/query/VisorQueryJob.java     |   13 +-
 .../internal/visor/query/VisorQueryTask.java    |    3 +-
 .../visor/util/VisorExceptionWrapper.java       |   81 +
 .../internal/visor/util/VisorTaskUtils.java     |   10 +
 .../ignite/mxbean/CacheMetricsMXBean.java       |   80 +
 .../apache/ignite/plugin/PluginProvider.java    |   26 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   70 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   57 +-
 .../ignite/spi/IgniteSpiTimeoutObject.java      |   44 +
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |    3 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  438 +-
 .../tcp/TcpCommunicationSpiMBean.java           |    2 -
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../discovery/DiscoverySpiCustomMessage.java    |   40 +
 .../spi/discovery/DiscoverySpiListener.java     |    5 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 1478 +++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 4733 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  212 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5771 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../tcp/ipfinder/TcpDiscoveryIpFinder.java      |   10 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   57 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   24 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   67 +
 .../messages/TcpDiscoveryClientPingRequest.java |   56 +
 .../TcpDiscoveryClientPingResponse.java         |   67 +
 .../TcpDiscoveryCustomEventMessage.java         |   41 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   28 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   43 +
 .../messages/TcpDiscoveryNodeAddedMessage.java  |    2 +-
 .../tcp/messages/TcpDiscoveryPingRequest.java   |    6 +
 .../tcp/messages/TcpDiscoveryPingResponse.java  |   15 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    8 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 ...cheStoreSessionListenerAbstractSelfTest.java |  315 +
 ...heStoreSessionListenerLifecycleSelfTest.java |  395 ++
 .../CacheJdbcStoreSessionListenerSelfTest.java  |  175 +
 .../ignite/internal/GridAffinitySelfTest.java   |    1 +
 .../internal/GridDiscoveryEventSelfTest.java    |    7 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    7 +-
 .../internal/GridProjectionAbstractTest.java    |   16 +
 .../GridProjectionForCachesSelfTest.java        |   11 +-
 .../internal/GridReleaseTypeSelfTest.java       |   77 +-
 .../apache/ignite/internal/GridSelfTest.java    |    4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   62 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |  122 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   46 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    1 +
 ...acheReadOnlyTransactionalClientSelfTest.java |  327 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  113 +
 .../cache/GridCacheAbstractMetricsSelfTest.java |   48 +-
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 .../GridCacheConcurrentTxMultiNodeTest.java     |    8 +-
 ...idCacheConfigurationConsistencySelfTest.java |   14 +-
 .../GridCacheExAbstractFullApiSelfTest.java     |  103 -
 .../cache/GridCacheMemoryModeSelfTest.java      |   23 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../processors/cache/GridCacheOffHeapTest.java  |    5 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../cache/GridCacheReloadSelfTest.java          |    6 +-
 .../GridCacheReturnValueTransferSelfTest.java   |    3 +
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  190 +
 .../processors/cache/GridCacheTestEntryEx.java  |    4 +
 .../GridCacheVariableTopologySelfTest.java      |   12 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |    2 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    3 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   29 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |   13 +-
 .../IgniteCachePartitionMapUpdateTest.java      |  226 +
 .../cache/IgniteCachePeekModesAbstractTest.java |    5 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   62 +
 ...niteDynamicCacheWithConfigStartSelfTest.java |   97 +
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../IgniteCountDownLatchAbstractSelfTest.java   |  102 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../IgniteCrossCacheTxStoreSelfTest.java        |  147 +-
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   11 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |   11 +-
 .../GridCacheExColocatedFullApiSelfTest.java    |   33 -
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   53 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 .../near/GridCacheExNearFullApiSelfTest.java    |   39 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 .../GridCachePartitionedEvictionSelfTest.java   |   11 +-
 .../GridCachePartitionedFullApiSelfTest.java    |   32 +
 ...ePartitionedMultiThreadedPutGetSelfTest.java |    6 +-
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |    2 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheExReplicatedFullApiSelfTest.java   |   33 -
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 .../cache/eviction/EvictionAbstractTest.java    | 1056 ++++
 .../GridCacheBatchEvictUnswapSelfTest.java      |    5 +-
 ...heConcurrentEvictionConsistencySelfTest.java |   82 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |   29 +-
 .../GridCacheDistributedEvictionsSelfTest.java  |    5 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |   11 +-
 .../eviction/GridCacheEvictionAbstractTest.java |  484 --
 .../GridCacheEvictionTouchSelfTest.java         |   22 +-
 .../cache/eviction/GridCacheMockEntry.java      |    5 +
 .../fifo/FifoEvictionPolicySelfTest.java        |  262 +
 ...ridCacheFifoBatchEvictionPolicySelfTest.java |  384 --
 .../GridCacheFifoEvictionPolicySelfTest.java    |  372 --
 .../lru/GridCacheLruEvictionPolicySelfTest.java |  417 --
 .../GridCacheLruNearEvictionPolicySelfTest.java |  136 -
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |  171 -
 .../eviction/lru/LruEvictionPolicySelfTest.java |  353 ++
 .../lru/LruNearEvictionPolicySelfTest.java      |  140 +
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |  172 +
 .../GridCacheRandomEvictionPolicySelfTest.java  |  258 -
 .../RandomEvictionPolicyCacheSizeSelfTest.java  |    6 +
 .../random/RandomEvictionPolicySelfTest.java    |  357 ++
 ...dCacheSortedBatchEvictionPolicySelfTest.java |  385 --
 ...acheSortedEvictionPolicyPerformanceTest.java |  135 -
 .../GridCacheSortedEvictionPolicySelfTest.java  |  373 --
 .../SortedEvictionPolicyPerformanceTest.java    |  134 +
 .../sorted/SortedEvictionPolicySelfTest.java    |  266 +
 .../IgniteCacheClientNearCacheExpiryTest.java   |  103 +
 .../IgniteCacheExpiryPolicyTestSuite.java       |    2 +
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java |  412 ++
 .../local/GridCacheExLocalFullApiSelfTest.java  |   30 -
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   96 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../DataStreamerMultiThreadedSelfTest.java      |   59 +-
 .../DataStreamerMultinodeCreateCacheTest.java   |   97 +
 .../igfs/IgfsClientCacheSelfTest.java           |   12 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |   16 +-
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../nio/GridNioDelimitedBufferSelfTest.java     |  112 +
 .../util/nio/GridNioDelimitedBufferTest.java    |  112 -
 .../internal/util/nio/GridNioSelfTest.java      |    2 +-
 .../loadtests/GridCacheMultiNodeLoadTest.java   |    5 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |    6 +-
 .../loadtests/hashmap/GridCacheTestContext.java |    4 +-
 .../swap/GridSwapEvictAllBenchmark.java         |    6 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../IgniteMessagingWithClientTest.java          |  166 +
 .../GridTcpCommunicationSpiAbstractTest.java    |    4 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |    2 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    2 -
 ...cpCommunicationSpiMultithreadedSelfTest.java |    2 +-
 .../discovery/AbstractDiscoverySelfTest.java    |   21 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiMulticastTest.java |  129 +
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1196 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../stream/socket/SocketStreamerSelfTest.java   |   29 +-
 .../testframework/GridSpiTestContext.java       |   35 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/cache/TestCacheSession.java          |   18 +
 .../cache/TestThreadLocalCacheSession.java      |   15 +
 .../junits/common/GridCommonAbstractTest.java   |   83 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    4 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheEvictionSelfTestSuite.java       |   14 +-
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |    6 -
 .../IgniteCacheMetricsSelfTestSuite.java        |    1 +
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |   41 -
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    5 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    8 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    4 +-
 .../testsuites/IgniteStreamSelfTestSuite.java   |   39 +
 .../testsuites/IgniteStreamTestSuite.java       |   39 -
 .../testsuites/IgniteUtilSelfTestSuite.java     |    2 +-
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |   43 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   14 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  165 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  137 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   32 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |    2 +-
 .../internal/processors/hadoop/HadoopUtils.java |  288 +-
 .../hadoop/SecondaryFileSystemProvider.java     |   57 +-
 .../hadoop/fs/HadoopDistributedFileSystem.java  |   91 -
 .../hadoop/fs/HadoopFileSystemsUtils.java       |   17 -
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |  204 +
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |    6 +
 .../hadoop/igfs/HadoopIgfsInProc.java           |  170 +-
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |    2 +-
 .../hadoop/igfs/HadoopIgfsOutProc.java          |   33 +-
 .../hadoop/igfs/HadoopIgfsWrapper.java          |   19 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |   20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |   31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |   26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |   52 +-
 .../hadoop/HadoopClientProtocolSelfTest.java    |    6 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   56 +-
 ...oopSecondaryFileSystemConfigurationTest.java |    4 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   63 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |    2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |    2 +
 .../hadoop/HadoopAbstractSelfTest.java          |   14 +-
 .../hadoop/HadoopCommandLineTest.java           |   14 +-
 .../hadoop/HadoopFileSystemsTest.java           |   23 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |  166 +-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |    2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |   15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |    5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |    5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |    6 +-
 .../collections/HadoopAbstractMapTest.java      |   12 +
 .../collections/HadoopSkipListSelfTest.java     |    4 +-
 .../CacheHibernateStoreSessionListener.java     |  216 +
 ...heHibernateStoreSessionListenerSelfTest.java |  228 +
 .../testsuites/IgniteHibernateTestSuite.java    |    2 +
 .../GridCacheOffheapIndexEntryEvictTest.java    |  200 +
 .../cache/GridCacheOffheapIndexGetSelfTest.java |   80 +-
 .../cache/GridCacheQueryMetricsSelfTest.java    |   84 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    6 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...acheConfigurationPrimitiveTypesSelfTest.java |  104 +
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 ...QueryMultiThreadedOffHeapTieredSelfTest.java |   37 +
 ...eQueryMultiThreadedOffHeapTiredSelfTest.java |   37 -
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   11 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |    6 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    4 +-
 .../IgniteCacheWithIndexingTestSuite.java       |    3 +
 modules/mesos/pom.xml                           |    8 +-
 modules/spring/pom.xml                          |   14 +
 .../spring/CacheSpringStoreSessionListener.java |  207 +
 ...CacheSpringStoreSessionListenerSelfTest.java |  197 +
 .../testsuites/IgniteSpringTestSuite.java       |    3 +
 .../commands/cache/VisorCacheScanCommand.scala  |    2 +-
 scripts/git-apply-patch.sh                      |    8 +-
 scripts/git-format-patch.sh                     |   16 +-
 scripts/git-patch-functions.sh                  |   36 +-
 449 files changed, 30906 insertions(+), 16883 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b5b90ea/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
----------------------------------------------------------------------
diff --cc modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
index 8149297,a1ef7ba..66c14b5
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
@@@ -110,14 -178,20 +183,20 @@@ public class HadoopMapReduceTest extend
  
              checkJobStatistics(jobId);
  
+             final String outFile = PATH_OUTPUT + "/" + (useNewReducer ? "part-r-" : "part-") + "00000";
+ 
+             checkOwner(new IgfsPath(PATH_OUTPUT + "/" + "_SUCCESS"));
+ 
+             checkOwner(new IgfsPath(outFile));
+ 
              assertEquals("Use new mapper: " + useNewMapper + ", new combiner: " + useNewCombiner + ", new reducer: " +
 -                    useNewReducer,
 -                "blue\t200000\n" +
 -                    "green\t150000\n" +
 -                    "red\t100000\n" +
 -                    "yellow\t70000\n",
 +                useNewReducer,
 +                "blue\t" + blue + "\n" +
 +                "green\t" + green + "\n" +
 +                "red\t" + red + "\n" +
 +                "yellow\t" + yellow + "\n",
-                 readAndSortFile(PATH_OUTPUT + "/" + (useNewReducer ? "part-r-" : "part-") + "00000")
+                 readAndSortFile(outFile)
 -                );
 +            );
          }
      }
  


[44/50] incubator-ignite git commit: # ignite-999 minor

Posted by sb...@apache.org.
# ignite-999 minor


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

Branch: refs/heads/ignite-633
Commit: d4f7a9af7dc63263cfb0cda01ba2de7cb30065ef
Parents: 80c6cf0
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 9 13:47:35 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 9 13:47:35 2015 +0300

----------------------------------------------------------------------
 .../datastreamer/DataStreamerMultinodeCreateCacheTest.java         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4f7a9af/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
index d258a33..12b6458 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
@@ -74,7 +74,7 @@ public class DataStreamerMultinodeCreateCacheTest extends GridCommonAbstractTest
                 int iter = 0;
 
                 while (System.currentTimeMillis() < stopTime) {
-                    String cacheName = "cache-" + threadIdx + "-" + iter;
+                    String cacheName = "cache-" + threadIdx + "-" + (iter % 10);
 
                     try (IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheName)) {
                         try (IgniteDataStreamer<Object, Object> stmr = ignite.dataStreamer(cacheName)) {


[11/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5


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

Branch: refs/heads/ignite-633
Commit: e87969c1023786e8b5fb7d19057bf715f4d3ecac
Parents: dc1d427 ead66e7
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 5 11:34:48 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 5 11:34:48 2015 +0300

----------------------------------------------------------------------
 .../processors/hadoop/HadoopTaskContext.java    |  14 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   2 +-
 .../ignite/internal/visor/cache/VisorCache.java |   2 +-
 .../VisorCacheConfigurationCollectorJob.java    |   6 +-
 .../internal/visor/cache/VisorCacheMetrics.java |  19 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |  10 +-
 .../cache/VisorCacheStoreConfiguration.java     |   5 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |  14 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  70 ++---
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   2 +-
 .../internal/processors/hadoop/HadoopUtils.java | 282 ++++++++++++++++++-
 .../hadoop/SecondaryFileSystemProvider.java     |   4 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  48 +++-
 .../hadoop/HadoopClientProtocolSelfTest.java    |   6 +-
 .../hadoop/HadoopAbstractSelfTest.java          |  14 +-
 .../hadoop/HadoopCommandLineTest.java           |  14 +-
 .../processors/hadoop/HadoopMapReduceTest.java  | 176 +++++++++++-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |   2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |  15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |  12 +
 27 files changed, 666 insertions(+), 146 deletions(-)
----------------------------------------------------------------------



[13/50] incubator-ignite git commit: # ignite-sprint-5 disabled hanging test

Posted by sb...@apache.org.
# ignite-sprint-5 disabled hanging test


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

Branch: refs/heads/ignite-633
Commit: 5f06f578e57001e964d823daebc3bc67b1a27675
Parents: e87969c
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 5 17:21:23 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 5 17:21:23 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/messaging/IgniteMessagingWithClientTest.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f06f578/modules/core/src/test/java/org/apache/ignite/messaging/IgniteMessagingWithClientTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/messaging/IgniteMessagingWithClientTest.java b/modules/core/src/test/java/org/apache/ignite/messaging/IgniteMessagingWithClientTest.java
index 855a4f7..09abcdb 100644
--- a/modules/core/src/test/java/org/apache/ignite/messaging/IgniteMessagingWithClientTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/messaging/IgniteMessagingWithClientTest.java
@@ -77,6 +77,8 @@ public class IgniteMessagingWithClientTest extends GridCommonAbstractTest implem
      * @throws Exception If failed.
      */
     public void testMessageSendWithClientJoin() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-996");
+
         startGrid(0);
 
         Ignite ignite1 = startGrid(1);


[47/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-999' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-999' into ignite-sprint-5


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

Branch: refs/heads/ignite-633
Commit: ac908e9cdaccdcd55812360b3392eebb20bea62b
Parents: 91104a2 d4f7a9a
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 10 10:09:11 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 10 10:09:11 2015 +0300

----------------------------------------------------------------------
 .../discovery/DiscoveryCustomMessage.java       |  6 ++++
 .../discovery/GridDiscoveryManager.java         | 32 ++++++++++++++++++++
 .../cache/DynamicCacheChangeBatch.java          | 19 +++++++++---
 .../continuous/AbstractContinuousMessage.java   |  9 ++++++
 .../DataStreamerMultinodeCreateCacheTest.java   |  6 ++--
 5 files changed, 64 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[27/50] incubator-ignite git commit: # ignite-sprint-5 added test for ignite-999

Posted by sb...@apache.org.
# ignite-sprint-5 added test for ignite-999


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

Branch: refs/heads/ignite-633
Commit: 837462f5dade1930a8faaffb466c7bf05f9e06f9
Parents: a5b5ec7
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 14:55:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 14:55:20 2015 +0300

----------------------------------------------------------------------
 .../DataStreamerMultinodeCreateCacheTest.java   | 97 ++++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |  1 +
 2 files changed, 98 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/837462f5/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
new file mode 100644
index 0000000..2d19d6f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.datastreamer;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class DataStreamerMultinodeCreateCacheTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setSocketTimeout(5000);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setAckTimeout(5000);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCreateCacheAndStream() throws Exception {
+        final int THREADS = 5;
+
+        startGrids(THREADS);
+
+        final AtomicInteger idx = new AtomicInteger();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int threadIdx = idx.getAndIncrement();
+
+                long stopTime = System.currentTimeMillis() + 60_000;
+
+                Ignite ignite = grid(threadIdx);
+
+                int iter = 0;
+
+                while (System.currentTimeMillis() < stopTime) {
+                    String cacheName = "cache-" + threadIdx + "-" + iter;
+
+                    try (IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheName)) {
+                        try (IgniteDataStreamer<Object, Object> stmr = ignite.dataStreamer(cacheName)) {
+                            ((DataStreamerImpl<Object, Object>)stmr).maxRemapCount(0);
+
+                            for (int i = 0; i < 1000; i++)
+                                stmr.addData(i, i);
+                        }
+                    }
+
+                    iter++;
+                }
+
+                return null;
+            }
+        }, THREADS, "create-cache");
+
+        fut.get(2 * 60_000);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/837462f5/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 9634e9a..7e4409d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -111,6 +111,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheStoreValueBytesSelfTest.class);
         suite.addTestSuite(DataStreamProcessorSelfTest.class);
         suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);
+        suite.addTestSuite(DataStreamerMultinodeCreateCacheTest.class);
         suite.addTestSuite(DataStreamerImplSelfTest.class);
         suite.addTestSuite(GridCacheEntryMemorySizeSelfTest.class);
         suite.addTestSuite(GridCacheClearAllSelfTest.class);


[31/50] incubator-ignite git commit: # ignite-sprint-5 minor optimization in force keys future

Posted by sb...@apache.org.
# ignite-sprint-5 minor optimization in force keys future


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

Branch: refs/heads/ignite-633
Commit: 0fa2853e060fee7c9c8c8484be412e91d30c52da
Parents: ff7827e
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 16:31:31 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 16:31:31 2015 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtForceKeysFuture.java   | 40 +++++++++++++-------
 1 file changed, 26 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0fa2853e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index 9637fd1..1d57ef7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -208,21 +208,21 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
      * @return {@code True} if some mapping was added.
      */
     private boolean map(Iterable<KeyCacheObject> keys, Collection<ClusterNode> exc) {
-        Map<ClusterNode, Set<KeyCacheObject>> mappings = new HashMap<>();
-
-        ClusterNode loc = cctx.localNode();
-
-        int curTopVer = topCntr.get();
+        Map<ClusterNode, Set<KeyCacheObject>> mappings = null;
 
         for (KeyCacheObject key : keys)
-            map(key, mappings, exc);
+            mappings = map(key, mappings, exc);
 
         if (isDone())
             return false;
 
         boolean ret = false;
 
-        if (!mappings.isEmpty()) {
+        if (mappings != null) {
+            ClusterNode loc = cctx.localNode();
+
+            int curTopVer = topCntr.get();
+
             preloader.addFuture(this);
 
             trackable = true;
@@ -275,22 +275,27 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
      * @param key Key.
      * @param exc Exclude nodes.
      * @param mappings Mappings.
+     * @return Mappings.
      */
-    private void map(KeyCacheObject key, Map<ClusterNode, Set<KeyCacheObject>> mappings, Collection<ClusterNode> exc) {
+    private Map<ClusterNode, Set<KeyCacheObject>> map(KeyCacheObject key,
+        @Nullable Map<ClusterNode, Set<KeyCacheObject>> mappings,
+        Collection<ClusterNode> exc)
+    {
         ClusterNode loc = cctx.localNode();
 
-        int part = cctx.affinity().partition(key);
-
         GridCacheEntryEx e = cctx.dht().peekEx(key);
 
         try {
             if (e != null && !e.isNewLocked()) {
-                if (log.isDebugEnabled())
+                if (log.isDebugEnabled()) {
+                    int part = cctx.affinity().partition(key);
+
                     log.debug("Will not rebalance key (entry is not new) [cacheName=" + cctx.name() +
                         ", key=" + key + ", part=" + part + ", locId=" + cctx.nodeId() + ']');
+                }
 
                 // Key has been rebalanced or retrieved already.
-                return;
+                return mappings;
             }
         }
         catch (GridCacheEntryRemovedException ignore) {
@@ -299,6 +304,8 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
                     ", locId=" + cctx.nodeId() + ']');
         }
 
+        int part = cctx.affinity().partition(key);
+
         List<ClusterNode> owners = F.isEmpty(exc) ? top.owners(part, topVer) :
             new ArrayList<>(F.view(top.owners(part, topVer), F.notIn(exc)));
 
@@ -308,7 +315,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
                     "topVer=" + topVer + ", locId=" + cctx.nodeId() + ']');
 
             // Key is already rebalanced.
-            return;
+            return mappings;
         }
 
         // Create partition.
@@ -337,9 +344,12 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
                     log.debug("Will not rebalance key (no nodes to request from with rebalancing disabled) [key=" +
                         key + ", part=" + part + ", locId=" + cctx.nodeId() + ']');
 
-                return;
+                return mappings;
             }
 
+            if (mappings == null)
+                mappings = U.newHashMap(keys.size());
+
             Collection<KeyCacheObject> mappedKeys = F.addIfAbsent(mappings, pick, F.<KeyCacheObject>newSet());
 
             assert mappedKeys != null;
@@ -357,6 +367,8 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
                 log.debug("Will not rebalance key (local partition is not MOVING) [cacheName=" + cctx.name() +
                     ", key=" + key + ", part=" + locPart + ", locId=" + cctx.nodeId() + ']');
         }
+
+        return mappings;
     }
 
     /**


[35/50] incubator-ignite git commit: # IGNITE-992 Review.

Posted by sb...@apache.org.
# IGNITE-992 Review.


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

Branch: refs/heads/ignite-633
Commit: 8740b6e76107eeb2885e9454f670044edf11e619
Parents: e934bca
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Jun 9 12:20:23 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Jun 9 12:20:23 2015 +0700

----------------------------------------------------------------------
 .../internal/util/IgniteExceptionRegistry.java     |  7 +++----
 .../visor/node/VisorNodeSuppressedErrorsTask.java  | 12 +++++++++++-
 .../internal/visor/util/VisorExceptionWrapper.java | 17 ++++++++++-------
 3 files changed, 24 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8740b6e7/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
index 8ad3348..ab113d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.util;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.internal.visor.util.*;
 
 import java.io.*;
 import java.util.*;
@@ -161,7 +160,7 @@ public class IgniteExceptionRegistry {
 
         /** */
         @GridToStringExclude
-        private final VisorExceptionWrapper error;
+        private final Throwable error;
 
         /** */
         private final long threadId;
@@ -187,7 +186,7 @@ public class IgniteExceptionRegistry {
          */
         public ExceptionInfo(long order, Throwable error, String msg, long threadId, String threadName, long time) {
             this.order = order;
-            this.error = new VisorExceptionWrapper(error);
+            this.error = error;
             this.threadId = threadId;
             this.threadName = threadName;
             this.time = time;
@@ -211,7 +210,7 @@ public class IgniteExceptionRegistry {
         /**
          * @return Suppressed error.
          */
-        public VisorExceptionWrapper error() {
+        public Throwable error() {
             return error;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8740b6e7/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeSuppressedErrorsTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeSuppressedErrorsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeSuppressedErrorsTask.java
index 8b39d09..9fc1cc4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeSuppressedErrorsTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeSuppressedErrorsTask.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.visor.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
@@ -83,12 +84,21 @@ public class VisorNodeSuppressedErrorsTask extends VisorMultiNodeTask<Map<UUID,
 
             List<IgniteExceptionRegistry.ExceptionInfo> errors = ignite.context().exceptionRegistry().getErrors(order);
 
+            List<IgniteExceptionRegistry.ExceptionInfo> wrapped = new ArrayList<>(errors.size());
+
             for (IgniteExceptionRegistry.ExceptionInfo error : errors) {
                 if (error.order() > order)
                     order = error.order();
+
+                wrapped.add(new IgniteExceptionRegistry.ExceptionInfo(error.order(),
+                    new VisorExceptionWrapper(error.error()),
+                    error.message(),
+                    error.threadId(),
+                    error.threadName(),
+                    error.time()));
             }
 
-            return new IgniteBiTuple<>(order, errors);
+            return new IgniteBiTuple<>(order, wrapped);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8740b6e7/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
index e253dcf..d2ae0e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
@@ -21,14 +21,17 @@ package org.apache.ignite.internal.visor.util;
  * Exception wrapper for safe for transferring to Visor.
  */
 public class VisorExceptionWrapper extends Throwable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** Detail message string of this throwable */
     private String detailMsg;
 
     /** Simple class name of base throwable object. */
-    private String classSimpleName;
+    private String clsSimpleName;
 
     /** Class name of base throwable object. */
-    private String className;
+    private String clsName;
 
     /**
      * Wrap throwable by presented on Visor throwable object.
@@ -38,8 +41,8 @@ public class VisorExceptionWrapper extends Throwable {
     public VisorExceptionWrapper(Throwable cause) {
         assert cause != null;
 
-        classSimpleName = cause.getClass().getSimpleName();
-        className = cause.getClass().getName();
+        clsSimpleName = cause.getClass().getSimpleName();
+        clsName = cause.getClass().getName();
 
         detailMsg = cause.getMessage();
 
@@ -56,14 +59,14 @@ public class VisorExceptionWrapper extends Throwable {
      * @return Class simple name of base throwable object.
      */
     public String getClassSimpleName() {
-        return classSimpleName;
+        return clsSimpleName;
     }
 
     /**
      * @return Class name of base throwable object.
      */
     public String getClassName() {
-        return className;
+        return clsName;
     }
 
     /** {@inheritDoc} */
@@ -73,6 +76,6 @@ public class VisorExceptionWrapper extends Throwable {
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return (detailMsg != null) ? (className + ": " + detailMsg) : className;
+        return (detailMsg != null) ? (clsName + ": " + detailMsg) : clsName;
     }
 }


[38/50] incubator-ignite git commit: ignite-999 ignore duplicated discovery custom messages

Posted by sb...@apache.org.
ignite-999 ignore duplicated discovery custom messages


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

Branch: refs/heads/ignite-633
Commit: 80c6cf0b3d763e3fcfab37653de3f2c6dddf30e8
Parents: 14bb076
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 9 11:14:09 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 9 11:14:09 2015 +0300

----------------------------------------------------------------------
 .../discovery/DiscoveryCustomMessage.java       |  6 ++++
 .../discovery/GridDiscoveryManager.java         | 32 ++++++++++++++++++++
 .../cache/DynamicCacheChangeBatch.java          | 19 +++++++++---
 .../continuous/AbstractContinuousMessage.java   |  9 ++++++
 .../DataStreamerMultinodeCreateCacheTest.java   |  4 +--
 5 files changed, 63 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/80c6cf0b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
index 693bbef..401486d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.managers.discovery;
 
 import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -27,6 +28,11 @@ import java.io.*;
  */
 public interface DiscoveryCustomMessage extends Serializable {
     /**
+     * @return Unique custom message ID.
+     */
+    public IgniteUuid id();
+
+    /**
      * Whether or not minor version of topology should be increased on message receive.
      *
      * @return {@code true} if minor topology version should be increased.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/80c6cf0b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 142dbaa..71fbc61 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -178,6 +178,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
 
+    /** Received custom messages history. */
+    private final ArrayDeque<IgniteUuid> rcvdCustomMsgs = new ArrayDeque<>();
+
     /** @param ctx Context. */
     public GridDiscoveryManager(GridKernalContext ctx) {
         super(ctx, ctx.config().getDiscoverySpi());
@@ -359,6 +362,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 DiscoveryCustomMessage customMsg = spiCustomMsg == null ? null
                     : ((CustomMessageWrapper)spiCustomMsg).delegate();
 
+                if (skipMessage(type, customMsg))
+                    return;
+
                 final ClusterNode locNode = localNode();
 
                 if (snapshots != null)
@@ -515,6 +521,32 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
+     * @param type Message type.
+     * @param customMsg Custom message.
+     * @return {@code True} if should not process message.
+     */
+    private boolean skipMessage(int type, @Nullable DiscoveryCustomMessage customMsg) {
+        if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
+            assert customMsg != null && customMsg.id() != null : customMsg;
+
+            if (rcvdCustomMsgs.contains(customMsg.id())) {
+                if (log.isDebugEnabled())
+                    log.debug("Received duplicated custom message, will ignore [msg=" + customMsg + "]");
+
+                return true;
+            }
+
+            rcvdCustomMsgs.addLast(customMsg.id());
+
+            while (rcvdCustomMsgs.size() > DISCOVERY_HISTORY_SIZE)
+                rcvdCustomMsgs.pollFirst();
+        }
+
+        return false;
+    }
+
+    /**
+     * @param msgCls Message class.
      * @param lsnr Custom event listener.
      */
     public <T extends DiscoveryCustomMessage> void setCustomEventListener(Class<T> msgCls, CustomEventListener<T> lsnr) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/80c6cf0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index 5fcd0e2..dfc39c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -39,6 +40,9 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     @GridToStringInclude
     private Map<String, Map<UUID, Boolean>> clientNodes;
 
+    /** Custom message ID. */
+    private IgniteUuid id = IgniteUuid.randomUuid();
+
     /**
      * @param reqs Requests.
      */
@@ -48,6 +52,11 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
         this.reqs = reqs;
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return id;
+    }
+
     /**
      * @return Collection of change requests.
      */
@@ -70,11 +79,6 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     }
 
     /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(DynamicCacheChangeBatch.class, this);
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean incrementMinorTopologyVersion() {
         return true;
     }
@@ -88,4 +92,9 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     @Override public boolean isMutable() {
         return false;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(DynamicCacheChangeBatch.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/80c6cf0b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
index f375777..91768a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.continuous;
 
 import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.lang.*;
 
 import java.util.*;
 
@@ -28,6 +29,9 @@ public abstract class AbstractContinuousMessage implements DiscoveryCustomMessag
     /** Routine ID. */
     protected final UUID routineId;
 
+    /** Custom message ID. */
+    private final IgniteUuid id = IgniteUuid.randomUuid();
+
     /**
      * @param id Id.
      */
@@ -35,6 +39,11 @@ public abstract class AbstractContinuousMessage implements DiscoveryCustomMessag
         routineId = id;
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return id;
+    }
+
     /**
      * @return Routine ID.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/80c6cf0b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
index 2d19d6f..d258a33 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
@@ -42,8 +42,8 @@ public class DataStreamerMultinodeCreateCacheTest extends GridCommonAbstractTest
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
 
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setSocketTimeout(5000);
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setAckTimeout(5000);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setSocketTimeout(50);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setAckTimeout(50);
 
         return cfg;
     }


[09/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-883_1

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-883_1


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

Branch: refs/heads/ignite-633
Commit: db576526600b965d14fe271ac4624401a5b4c239
Parents: a676152 ead66e7
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 5 10:02:10 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 5 10:02:10 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/visor/cache/VisorCache.java  |  2 +-
 .../VisorCacheConfigurationCollectorJob.java     |  6 +++---
 .../internal/visor/cache/VisorCacheMetrics.java  | 19 +++++++++++--------
 .../cache/VisorCacheMetricsCollectorTask.java    | 10 ++++++----
 .../cache/VisorCacheStoreConfiguration.java      |  5 ++---
 5 files changed, 23 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[17/50] incubator-ignite git commit: Merge branches 'ignite-941' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-941

Posted by sb...@apache.org.
Merge branches 'ignite-941' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-941


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

Branch: refs/heads/ignite-633
Commit: 59db4a5e0f1dd83d56cb95f0a4184451f0a387fd
Parents: b4d1915 ab20fd9
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri Jun 5 13:55:21 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri Jun 5 13:55:21 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |  11 ++
 .../affinity/GridAffinityProcessor.java         |  23 ++++-
 .../processors/cache/GridCacheAdapter.java      |  17 ++-
 .../processors/cache/GridCacheProxyImpl.java    |  14 ++-
 .../processors/cache/IgniteCacheProxy.java      |  23 +++++
 .../processors/cache/IgniteInternalCache.java   |  11 +-
 .../datastreamer/DataStreamerImpl.java          |  92 ++++++++++++-----
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../ignite/internal/visor/cache/VisorCache.java |   2 +-
 .../VisorCacheConfigurationCollectorJob.java    |   6 +-
 .../internal/visor/cache/VisorCacheMetrics.java |  19 ++--
 .../cache/VisorCacheMetricsCollectorTask.java   |  10 +-
 .../cache/VisorCacheStoreConfiguration.java     |   5 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  83 +++++++++++++++
 .../GridCacheExAbstractFullApiSelfTest.java     | 103 -------------------
 .../GridCacheExColocatedFullApiSelfTest.java    |  33 ------
 .../near/GridCacheExNearFullApiSelfTest.java    |  39 -------
 .../GridCacheExReplicatedFullApiSelfTest.java   |  33 ------
 .../local/GridCacheExLocalFullApiSelfTest.java  |  30 ------
 .../DataStreamerMultiThreadedSelfTest.java      |  59 ++++++++---
 .../IgniteMessagingWithClientTest.java          |   2 +
 .../IgniteCacheFullApiSelfTestSuite.java        |   6 --
 22 files changed, 309 insertions(+), 314 deletions(-)
----------------------------------------------------------------------



[28/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5


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

Branch: refs/heads/ignite-633
Commit: 2a8e2abe4081334802f9950757b7d3776a3a6179
Parents: 837462f 015afdb
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 14:55:48 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 14:55:48 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |  26 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   8 +-
 .../internal/MarshallerContextAdapter.java      |  36 +-
 .../internal/managers/GridManagerAdapter.java   |   9 +
 .../checkpoint/GridCheckpointManager.java       |  52 +-
 .../discovery/GridDiscoveryManager.java         |  28 +-
 .../processors/cache/GridCacheAdapter.java      |   4 +
 .../GridCachePartitionExchangeManager.java      |  26 +-
 .../processors/cache/GridCacheTtlManager.java   |   9 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  18 +-
 .../GridDhtPartitionsExchangeFuture.java        |  50 +-
 .../cache/transactions/IgniteTxManager.java     |   3 -
 .../datastructures/DataStructuresProcessor.java | 107 +++-
 .../service/GridServiceProcessor.java           |   4 +-
 .../timeout/GridSpiTimeoutObject.java           |  73 +++
 .../timeout/GridTimeoutProcessor.java           | 105 +++-
 .../util/nio/GridCommunicationClient.java       |  30 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |  13 +-
 .../util/nio/GridTcpCommunicationClient.java    | 554 -------------------
 .../util/nio/GridTcpNioCommunicationClient.java |   8 -
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  27 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  10 +
 .../ignite/spi/IgniteSpiTimeoutObject.java      |  44 ++
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |   3 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 438 ++++-----------
 .../tcp/TcpCommunicationSpiMBean.java           |   2 -
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   3 -
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  10 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 156 +-----
 .../IgniteCountDownLatchAbstractSelfTest.java   | 102 ++++
 .../IgniteCacheClientNearCacheExpiryTest.java   | 103 ++++
 .../IgniteCacheExpiryPolicyTestSuite.java       |   2 +
 .../internal/util/nio/GridNioSelfTest.java      |   2 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |   4 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   2 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |   2 -
 ...cpCommunicationSpiMultithreadedSelfTest.java |   2 +-
 .../discovery/AbstractDiscoverySelfTest.java    |  13 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  25 +
 .../testframework/GridSpiTestContext.java       |  10 +
 40 files changed, 920 insertions(+), 1203 deletions(-)
----------------------------------------------------------------------



[14/50] incubator-ignite git commit: # ignite-883

Posted by sb...@apache.org.
# ignite-883


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

Branch: refs/heads/ignite-633
Commit: e237b00d84039383723e00929b639fa62be03de7
Parents: fb827a77
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 5 16:12:46 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 5 18:10:34 2015 +0300

----------------------------------------------------------------------
 .../util/nio/GridNioRecoveryDescriptor.java     | 11 ++++++---
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  9 ++++---
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 25 ++++++++++++++++++++
 3 files changed, 37 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e237b00d/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
index 08a9937..3bc81ea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
@@ -176,9 +176,11 @@ public class GridNioRecoveryDescriptor {
         while (acked < rcvCnt) {
             GridNioFuture<?> fut = msgFuts.pollFirst();
 
-            assert fut != null;
+            assert fut != null : "Missed message future [rcvCnt=" + rcvCnt +
+                ", acked=" + acked +
+                ", desc=" + this + ']';
 
-            assert fut.isDone();
+            assert fut.isDone() : fut;
 
             acked++;
         }
@@ -239,7 +241,10 @@ public class GridNioRecoveryDescriptor {
      * @param rcvCnt Number of messages received by remote node.
      */
     public void onHandshake(long rcvCnt) {
-        ackReceived(rcvCnt);
+        synchronized (this) {
+            if (!nodeLeft)
+                ackReceived(rcvCnt);
+        }
 
         resendCnt = msgFuts.size();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e237b00d/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 302b721..0270a7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -1752,7 +1752,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         @Nullable Collection<TcpDiscoveryAbstractMessage> messages(IgniteUuid lastMsgId) {
             assert lastMsgId != null;
 
-            Collection<TcpDiscoveryAbstractMessage> copy = new ArrayList<>(msgs.size());
+            Collection<TcpDiscoveryAbstractMessage> cp = new ArrayList<>(msgs.size());
 
             boolean skip = true;
 
@@ -1762,10 +1762,10 @@ class ServerImpl extends TcpDiscoveryImpl {
                         skip = false;
                 }
                 else
-                    copy.add(msg);
+                    cp.add(msg);
             }
 
-            return !skip ? copy : null;
+            return !skip ? cp : null;
         }
 
         /**
@@ -2704,8 +2704,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 node.aliveCheck(spi.maxMissedClientHbs);
 
                 if (isLocalNodeCoordinator()) {
-                    Collection<TcpDiscoveryAbstractMessage> pending =
-                        pendingMsgs.messages(msg.lastMessageId());
+                    Collection<TcpDiscoveryAbstractMessage> pending = pendingMsgs.messages(msg.lastMessageId());
 
                     if (pending != null) {
                         msg.pendingMessages(pending);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e237b00d/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
index f7be340..7333020 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -469,6 +469,31 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testClientReconnectOneServerOneClient() throws Exception {
+        clientsPerSrv = 1;
+
+        startServerNodes(1);
+        startClientNodes(1);
+
+        checkNodes(1, 1);
+
+        srvLeftLatch = new CountDownLatch(1);
+        srvFailedLatch = new CountDownLatch(1);
+
+        attachListeners(1, 0);
+
+        ((TcpDiscoverySpi)G.ignite("client-0").configuration().getDiscoverySpi()).brakeConnection();
+
+        assertFalse(srvFailedLatch.await(2000, TimeUnit.MILLISECONDS));
+
+        assertEquals(1L, srvLeftLatch.getCount());
+
+        checkNodes(1, 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testGetMissedMessagesOnReconnect() throws Exception {
         clientsPerSrv = 1;
 


[20/50] incubator-ignite git commit: # ignite-883 minor

Posted by sb...@apache.org.
# ignite-883 minor


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

Branch: refs/heads/ignite-633
Commit: 730ef104ad8938c88793c374abd2396cf8899f17
Parents: 0a4e7dd
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 10:19:46 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 10:19:46 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/nio/GridNioRecoveryDescriptor.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/730ef104/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
index 3bc81ea..733ae81 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
@@ -244,9 +244,9 @@ public class GridNioRecoveryDescriptor {
         synchronized (this) {
             if (!nodeLeft)
                 ackReceived(rcvCnt);
-        }
 
-        resendCnt = msgFuts.size();
+            resendCnt = msgFuts.size();
+        }
     }
 
     /**


[10/50] incubator-ignite git commit: # ignite-sprint-5 use consistent topology version in streamer

Posted by sb...@apache.org.
# ignite-sprint-5 use consistent topology version in streamer


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

Branch: refs/heads/ignite-633
Commit: dc1d427fa2f5235a5779058a444e3845946a7e07
Parents: 20e5677
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 5 09:54:37 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 5 11:21:36 2015 +0300

----------------------------------------------------------------------
 .../affinity/GridAffinityProcessor.java         | 23 ++++-
 .../datastreamer/DataStreamerImpl.java          | 92 ++++++++++++++------
 .../DataStreamerMultiThreadedSelfTest.java      | 59 +++++++++----
 3 files changed, 129 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc1d427f/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
index daa2bc2..aac63c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
@@ -164,14 +164,17 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
      *
      * @param cacheName Cache name.
      * @param key Key to map.
+     * @param topVer Topology version.
      * @return Affinity nodes, primary first.
      * @throws IgniteCheckedException If failed.
      */
-    public <K> List<ClusterNode> mapKeyToPrimaryAndBackups(@Nullable String cacheName, K key) throws IgniteCheckedException {
+    public <K> List<ClusterNode> mapKeyToPrimaryAndBackups(@Nullable String cacheName,
+        K key,
+        AffinityTopologyVersion topVer)
+        throws IgniteCheckedException
+    {
         A.notNull(key, "key");
 
-        AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
-
         AffinityInfo affInfo = affinityCache(cacheName, topVer);
 
         if (affInfo == null)
@@ -181,6 +184,20 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Map single key to primary and backup nodes.
+     *
+     * @param cacheName Cache name.
+     * @param key Key to map.
+     * @return Affinity nodes, primary first.
+     * @throws IgniteCheckedException If failed.
+     */
+    public <K> List<ClusterNode> mapKeyToPrimaryAndBackups(@Nullable String cacheName, K key)
+        throws IgniteCheckedException
+    {
+        return mapKeyToPrimaryAndBackups(cacheName, key, ctx.discovery().topologyVersionEx());
+    }
+
+    /**
      * Gets affinity key for cache key.
      *
      * @param cacheName Cache name.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc1d427f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index d16167a..ed8e573 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -198,19 +198,14 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                 // Remap regular mappings.
                 final Buffer buf = bufMappings.remove(id);
 
+                // Only async notification is possible since
+                // discovery thread may be trapped otherwise.
                 if (buf != null) {
-                    // Only async notification is possible since
-                    // discovery thread may be trapped otherwise.
-                    ctx.closure().callLocalSafe(
-                        new Callable<Object>() {
-                            @Override public Object call() throws Exception {
-                                buf.onNodeLeft();
-
-                                return null;
-                            }
-                        },
-                        true /* system pool */
-                    );
+                    waitAffinityAndRun(new Runnable() {
+                        @Override public void run() {
+                            buf.onNodeLeft();
+                        }
+                    }, discoEvt.topologyVersion(), true);
                 }
             }
         };
@@ -248,6 +243,31 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     }
 
     /**
+     * @param c Closure to run.
+     * @param topVer Topology version to wait for.
+     * @param async Async flag.
+     */
+    private void waitAffinityAndRun(final Runnable c, long topVer, boolean async) {
+        AffinityTopologyVersion topVer0 = new AffinityTopologyVersion(topVer, 0);
+
+        IgniteInternalFuture<?> fut = ctx.cache().context().exchange().affinityReadyFuture(topVer0);
+
+        if (fut != null && !fut.isDone()) {
+            fut.listen(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> fut) {
+                    ctx.closure().runLocalSafe(c, true);
+                }
+            });
+        }
+        else {
+            if (async)
+                ctx.closure().runLocalSafe(c, true);
+            else
+                c.run();
+        }
+    }
+
+    /**
      * @return Cache object context.
      */
     public CacheObjectContext cacheObjectContext() {
@@ -527,6 +547,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
         boolean initPda = ctx.deploy().enabled() && jobPda == null;
 
+        AffinityTopologyVersion topVer = ctx.cache().context().exchange().readyAffinityVersion();
+
         for (DataStreamerEntry entry : entries) {
             List<ClusterNode> nodes;
 
@@ -543,7 +565,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     initPda = false;
                 }
 
-                nodes = nodes(key);
+                nodes = nodes(key, topVer);
             }
             catch (IgniteCheckedException e) {
                 resFut.onDone(e);
@@ -621,10 +643,10 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                 }
             };
 
-            GridFutureAdapter<?> f;
+            final GridFutureAdapter<?> f;
 
             try {
-                f = buf.update(entriesForNode, lsnr);
+                f = buf.update(entriesForNode, topVer, lsnr);
             }
             catch (IgniteInterruptedCheckedException e1) {
                 resFut.onDone(e1);
@@ -633,30 +655,38 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             }
 
             if (ctx.discovery().node(nodeId) == null) {
-                if (bufMappings.remove(nodeId, buf))
-                    buf.onNodeLeft();
+                if (bufMappings.remove(nodeId, buf)) {
+                    final Buffer buf0 = buf;
 
-                if (f != null)
-                    f.onDone(new ClusterTopologyCheckedException("Failed to wait for request completion " +
-                        "(node has left): " + nodeId));
+                    waitAffinityAndRun(new Runnable() {
+                        @Override public void run() {
+                            buf0.onNodeLeft();
+
+                            if (f != null)
+                                f.onDone(new ClusterTopologyCheckedException("Failed to wait for request completion " +
+                                    "(node has left): " + nodeId));
+                        }
+                    }, ctx.discovery().topologyVersion(), false);
+                }
             }
         }
     }
 
     /**
      * @param key Key to map.
+     * @param topVer Topology version.
      * @return Nodes to send requests to.
      * @throws IgniteCheckedException If failed.
      */
-    private List<ClusterNode> nodes(KeyCacheObject key) throws IgniteCheckedException {
+    private List<ClusterNode> nodes(KeyCacheObject key, AffinityTopologyVersion topVer) throws IgniteCheckedException {
         GridAffinityProcessor aff = ctx.affinity();
 
         List<ClusterNode> res = null;
 
         if (!allowOverwrite())
-            res = aff.mapKeyToPrimaryAndBackups(cacheName, key);
+            res = aff.mapKeyToPrimaryAndBackups(cacheName, key, topVer);
         else {
-            ClusterNode node = aff.mapKeyToNode(cacheName, key);
+            ClusterNode node = aff.mapKeyToNode(cacheName, key, topVer);
 
             if (node != null)
                 res = Collections.singletonList(node);
@@ -959,11 +989,13 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
         /**
          * @param newEntries Infos.
+         * @param topVer Topology version.
          * @param lsnr Listener for the operation future.
          * @throws IgniteInterruptedCheckedException If failed.
          * @return Future for operation.
          */
         @Nullable GridFutureAdapter<?> update(Iterable<DataStreamerEntry> newEntries,
+            AffinityTopologyVersion topVer,
             IgniteInClosure<IgniteInternalFuture<?>> lsnr) throws IgniteInterruptedCheckedException {
             List<DataStreamerEntry> entries0 = null;
             GridFutureAdapter<Object> curFut0;
@@ -986,7 +1018,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             }
 
             if (entries0 != null) {
-                submit(entries0, curFut0);
+                submit(entries0, topVer, curFut0);
 
                 if (cancelled)
                     curFut0.onDone(new IgniteCheckedException("Data streamer has been cancelled: " + DataStreamerImpl.this));
@@ -1023,7 +1055,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             }
 
             if (entries0 != null)
-                submit(entries0, curFut0);
+                submit(entries0, null, curFut0);
 
             // Create compound future for this flush.
             GridCompoundFuture<Object, Object> res = null;
@@ -1068,10 +1100,13 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
         /**
          * @param entries Entries to submit.
+         * @param topVer Topology version.
          * @param curFut Current future.
          * @throws IgniteInterruptedCheckedException If interrupted.
          */
-        private void submit(final Collection<DataStreamerEntry> entries, final GridFutureAdapter<Object> curFut)
+        private void submit(final Collection<DataStreamerEntry> entries,
+            @Nullable AffinityTopologyVersion topVer,
+            final GridFutureAdapter<Object> curFut)
             throws IgniteInterruptedCheckedException {
             assert entries != null;
             assert !entries.isEmpty();
@@ -1160,6 +1195,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
                 reqs.put(reqId, (GridFutureAdapter<Object>)fut);
 
+                if (topVer == null)
+                    topVer = ctx.cache().context().exchange().readyAffinityVersion();
+
                 DataStreamerRequest req = new DataStreamerRequest(
                     reqId,
                     topicBytes,
@@ -1174,7 +1212,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     dep != null ? dep.participants() : null,
                     dep != null ? dep.classLoaderId() : null,
                     dep == null,
-                    ctx.cache().context().exchange().readyAffinityVersion());
+                    topVer);
 
                 try {
                     ctx.io().send(node, TOPIC_DATASTREAM, req, PUBLIC_POOL);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc1d427f/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
index 2382a66..e0092d4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
@@ -41,6 +41,9 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
+    /** */
+    private boolean dynamicCache;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -50,13 +53,22 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
 
+        if (!dynamicCache)
+            cfg.setCacheConfiguration(cacheConfiguration());
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration() {
         CacheConfiguration ccfg = defaultCacheConfiguration();
 
         ccfg.setCacheMode(PARTITIONED);
         ccfg.setBackups(1);
-        cfg.setCacheConfiguration(ccfg);
 
-        return cfg;
+        return ccfg;
     }
 
     /** {@inheritDoc} */
@@ -68,6 +80,22 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStartStopIgnites() throws Exception {
+        startStopIgnites();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStopIgnitesDynamicCache() throws Exception {
+        dynamicCache = true;
+
+        startStopIgnites();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void startStopIgnites() throws Exception {
         for (int attempt = 0; attempt < 3; ++attempt) {
             log.info("Iteration: " + attempt);
             
@@ -75,28 +103,29 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
 
             Set<IgniteFuture> futs = new HashSet<>();
 
-            IgniteInternalFuture<?> fut;
+            final AtomicInteger igniteId = new AtomicInteger(1);
 
-            try (final DataStreamerImpl dataLdr = (DataStreamerImpl)ignite.dataStreamer(null)) {
-                dataLdr.maxRemapCount(0);
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    for (int i = 1; i < 5; ++i)
+                        startGrid(igniteId.incrementAndGet());
 
-                final AtomicInteger igniteId = new AtomicInteger(1);
+                    return true;
+                }
+            }, 2, "start-node-thread");
 
-                fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        for (int i = 1; i < 5; ++i)
-                            startGrid(igniteId.incrementAndGet());
+            if (dynamicCache)
+                ignite.getOrCreateCache(cacheConfiguration());
 
-                        return true;
-                    }
-                }, 2, "start-node-thread");
+            try (final DataStreamerImpl dataLdr = (DataStreamerImpl)ignite.dataStreamer(null)) {
+                dataLdr.maxRemapCount(0);
 
-                Random random = new Random();
+                Random rnd = new Random();
 
                 long endTime = U.currentTimeMillis() + 15_000;
 
                 while (!fut.isDone() && U.currentTimeMillis() < endTime)
-                    futs.add(dataLdr.addData(random.nextInt(100_000), random.nextInt(100_000)));
+                    futs.add(dataLdr.addData(rnd.nextInt(100_000), String.valueOf(rnd.nextInt(100_000))));
             }
 
             for (IgniteFuture f : futs)


[32/50] incubator-ignite git commit: # IGNITE-992 Review.

Posted by sb...@apache.org.
# IGNITE-992 Review.


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

Branch: refs/heads/ignite-633
Commit: ea12580dabd60dc0c2b1a0e86d7112b409366ba7
Parents: 662f733
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Jun 9 10:15:41 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Jun 9 10:15:41 2015 +0700

----------------------------------------------------------------------
 .../apache/ignite/internal/util/IgniteExceptionRegistry.java | 2 +-
 .../internal/visor/node/VisorNodeDataCollectorTask.java      | 8 ++++----
 .../apache/ignite/internal/visor/query/VisorQueryJob.java    | 2 +-
 .../ignite/internal/visor/util/VisorExceptionWrapper.java    | 2 +-
 4 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ea12580d/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
index a56570a..8ad3348 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteExceptionRegistry.java
@@ -187,7 +187,7 @@ public class IgniteExceptionRegistry {
          */
         public ExceptionInfo(long order, Throwable error, String msg, long threadId, String threadName, long time) {
             this.order = order;
-            this.error = VisorTaskUtils.wrap(error);
+            this.error = new VisorExceptionWrapper(error);
             this.threadId = threadId;
             this.threadName = threadName;
             this.time = time;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ea12580d/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
index 7dbfd39..3b2d45c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
@@ -88,7 +88,7 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
                 else {
                     // Ignore nodes that left topology.
                     if (!(unhandledEx instanceof ClusterGroupEmptyException))
-                        taskRes.unhandledEx().put(nid, VisorTaskUtils.wrap(unhandledEx));
+                        taskRes.unhandledEx().put(nid, new VisorExceptionWrapper(unhandledEx));
                 }
             }
         }
@@ -117,13 +117,13 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
             taskRes.events().addAll(jobRes.events());
 
         if (jobRes.eventsEx() != null)
-            taskRes.eventsEx().put(nid, VisorTaskUtils.wrap(jobRes.eventsEx()));
+            taskRes.eventsEx().put(nid, new VisorExceptionWrapper(jobRes.eventsEx()));
 
         if (!jobRes.caches().isEmpty())
             taskRes.caches().put(nid, jobRes.caches());
 
         if (jobRes.cachesEx() != null)
-            taskRes.cachesEx().put(nid, VisorTaskUtils.wrap(jobRes.cachesEx()));
+            taskRes.cachesEx().put(nid, new VisorExceptionWrapper(jobRes.cachesEx()));
 
         if (!jobRes.igfss().isEmpty())
             taskRes.igfss().put(nid, jobRes.igfss());
@@ -132,6 +132,6 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
             taskRes.igfsEndpoints().put(nid, jobRes.igfsEndpoints());
 
         if (jobRes.igfssEx() != null)
-            taskRes.igfssEx().put(nid, VisorTaskUtils.wrap(jobRes.igfssEx()));
+            taskRes.igfssEx().put(nid, new VisorExceptionWrapper(jobRes.igfssEx()));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ea12580d/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 82555cc..e977d2e 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
@@ -112,7 +112,7 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
 
                 if (meta == null)
                     return new IgniteBiTuple<>(
-                        VisorTaskUtils.wrap(new SQLException("Fail to execute query. No metadata available.")), null);
+                        new VisorExceptionWrapper(new SQLException("Fail to execute query. No metadata available.")), null);
                 else {
                     List<VisorQueryField> names = new ArrayList<>(meta.size());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ea12580d/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
index be6f63c..a2965d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.visor.util;
 
 /**
- * Wrapper of exceptions for transferring to Visor with absent exception classes.
+ * Exception wrapper for safe for transferring to Visor.
  */
 public class VisorExceptionWrapper extends Throwable {
     /** Detail message string of this throwable */


[45/50] incubator-ignite git commit: ignite-1002 Inject Ignite before applying predicate

Posted by sb...@apache.org.
ignite-1002 Inject Ignite before applying predicate


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

Branch: refs/heads/ignite-633
Commit: 91104a29bc32b6d73e7bf4c70eafa9f777b62352
Parents: e3fe8ce
Author: agura <ag...@gridgain.com>
Authored: Mon Jun 8 18:41:43 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jun 9 20:17:33 2015 +0300

----------------------------------------------------------------------
 .../continuous/GridContinuousProcessor.java     |   2 +
 .../util/nio/GridNioDelimitedBuffer.java        |   2 +-
 .../nio/GridNioDelimitedBufferSelfTest.java     | 112 +++++++++++++++++++
 .../util/nio/GridNioDelimitedBufferTest.java    | 112 -------------------
 .../stream/socket/SocketStreamerSelfTest.java   |  29 ++---
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteStreamSelfTestSuite.java   |  39 +++++++
 .../testsuites/IgniteStreamTestSuite.java       |  39 -------
 .../testsuites/IgniteUtilSelfTestSuite.java     |   2 +-
 9 files changed, 168 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/91104a29/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 67b32a6..38d970b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -710,6 +710,8 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             try {
                 IgnitePredicate<ClusterNode> prjPred = data.projectionPredicate();
 
+                ctx.resource().injectGeneric(prjPred);
+
                 if (prjPred == null || prjPred.apply(ctx.discovery().node(ctx.localNodeId()))) {
                     registered = registerHandler(node.id(), routineId, hnd, data.bufferSize(), data.interval(),
                         data.autoUnsubscribe(), false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/91104a29/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBuffer.java
index 2b764ec..44ab4a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBuffer.java
@@ -85,7 +85,7 @@ public class GridNioDelimitedBuffer {
                         idx++;
                     }
                     else {
-                        pos = cnt - idx;
+                        pos = cnt - (i - pos) - 1;
 
                         idx = 0;
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/91104a29/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferSelfTest.java
new file mode 100644
index 0000000..cbf7d89
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferSelfTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.util.nio;
+
+import junit.framework.TestCase;
+
+import java.nio.*;
+import java.util.*;
+
+/**
+ * Tests for {@link GridNioDelimitedBuffer}.
+ */
+public class GridNioDelimitedBufferSelfTest extends TestCase {
+    /** */
+    private static final String ASCII = "ASCII";
+
+    /**
+     * Tests simple delimiter (excluded from alphabet)
+     */
+    public void testReadZString() throws Exception {
+        Random rnd = new Random();
+
+        int buffSize = 0;
+
+        byte[] delim = new byte[] {0};
+
+        List<String> strs = new ArrayList<>(50);
+
+        for (int i = 0; i < 50; i++) {
+            int len = rnd.nextInt(128) + 1;
+
+            buffSize += len + delim.length;
+
+            StringBuilder sb = new StringBuilder(len);
+
+            for (int j = 0; j < len; j++)
+                sb.append((char)(rnd.nextInt(26) + 'a'));
+
+
+            strs.add(sb.toString());
+        }
+
+        ByteBuffer buff = ByteBuffer.allocate(buffSize);
+
+        for (String str : strs) {
+            buff.put(str.getBytes(ASCII));
+            buff.put(delim);
+        }
+
+        buff.flip();
+
+        byte[] msg;
+
+        GridNioDelimitedBuffer delimBuff = new GridNioDelimitedBuffer(delim);
+
+        List<String> res = new ArrayList<>(strs.size());
+
+        while ((msg = delimBuff.read(buff)) != null)
+            res.add(new String(msg, ASCII));
+
+        assertEquals(strs, res);
+    }
+
+    /**
+     * Tests compound delimiter (included to alphabet)
+     */
+    public void testDelim() throws Exception {
+        byte[] delim = "aabb".getBytes(ASCII);
+
+        List<String> strs = Arrays.asList("za", "zaa", "zaab", "zab", "zaabaababbbbabaab");
+
+        int buffSize = 0;
+
+        for (String str : strs)
+            buffSize += str.length() + delim.length;
+
+        ByteBuffer buff = ByteBuffer.allocate(buffSize);
+
+        for (String str : strs) {
+            buff.put(str.getBytes(ASCII));
+            buff.put(delim);
+        }
+
+        buff.flip();
+
+        byte[] msg;
+
+        GridNioDelimitedBuffer delimBuff = new GridNioDelimitedBuffer(delim);
+
+        List<String> res = new ArrayList<>(strs.size());
+
+        while ((msg = delimBuff.read(buff)) != null)
+            res.add(new String(msg, ASCII));
+
+        assertEquals(strs, res);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/91104a29/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferTest.java
deleted file mode 100644
index a0dd2e5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferTest.java
+++ /dev/null
@@ -1,112 +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.util.nio;
-
-import junit.framework.TestCase;
-
-import java.nio.*;
-import java.util.*;
-
-/**
- * Tests for {@link GridNioDelimitedBuffer}.
- */
-public class GridNioDelimitedBufferTest extends TestCase {
-    /** */
-    private static final String ASCII = "ASCII";
-
-    /**
-     * Tests simple delimiter (excluded from alphabet)
-     */
-    public void testReadZString() throws Exception {
-        Random rnd = new Random();
-
-        int buffSize = 0;
-
-        byte[] delim = new byte[] {0};
-
-        List<String> strs = new ArrayList<>(50);
-
-        for (int i = 0; i < 50; i++) {
-            int len = rnd.nextInt(128) + 1;
-
-            buffSize += len + delim.length;
-
-            StringBuilder sb = new StringBuilder(len);
-
-            for (int j = 0; j < len; j++)
-                sb.append((char)(rnd.nextInt(26) + 'a'));
-
-
-            strs.add(sb.toString());
-        }
-
-        ByteBuffer buff = ByteBuffer.allocate(buffSize);
-
-        for (String str : strs) {
-            buff.put(str.getBytes(ASCII));
-            buff.put(delim);
-        }
-
-        buff.flip();
-
-        byte[] msg;
-
-        GridNioDelimitedBuffer delimBuff = new GridNioDelimitedBuffer(delim);
-
-        List<String> res = new ArrayList<>(strs.size());
-
-        while ((msg = delimBuff.read(buff)) != null)
-            res.add(new String(msg, ASCII));
-
-        assertEquals(strs, res);
-    }
-
-    /**
-     * Tests compound delimiter (included to alphabet)
-     */
-    public void testDelim() throws Exception {
-        byte[] delim = "aabb".getBytes(ASCII);
-
-        List<String> strs = Arrays.asList("za", "zaa", "zaab", "zab", "zaabaababbbbabaab");
-
-        int buffSize = 0;
-
-        for (String str : strs)
-            buffSize += str.length() + delim.length;
-
-        ByteBuffer buff = ByteBuffer.allocate(buffSize);
-
-        for (String str : strs) {
-            buff.put(str.getBytes(ASCII));
-            buff.put(delim);
-        }
-
-        buff.flip();
-
-        byte[] msg;
-
-        GridNioDelimitedBuffer delimBuff = new GridNioDelimitedBuffer(delim);
-
-        List<String> res = new ArrayList<>(strs.size());
-
-        while ((msg = delimBuff.read(buff)) != null)
-            res.add(new String(msg, ASCII));
-
-        assertEquals(strs, res);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/91104a29/modules/core/src/test/java/org/apache/ignite/stream/socket/SocketStreamerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/stream/socket/SocketStreamerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/stream/socket/SocketStreamerSelfTest.java
index 752e43c..04f9b41 100644
--- a/modules/core/src/test/java/org/apache/ignite/stream/socket/SocketStreamerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/stream/socket/SocketStreamerSelfTest.java
@@ -58,14 +58,11 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
     /** Port. */
     private static int port;
 
-    /** Ignite. */
-    private static Ignite ignite;
-
     /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration() throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration();
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        CacheConfiguration ccfg = cacheConfiguration(cfg, null);
+        CacheConfiguration ccfg = defaultCacheConfiguration();
 
         cfg.setCacheConfiguration(ccfg);
 
@@ -81,8 +78,7 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
-        ignite = startGrids(GRID_CNT);
-        ignite.<Integer, String>getOrCreateCache(defaultCacheConfiguration());
+        startGrids(GRID_CNT);
 
         try (ServerSocket sock = new ServerSocket(0)) {
             port = sock.getLocalPort();
@@ -94,11 +90,6 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
         stopAllGrids();
     }
 
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        ignite.cache(null).clear();
-    }
-
     /**
      * @throws Exception If failed.
      */
@@ -235,6 +226,12 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
     {
         SocketStreamer<Tuple, Integer, String> sockStmr = null;
 
+        Ignite ignite = grid(0);
+
+        IgniteCache<Integer, String> cache = ignite.cache(null);
+
+        cache.clear();
+
         try (IgniteDataStreamer<Integer, String> stmr = ignite.dataStreamer(null)) {
 
             stmr.allowOverwrite(true);
@@ -242,8 +239,6 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
 
             sockStmr = new SocketStreamer<>();
 
-            IgniteCache<Integer, String> cache = ignite.cache(null);
-
             sockStmr.setIgnite(ignite);
 
             sockStmr.setStreamer(stmr);
@@ -279,10 +274,10 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
 
             latch.await();
 
-            assertEquals(CNT, cache.size(CachePeekMode.PRIMARY));
-
             for (int i = 0; i < CNT; i++)
                 assertEquals(Integer.toString(i), cache.get(i));
+
+            assertEquals(CNT, cache.size(CachePeekMode.PRIMARY));
         }
         finally {
             if (sockStmr != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/91104a29/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index e0a1e6e..cc3abb4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -49,6 +49,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTest(IgniteExternalizableSelfTestSuite.suite());
         suite.addTest(IgniteP2PSelfTestSuite.suite());
         suite.addTest(IgniteCacheP2pUnmarshallingErrorTestSuite.suite());
+        suite.addTest(IgniteStreamSelfTestSuite.suite());
 
         suite.addTest(new TestSuite(GridSelfTest.class));
         suite.addTest(new TestSuite(GridProjectionSelfTest.class));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/91104a29/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamSelfTestSuite.java
new file mode 100644
index 0000000..a277fc8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamSelfTestSuite.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.testsuites;
+
+import org.apache.ignite.stream.socket.*;
+
+import junit.framework.*;
+
+/**
+ * Stream test suite.
+ */
+public class IgniteStreamSelfTestSuite extends TestSuite {
+    /**
+     * @return Stream tests suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Ignite Stream Test Suite");
+
+        suite.addTest(new TestSuite(SocketStreamerSelfTest.class));
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/91104a29/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamTestSuite.java
deleted file mode 100644
index 61be976..0000000
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamTestSuite.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.testsuites;
-
-import org.apache.ignite.stream.socket.*;
-
-import junit.framework.*;
-
-/**
- * Stream test suite.
- */
-public class IgniteStreamTestSuite extends TestSuite {
-    /**
-     * @return Stream tests suite.
-     * @throws Exception If failed.
-     */
-    public static TestSuite suite() throws Exception {
-        TestSuite suite = new TestSuite("Ignite Stream Test Suite");
-
-        suite.addTest(new TestSuite(SocketStreamerSelfTest.class));
-
-        return suite;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/91104a29/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
index 32cd038..1c75a7f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
@@ -67,7 +67,7 @@ public class IgniteUtilSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridNioSelfTest.class);
         suite.addTestSuite(GridNioFilterChainSelfTest.class);
         suite.addTestSuite(GridNioSslSelfTest.class);
-        suite.addTestSuite(GridNioDelimitedBufferTest.class);
+        suite.addTestSuite(GridNioDelimitedBufferSelfTest.class);
 
         return suite;
     }


[04/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-883_1

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-883_1


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

Branch: refs/heads/ignite-633
Commit: 3417b3dc0904708dcbea6396f88737a2aa99fa79
Parents: 6bfc78e c9f7291
Author: sboikov <se...@inria.fr>
Authored: Thu Jun 4 21:28:32 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Thu Jun 4 21:28:32 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  13 +-
 .../apache/ignite/internal/IgniteKernal.java    |   3 +
 .../managers/communication/GridIoManager.java   | 117 ++++----
 .../dht/GridClientPartitionTopology.java        |   2 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   8 +-
 .../GridDhtPartitionsExchangeFuture.java        |  19 +-
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../processors/hadoop/HadoopTaskContext.java    |  14 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   2 +-
 .../internal/visor/query/VisorQueryJob.java     |   2 +-
 ...niteDynamicCacheWithConfigStartSelfTest.java |  97 +++++++
 .../igfs/IgfsClientCacheSelfTest.java           |   9 +-
 .../IgniteMessagingWithClientTest.java          | 164 +++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   1 +
 .../fs/IgniteHadoopFileSystemCounterWriter.java |  14 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  70 ++---
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   2 +-
 .../internal/processors/hadoop/HadoopUtils.java | 282 ++++++++++++++++++-
 .../hadoop/SecondaryFileSystemProvider.java     |   4 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  48 +++-
 .../hadoop/HadoopClientProtocolSelfTest.java    |   6 +-
 .../hadoop/HadoopAbstractSelfTest.java          |  14 +-
 .../hadoop/HadoopCommandLineTest.java           |  14 +-
 .../processors/hadoop/HadoopMapReduceTest.java  | 176 +++++++++++-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |   2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |  15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |  12 +
 ...acheConfigurationPrimitiveTypesSelfTest.java | 104 +++++++
 .../IgniteCacheWithIndexingTestSuite.java       |   2 +
 .../commands/cache/VisorCacheScanCommand.scala  |   2 +-
 38 files changed, 1122 insertions(+), 194 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3417b3dc/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------


[29/50] incubator-ignite git commit: Devnotes sprint-5

Posted by sb...@apache.org.
Devnotes sprint-5


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

Branch: refs/heads/ignite-633
Commit: f1cfd2985803c8b05462737b847f956ea043341a
Parents: 015afdb
Author: avinogradov <av...@gridgain.com>
Authored: Mon Jun 8 15:17:54 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Jun 8 15:17:54 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt | 42 +++++++++++++++++++++++++++++-------------
 1 file changed, 29 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f1cfd298/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index 1562dc4..b7fea83 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -24,12 +24,14 @@ NOTE: JDK version should be 1.7.0-* or >= 1.8.0-u40.
 
 Ignite Release Instructions
 ===========================
-Use people.apache.org/keys/committer/<username>.asc key to generate KEYS file.
-Execute these commands at source root:
+Use your people.apache.org/keys/committer/<username>.asc key to generate KEYS file.
+Download https://dist.apache.org/repos/dist/release/incubator/ignite/KEYS and append you key using commands:
 
 gpg --list-sigs <keyname> >> KEYS
 gpg --armor --export <keyname> >> KEYS
 
+Upload modified KEYS file.
+
 Specify gpg profile at settings.xml. It will be used to sign sources and artifacts.
 
 <profile>
@@ -43,24 +45,38 @@ Specify gpg profile at settings.xml. It will be used to sign sources and artifac
 Ensure you have RELEASE (not SNAPSHOT) version at Ignite poms.
 Maven release plugin release:prepare goal can be used to make release tag.
 
-Deploy Ignite release candidate to maven repository and dev-svn:
+Deploy Ignite release candidate to maven repository and dev-svn, make tag:
+
+   Following command deploys Ignite to maven repository, prepares sources and fabric edition binaries.
+      mvn deploy -P apache-release,gpg,release,scala,lgpl,deploy-ignite-site -Dignite.edition=fabric -DskipTests -B
+
+   In case you want to release both fabric and hadoop editions you have to build hadoop edition first using command
+      mvn package -P apache-release,gpg,release,scala,lgpl -Dignite.edition=hadoop -DskipTests -B
+   save /target/bin/*.zip, make "mvn clean" and restore saved files before deploying fabric.
 
-   mvn deploy -P apache-release,gpg,release,scala,lgpl,deploy-ignite-site -Dignite.edition=fabric -DskipTests -B
+   Binary artifact name can be changed by setting additional property -Dignite.zip.pattern. Binary artifact will be
+   created inside /target/bin folder when release profile is used.
 
-   Binary artifact name can be changed by setting additional property -Dignite.zip.pattern.
-   Sources package name is fixed. Sources package zip will be created automatically when apache-release profile used.
+   Sources artifact name is fixed. Sources artifact will be created inside /target dir when apache-release profile is used.
 
-   In case you want to release both fabric and hadoop editions you have to build hadoop first, save /target/bin/*.zip,
-   make "mvn clean" and restore them before deploy step.
+   Nexus staging (repository.apache.org) should be closed with appropriate comment contains release version and
+   release candidate number, for example "Apache Ignite 1.0.0-rc7", when mvn deploy finished.
 
-   Nexus staging (repository.apache.org) should be closed when mvn deploy finished.
-   Checkout https://dist.apache.org/repos/dist/dev/incubator/ignite svn. Create new folder with name equals to released
-   version at svn root. Copy target/site folder content to svn/ignite/<version> folder and commit.
+   Checkout https://dist.apache.org/repos/dist/dev/incubator/ignite svn. Create release candidate folder with name
+   equals to release version with "-rc*" ending, for example "1.0.0-rc7", at svn root.
+   Copy /target/site folder content to svn/ignite/<rc-version> folder and commit with appropriate comment.
 
-Start vote based on https://dist.apache.org/repos/dist/dev/incubator/ignite/<version>.
+   Make appropriate git tag for release candidate, for example "ignite-1.0.0-incubating-rc7".
 
-Release nexus staging and copy binaries and sources from https://dist.apache.org/repos/dist/dev/incubator/ignite/<version>
+Start vote based on https://dist.apache.org/repos/dist/dev/incubator/ignite/<rc-version>.
+
+Release nexus staging, move binaries and sources from https://dist.apache.org/repos/dist/dev/incubator/ignite/<rc-version>
 to https://dist.apache.org/repos/dist/release/incubator/ignite/<version> when version accepted.
+Use svn mv ^/dev/incubator/ignite/<rc-version> ^/release/incubator/ignite/<version> command for proper moving.
+
+Make appropriate git tag for released version, for example "ignite-1.0.0-incubating".
+
+Send an email to dev@ignite.incubator.apache.org contains release svn url.
 
 
 JCache TCK compliance


[42/50] incubator-ignite git commit: Merge branch 'ignite-992' into ignite-sprint-5

Posted by sb...@apache.org.
Merge branch 'ignite-992' into ignite-sprint-5


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

Branch: refs/heads/ignite-633
Commit: abeddc9c49d5e38d26e35dbf2e662bc7a11bfd0c
Parents: 638dd31 8740b6e
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Jun 9 16:38:25 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Jun 9 16:38:25 2015 +0700

----------------------------------------------------------------------
 .../visor/node/VisorNodeDataCollectorTask.java  |  9 ++-
 .../node/VisorNodeDataCollectorTaskResult.java  | 17 ++--
 .../node/VisorNodeSuppressedErrorsTask.java     | 12 ++-
 .../internal/visor/query/VisorQueryJob.java     | 11 +--
 .../internal/visor/query/VisorQueryTask.java    |  3 +-
 .../visor/util/VisorExceptionWrapper.java       | 81 ++++++++++++++++++++
 .../internal/visor/util/VisorTaskUtils.java     | 10 +++
 7 files changed, 124 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[40/50] incubator-ignite git commit: # ignite-sprint-5 more debug info in test

Posted by sb...@apache.org.
# ignite-sprint-5 more debug info in test


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

Branch: refs/heads/ignite-633
Commit: 410c1d7941fd0283bc6c419e3fc54bce89f59e3f
Parents: e7d8b5a
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 9 11:43:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 9 11:43:52 2015 +0300

----------------------------------------------------------------------
 .../processors/continuous/GridEventConsumeSelfTest.java       | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/410c1d79/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
index 2c9e513..9ffef4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
@@ -114,8 +114,11 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
                 GridContinuousProcessor proc = grid.context().continuous();
 
                 try {
-                    if (!noAutoUnsubscribe)
-                        assertEquals(0, U.<Map>field(proc, "rmtInfos").size());
+                    if (!noAutoUnsubscribe) {
+                        Map rmtInfos = U.field(proc, "rmtInfos");
+
+                        assertTrue("Unexpected remote infos: " + rmtInfos, rmtInfos.isEmpty());
+                    }
                 }
                 finally {
                     U.<Map>field(proc, "rmtInfos").clear();


[36/50] incubator-ignite git commit: # ignite-sprint-5 fixed javadoc

Posted by sb...@apache.org.
# ignite-sprint-5 fixed javadoc


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

Branch: refs/heads/ignite-633
Commit: 14bb076c2ff6343c1681279d867c2a2bc56b4714
Parents: 0fa2853
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 9 09:00:04 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 9 09:00:04 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/spi/IgniteSpiAdapter.java | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/14bb076c/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index d095491..6e7a706 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -543,12 +543,18 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         return U.spiAttribute(this, attrName);
     }
 
-    /** {@inheritDoc} */
+    /**
+     * @param obj Timeout object.
+     * @see IgniteSpiContext#addTimeoutObject(IgniteSpiTimeoutObject)
+     */
     protected void addTimeoutObject(IgniteSpiTimeoutObject obj) {
         spiCtx.addTimeoutObject(obj);
     }
 
-    /** {@inheritDoc} */
+    /**
+     * @param obj Timeout object.
+     * @see IgniteSpiContext#removeTimeoutObject(IgniteSpiTimeoutObject)
+     */
     protected void removeTimeoutObject(IgniteSpiTimeoutObject obj) {
         spiCtx.removeTimeoutObject(obj);
     }


[08/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-883_1' into ignite-883_1

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-883_1' into ignite-883_1


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

Branch: refs/heads/ignite-633
Commit: a6761529504b9df26703558f9452d53eb9e6dd0f
Parents: ad2f4ef 3417b3d
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 5 09:57:06 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 5 09:57:06 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  13 +-
 .../apache/ignite/internal/IgniteKernal.java    |   3 +
 .../managers/communication/GridIoManager.java   | 117 ++++----
 .../dht/GridClientPartitionTopology.java        |   2 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   8 +-
 .../GridDhtPartitionsExchangeFuture.java        |  19 +-
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../processors/hadoop/HadoopTaskContext.java    |  14 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   2 +-
 .../internal/visor/query/VisorQueryJob.java     |   2 +-
 ...niteDynamicCacheWithConfigStartSelfTest.java |  97 +++++++
 .../igfs/IgfsClientCacheSelfTest.java           |   9 +-
 .../IgniteMessagingWithClientTest.java          | 164 +++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   1 +
 .../fs/IgniteHadoopFileSystemCounterWriter.java |  14 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  70 ++---
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   2 +-
 .../internal/processors/hadoop/HadoopUtils.java | 282 ++++++++++++++++++-
 .../hadoop/SecondaryFileSystemProvider.java     |   4 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  48 +++-
 .../hadoop/HadoopClientProtocolSelfTest.java    |   6 +-
 .../hadoop/HadoopAbstractSelfTest.java          |  14 +-
 .../hadoop/HadoopCommandLineTest.java           |  14 +-
 .../processors/hadoop/HadoopMapReduceTest.java  | 176 +++++++++++-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |   2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |  15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |  12 +
 ...acheConfigurationPrimitiveTypesSelfTest.java | 104 +++++++
 .../IgniteCacheWithIndexingTestSuite.java       |   2 +
 .../commands/cache/VisorCacheScanCommand.scala  |   2 +-
 38 files changed, 1122 insertions(+), 194 deletions(-)
----------------------------------------------------------------------


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


[24/50] incubator-ignite git commit: # ignite-sprint-5 sanity check for classname.properies load

Posted by sb...@apache.org.
# ignite-sprint-5 sanity check for classname.properies load


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

Branch: refs/heads/ignite-633
Commit: 8467a3c36414192268b8d852715fc501f53b4772
Parents: a5b5ec7
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 12:42:36 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 12:42:36 2015 +0300

----------------------------------------------------------------------
 .../internal/MarshallerContextAdapter.java      | 36 ++++++++++++++++++--
 1 file changed, 34 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8467a3c3/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
index 5dca2f2..21f2264 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.marshaller.*;
 import org.jsr166.*;
@@ -49,10 +50,29 @@ public abstract class MarshallerContextAdapter implements MarshallerContext {
 
             Enumeration<URL> urls = ldr.getResources(CLS_NAMES_FILE);
 
-            while (urls.hasMoreElements())
+            boolean foundClsNames = false;
+
+            while (urls.hasMoreElements()) {
                 processResource(urls.nextElement());
 
-            processResource(ldr.getResource(JDK_CLS_NAMES_FILE));
+                foundClsNames = true;
+            }
+
+            if (!foundClsNames)
+                throw new IgniteException("Failed to load class names properties file packaged with ignite binaries " +
+                    "[file=" + CLS_NAMES_FILE + ", ldr=" + ldr + ']');
+
+            URL jdkClsNames = ldr.getResource(JDK_CLS_NAMES_FILE);
+
+            if (jdkClsNames == null)
+                throw new IgniteException("Failed to load class names properties file packaged with ignite binaries " +
+                    "[file=" + JDK_CLS_NAMES_FILE + ", ldr=" + ldr + ']');
+
+            processResource(jdkClsNames);
+
+            checkHasClassName(GridDhtPartitionFullMap.class.getName(), ldr, CLS_NAMES_FILE);
+            checkHasClassName(GridDhtPartitionMap.class.getName(), ldr, CLS_NAMES_FILE);
+            checkHasClassName(HashMap.class.getName(), ldr, JDK_CLS_NAMES_FILE);
         }
         catch (IOException e) {
             throw new IllegalStateException("Failed to initialize marshaller context.", e);
@@ -60,6 +80,18 @@ public abstract class MarshallerContextAdapter implements MarshallerContext {
     }
 
     /**
+     * @param clsName Class name.
+     * @param ldr Class loader used to get properties file.
+     * @param fileName File name.
+     */
+    private void checkHasClassName(String clsName, ClassLoader ldr, String fileName) {
+        if (!map.containsKey(clsName.hashCode()))
+            throw new IgniteException("Failed to read class name from class names properties file. " +
+                "Make sure class names properties file packaged with ignite binaries is not corrupted " +
+                "[clsName=" + clsName + ", fileName=" + fileName + ", ldr=" + ldr + ']');
+    }
+
+    /**
      * @param url Resource URL.
      * @throws IOException In case of error.
      */


[07/50] incubator-ignite git commit: Merge branches 'ignite-gg-9621' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-gg-9621

Posted by sb...@apache.org.
Merge branches 'ignite-gg-9621' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-gg-9621


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

Branch: refs/heads/ignite-633
Commit: ead66e7ed62f5134b98be984f098655ef4d04c68
Parents: 2b443d9 c9f7291
Author: AKuznetsov <ak...@gridgain.com>
Authored: Fri Jun 5 10:14:08 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Fri Jun 5 10:14:08 2015 +0700

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java   | 117 ++++----
 .../dht/GridClientPartitionTopology.java        |   2 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   8 +-
 .../GridDhtPartitionsExchangeFuture.java        |  19 +-
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../processors/hadoop/HadoopTaskContext.java    |  14 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   2 +-
 ...niteDynamicCacheWithConfigStartSelfTest.java |  97 +++++++
 .../igfs/IgfsClientCacheSelfTest.java           |   9 +-
 .../IgniteMessagingWithClientTest.java          | 164 +++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   1 +
 .../fs/IgniteHadoopFileSystemCounterWriter.java |  14 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  70 ++---
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   2 +-
 .../internal/processors/hadoop/HadoopUtils.java | 282 ++++++++++++++++++-
 .../hadoop/SecondaryFileSystemProvider.java     |   4 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  48 +++-
 .../hadoop/HadoopClientProtocolSelfTest.java    |   6 +-
 .../hadoop/HadoopAbstractSelfTest.java          |  14 +-
 .../hadoop/HadoopCommandLineTest.java           |  14 +-
 .../processors/hadoop/HadoopMapReduceTest.java  | 176 +++++++++++-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |   2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |  15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |  12 +
 32 files changed, 999 insertions(+), 191 deletions(-)
----------------------------------------------------------------------



[26/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-883_1' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-883_1' into ignite-sprint-5


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

Branch: refs/heads/ignite-633
Commit: 015afdbf1921e6ae5e8cd2f92efc8b4a695f05ca
Parents: 8467a3c c7bc598
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 13:38:48 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 13:38:48 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |  26 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   8 +-
 .../internal/managers/GridManagerAdapter.java   |   9 +
 .../checkpoint/GridCheckpointManager.java       |  52 +-
 .../discovery/GridDiscoveryManager.java         |  28 +-
 .../processors/cache/GridCacheAdapter.java      |   4 +
 .../GridCachePartitionExchangeManager.java      |  26 +-
 .../processors/cache/GridCacheTtlManager.java   |   9 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  18 +-
 .../GridDhtPartitionsExchangeFuture.java        |  50 +-
 .../cache/transactions/IgniteTxManager.java     |   3 -
 .../datastructures/DataStructuresProcessor.java | 107 +++-
 .../service/GridServiceProcessor.java           |   4 +-
 .../timeout/GridSpiTimeoutObject.java           |  73 +++
 .../timeout/GridTimeoutProcessor.java           | 105 +++-
 .../util/nio/GridCommunicationClient.java       |  30 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |  13 +-
 .../util/nio/GridTcpCommunicationClient.java    | 554 -------------------
 .../util/nio/GridTcpNioCommunicationClient.java |   8 -
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  27 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  10 +
 .../ignite/spi/IgniteSpiTimeoutObject.java      |  44 ++
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |   3 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 438 ++++-----------
 .../tcp/TcpCommunicationSpiMBean.java           |   2 -
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   3 -
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  10 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 156 +-----
 .../IgniteCountDownLatchAbstractSelfTest.java   | 102 ++++
 .../IgniteCacheClientNearCacheExpiryTest.java   | 103 ++++
 .../IgniteCacheExpiryPolicyTestSuite.java       |   2 +
 .../internal/util/nio/GridNioSelfTest.java      |   2 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |   4 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   2 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |   2 -
 ...cpCommunicationSpiMultithreadedSelfTest.java |   2 +-
 .../discovery/AbstractDiscoverySelfTest.java    |  13 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  25 +
 .../testframework/GridSpiTestContext.java       |  10 +
 39 files changed, 886 insertions(+), 1201 deletions(-)
----------------------------------------------------------------------



[06/50] incubator-ignite git commit: IGNITE-941 - Read-only mode for transactional cache

Posted by sb...@apache.org.
IGNITE-941 - Read-only mode for transactional cache


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

Branch: refs/heads/ignite-633
Commit: b4d19159a76559152435946d1fff421679712599
Parents: b58c892
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 4 18:08:35 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 4 18:08:35 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |  2 +
 .../ignite/internal/IgniteNodeAttributes.java   |  5 +-
 .../processors/cache/GridCacheProcessor.java    |  2 +-
 .../transactions/IgniteTxLocalAdapter.java      | 29 +++++++--
 .../IgniteTxRollbackCheckedException.java       |  9 +++
 ...acheReadOnlyTransactionalClientSelfTest.java | 68 +++++++++++++++-----
 6 files changed, 94 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b4d19159/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index a0d97c7..9bc6cce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -1168,6 +1168,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
         add(ATTR_CLIENT_MODE, cfg.isClientMode());
 
+        add(ATTR_CONSISTENCY_CHECK_SKIPPED, getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK));
+
         // Build a string from JVM arguments, because parameters with spaces are split.
         SB jvmArgs = new SB(512);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b4d19159/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index 98cc3a7..928db5e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -126,9 +126,12 @@ public final class IgniteNodeAttributes {
     /** Security subject for authenticated node. */
     public static final String ATTR_SECURITY_SUBJECT = ATTR_PREFIX + ".security.subject";
 
-    /** Cache interceptors. */
+    /** Client mode flag. */
     public static final String ATTR_CLIENT_MODE = ATTR_PREFIX + ".cache.client";
 
+    /** Configuration consistency check disabled flag. */
+    public static final String ATTR_CONSISTENCY_CHECK_SKIPPED = ATTR_PREFIX + ".consistency.check.skipped";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b4d19159/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 bbd5fe9..871cd77 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -674,7 +674,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
                 for (ClusterNode n : ctx.discovery().remoteNodes()) {
-                    if (Boolean.valueOf(n.<String>attribute(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)))
+                    if (n.attribute(ATTR_CONSISTENCY_CHECK_SKIPPED))
                         continue;
 
                     checkTransactionConfiguration(n);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b4d19159/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 4063be2..8b5eaec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -2535,6 +2535,19 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     }
 
     /**
+     * @param cacheCtx Cache context.
+     * @throws IgniteCheckedException If updates are not allowed.
+     */
+    private void checkUpdatesAllowed(GridCacheContext cacheCtx) throws IgniteCheckedException {
+        if (!cacheCtx.updatesAllowed()) {
+            throw new IgniteTxRollbackCheckedException(new CacheException(
+                "Updates are not allowed for transactional cache: " + cacheCtx.name() + ". Configure " +
+                "persistence store on client or use remote closure execution to start transactions " +
+                "from server nodes."));
+        }
+    }
+
+    /**
      * Internal method for all put and transform operations. Only one of {@code map}, {@code transformMap}
      * maps must be non-null.
      *
@@ -2561,8 +2574,12 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     ) {
         assert filter == null || invokeMap == null;
 
-        if (!cacheCtx.updatesAllowed())
-            throw new CacheException("Updates are not allowed for cache: " + cacheCtx.name());
+        try {
+            checkUpdatesAllowed(cacheCtx);
+        }
+        catch (IgniteCheckedException e) {
+            return new GridFinishedFuture(e);
+        }
 
         cacheCtx.checkSecurity(SecurityPermission.CACHE_PUT);
 
@@ -2784,8 +2801,12 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         @Nullable GridCacheEntryEx cached,
         final boolean retval,
         @Nullable final CacheEntryPredicate[] filter) {
-        if (!cacheCtx.updatesAllowed())
-            throw new CacheException("Updates are not allowed for cache: " + cacheCtx.name());
+        try {
+            checkUpdatesAllowed(cacheCtx);
+        }
+        catch (IgniteCheckedException e) {
+            return new GridFinishedFuture(e);
+        }
 
         cacheCtx.checkSecurity(SecurityPermission.CACHE_REMOVE);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b4d19159/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxRollbackCheckedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxRollbackCheckedException.java b/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxRollbackCheckedException.java
index 1d1e022..f8ee265 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxRollbackCheckedException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxRollbackCheckedException.java
@@ -36,6 +36,15 @@ public class IgniteTxRollbackCheckedException extends IgniteCheckedException {
     }
 
     /**
+     * Creates new exception with given nested exception.
+     *
+     * @param cause Nested exception.
+     */
+    public IgniteTxRollbackCheckedException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
      * Creates new rollback exception with given error message and optional nested exception.
      *
      * @param msg Error message.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b4d19159/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
index e2a1def..f2c38e1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
@@ -31,6 +31,8 @@ import javax.cache.*;
 import javax.cache.configuration.*;
 import javax.cache.processor.*;
 
+import static org.apache.ignite.IgniteSystemProperties.*;
+
 /**
  * Tests for read-only transactional cache client.
  */
@@ -39,6 +41,9 @@ public class CacheReadOnlyTransactionalClientSelfTest extends GridCommonAbstract
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** */
+    private static final String CACHE_NAME = "test-cache";
+
+    /** */
     private boolean client;
 
     /** */
@@ -55,6 +60,7 @@ public class CacheReadOnlyTransactionalClientSelfTest extends GridCommonAbstract
 
         CacheConfiguration cc = new CacheConfiguration();
 
+        cc.setName(CACHE_NAME);
         cc.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
         cc.setCacheStoreFactory(factory);
 
@@ -100,7 +106,7 @@ public class CacheReadOnlyTransactionalClientSelfTest extends GridCommonAbstract
 
         Ignite ignite = startGrid();
 
-        IgniteCache cache = ignite.cache(null);
+        IgniteCache cache = ignite.cache(CACHE_NAME);
 
         cache.get(0);
         cache.getAll(F.asSet(0, 1));
@@ -140,6 +146,25 @@ public class CacheReadOnlyTransactionalClientSelfTest extends GridCommonAbstract
     /**
      * @throws Exception If failed.
      */
+    public void testDisabledConsistencyCheck() throws Exception {
+        client = false;
+        nearEnabled = false;
+        factory = new Factory2();
+
+        System.setProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK, "true");
+
+        startGrid("client-1");
+
+        factory = new Factory1();
+
+        System.clearProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK);
+
+        startGrid("client-2");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testNoStoreNearDisabled() throws Exception {
         nearEnabled = false;
 
@@ -164,7 +189,7 @@ public class CacheReadOnlyTransactionalClientSelfTest extends GridCommonAbstract
 
         Ignite ignite = startGrid();
 
-        IgniteCache cache = ignite.cache(null);
+        IgniteCache cache = ignite.cache(CACHE_NAME);
 
         cache.get(0);
         cache.getAll(F.asSet(0, 1));
@@ -173,91 +198,104 @@ public class CacheReadOnlyTransactionalClientSelfTest extends GridCommonAbstract
             cache.getAndPut(0, 0);
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.getAndPutIfAbsent(0, 0);
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.getAndRemove(0);
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.getAndReplace(0, 0);
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.put(0, 0);
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.putAll(F.asMap(0, 0, 1, 1));
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.putIfAbsent(0, 0);
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.remove(0);
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.remove(0, 0);
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.removeAll(F.asSet(0, 1));
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.removeAll();
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.invoke(0, new EP());
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
 
         try {
             cache.invokeAll(F.asSet(0, 1), new EP());
         }
         catch (CacheException e) {
-            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
         }
     }
 


[15/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-883_1

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-883_1


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

Branch: refs/heads/ignite-633
Commit: 75006b805a9997354ea9a1c645ad11ab8f9ba631
Parents: e237b00 5f06f57
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 5 18:11:06 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 5 18:11:06 2015 +0300

----------------------------------------------------------------------
 .../affinity/GridAffinityProcessor.java         | 23 ++++-
 .../datastreamer/DataStreamerImpl.java          | 92 ++++++++++++++------
 .../DataStreamerMultiThreadedSelfTest.java      | 59 +++++++++----
 .../IgniteMessagingWithClientTest.java          |  2 +
 4 files changed, 131 insertions(+), 45 deletions(-)
----------------------------------------------------------------------



[16/50] incubator-ignite git commit: #ignite-990: Add method getAllOutTx to IgniteCache.

Posted by sb...@apache.org.
#ignite-990: Add method getAllOutTx to IgniteCache.


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

Branch: refs/heads/ignite-633
Commit: ab20fd9059c21c65f3b34da67d75fc94098850e3
Parents: 5f06f57
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Jun 5 19:06:10 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Jun 5 19:06:10 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |  11 ++
 .../processors/cache/GridCacheAdapter.java      |  17 ++-
 .../processors/cache/GridCacheProxyImpl.java    |  14 ++-
 .../processors/cache/IgniteCacheProxy.java      |  23 +++++
 .../processors/cache/IgniteInternalCache.java   |  11 +-
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  83 +++++++++++++++
 .../GridCacheExAbstractFullApiSelfTest.java     | 103 -------------------
 .../GridCacheExColocatedFullApiSelfTest.java    |  33 ------
 .../near/GridCacheExNearFullApiSelfTest.java    |  39 -------
 .../GridCacheExReplicatedFullApiSelfTest.java   |  33 ------
 .../local/GridCacheExLocalFullApiSelfTest.java  |  30 ------
 .../IgniteCacheFullApiSelfTestSuite.java        |   6 --
 13 files changed, 155 insertions(+), 250 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index f9007a2..07f4f3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -311,10 +311,21 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
     @IgniteAsyncSupported
     @Override public Map<K, V> getAll(Set<? extends K> keys);
 
+    /**
+     * Gets values from cache. Will bypass started transaction, if any, i.e. will not enlist entries
+     * and will not lock any keys if pessimistic transaction is started by thread.
+     *
+     * @param keys The keys whose associated values are to be returned.
+     * @return A map of entries that were found for the given keys.
+     */
+    @IgniteAsyncSupported
+    public Map<K, V> getAllOutTx(Set<? extends K> keys);
+
     /** {@inheritDoc} */
     @IgniteAsyncSupported
     @Override public boolean containsKey(K key);
 
+    /** {@inheritDoc} */
     @IgniteAsyncSupported
     public boolean containsKeys(Set<? extends K> keys);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 9d98ce7..56b5c15 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -1220,11 +1220,22 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<K, V> getAllOutTx(List<K> keys) throws IgniteCheckedException {
+    @Nullable @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
+        return getAllOutTxAsync(keys).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
-        return getAllAsync(keys, !ctx.config().isReadFromBackup(), /*skip tx*/true, null, null, taskName, true, false)
-            .get();
+        return getAllAsync(keys,
+            !ctx.config().isReadFromBackup(),
+            /*skip tx*/true,
+            null,
+            null,
+            taskName,
+            !ctx.keepPortable(),
+            false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index 9a6d08a..63ba242 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -329,7 +329,7 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<K, V> getAllOutTx(List<K> keys) throws IgniteCheckedException {
+    @Nullable @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
         CacheOperationContext prev = gate.enter(opCtx);
 
         try {
@@ -341,6 +341,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.getAllOutTxAsync(keys);
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isIgfsDataCache() {
         CacheOperationContext prev = gate.enter(opCtx);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f840015..4390993 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -699,6 +699,29 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) {
+        try {
+            CacheOperationContext prev = onEnter(opCtx);
+
+            try {
+                if (isAsync()) {
+                    setFuture(delegate.getAllOutTxAsync(keys));
+
+                    return null;
+                }
+                else
+                    return delegate.getAllOutTx(keys);
+            }
+            finally {
+                onLeave(prev);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw cacheException(e);
+        }
+    }
+
     /**
      * @param keys Keys.
      * @return Values map.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index ccce1b7..d98379c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -1623,7 +1623,16 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
      * @return Value.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public Map<K, V> getAllOutTx(List<K> keys) throws IgniteCheckedException;
+    public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException;
+
+    /**
+     * Gets values from cache. Will bypass started transaction, if any, i.e. will not enlist entries
+     * and will not lock any keys if pessimistic transaction is started by thread.
+     *
+     * @param keys Keys to get values for.
+     * @return Future for getAllOutTx operation.
+     */
+    public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys);
 
     /**
      * Checks whether this cache is IGFS data cache.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index e33e0d4..b98c5d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -669,7 +669,7 @@ public class IgfsMetaManager extends IgfsManager {
     private Map<String, IgfsListingEntry> directoryListing(IgniteUuid fileId, boolean skipTx) throws IgniteCheckedException {
         assert fileId != null;
 
-        IgfsFileInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singletonList(fileId)).get(fileId) :
+        IgfsFileInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singleton(fileId)).get(fileId) :
             id2InfoPrj.get(fileId);
 
         return info == null ? Collections.<String, IgfsListingEntry>emptyMap() : info.listing();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index da20869..9bfbd15 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -4893,6 +4893,89 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testGetOutTx() throws Exception {
+        checkGetOutTx(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetOutTxAsync() throws Exception {
+        checkGetOutTx(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkGetOutTx(boolean async) throws Exception {
+        final AtomicInteger lockEvtCnt = new AtomicInteger();
+
+        IgnitePredicate<Event> lsnr = new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                lockEvtCnt.incrementAndGet();
+
+                return true;
+            }
+        };
+
+        try {
+            IgniteCache<String, Integer> cache = grid(0).cache(null);
+
+            List<String> keys = primaryKeysForCache(cache, 2);
+
+            assertEquals(2, keys.size());
+
+            cache.put(keys.get(0), 0);
+            cache.put(keys.get(1), 1);
+
+            grid(0).events().localListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
+
+            if (async)
+                cache = cache.withAsync();
+
+            try (Transaction tx = transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                Integer val0 = cache.get(keys.get(0));
+
+                if (async)
+                    val0 = cache.<Integer>future().get();
+
+                assertEquals(0, val0.intValue());
+
+                Map<String, Integer> allOutTx = cache.getAllOutTx(F.asSet(keys.get(1)));
+
+                if (async)
+                    allOutTx = cache.<Map<String, Integer>>future().get();
+
+                assertEquals(1, allOutTx.size());
+
+                assertTrue(allOutTx.containsKey(keys.get(1)));
+
+                assertEquals(1, allOutTx.get(keys.get(1)).intValue());
+            }
+
+            assertTrue(GridTestUtils.waitForCondition(new PA() {
+                @Override public boolean apply() {
+                    info("Lock event count: " + lockEvtCnt.get());
+                    if (atomicityMode() == ATOMIC)
+                        return lockEvtCnt.get() == 0;
+
+                    if (cacheMode() == PARTITIONED && nearEnabled()) {
+                        if (!grid(0).configuration().isClientMode())
+                            return lockEvtCnt.get() == 4;
+                    }
+
+                    return lockEvtCnt.get() == 2;
+                }
+            }, 15000));
+        }
+        finally {
+            grid(0).events().stopLocalListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
+        }
+    }
+
+    /**
      * Sets given value, returns old value.
      */
     public static final class SetValueProcessor implements EntryProcessor<String, Integer, Integer> {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java
deleted file mode 100644
index 7a1b7cc..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java
+++ /dev/null
@@ -1,103 +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.cache;
-
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.transactions.*;
-
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.events.EventType.*;
-import static org.apache.ignite.transactions.TransactionConcurrency.*;
-import static org.apache.ignite.transactions.TransactionIsolation.*;
-
-/**
- * Abstract test for private cache interface.
- */
-public abstract class GridCacheExAbstractFullApiSelfTest extends GridCacheAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 4;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetOutTx() throws Exception {
-        final AtomicInteger lockEvtCnt = new AtomicInteger();
-
-        IgnitePredicate<Event> lsnr = new IgnitePredicate<Event>() {
-            @Override public boolean apply(Event evt) {
-                lockEvtCnt.incrementAndGet();
-
-                return true;
-            }
-        };
-
-        try {
-            grid(0).events().localListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
-
-            GridCacheAdapter<String, Integer> cache = ((IgniteKernal)grid(0)).internalCache();
-
-            try (Transaction tx = transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                int key = 0;
-
-                for (int i = 0; i < 1000; i++) {
-                    if (grid(0).affinity(null).mapKeyToNode("key" + i).id().equals(grid(0).localNode().id())) {
-                        key = i;
-
-                        break;
-                    }
-                }
-
-                cache.get("key" + key);
-
-                for (int i = key + 1; i < 1000; i++) {
-                    if (grid(0).affinity(null).mapKeyToNode("key" + i).id().equals(grid(0).localNode().id())) {
-                        key = i;
-
-                        break;
-                    }
-                }
-
-                cache.getAllOutTx(F.asList("key" + key));
-            }
-
-            assertTrue(GridTestUtils.waitForCondition(new PA() {
-                @Override public boolean apply() {
-                    info("Lock event count: " + lockEvtCnt.get());
-
-                    return lockEvtCnt.get() == (nearEnabled() ? 4 : 2);
-                }
-            }, 15000));
-        }
-        finally {
-            grid(0).events().stopLocalListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
deleted file mode 100644
index 55b1f92..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
+++ /dev/null
@@ -1,33 +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.cache.distributed.dht;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on colocated cache.
- */
-public class GridCacheExColocatedFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
deleted file mode 100644
index 11ec96a..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
+++ /dev/null
@@ -1,39 +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.cache.distributed.near;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on partitioned cache with near enabled.
- */
-public class GridCacheExNearFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return new NearCacheConfiguration();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
deleted file mode 100644
index 32ee784..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
+++ /dev/null
@@ -1,33 +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.cache.distributed.replicated;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on replicated cache.
- */
-public class GridCacheExReplicatedFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return REPLICATED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java
deleted file mode 100644
index 71c0495..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java
+++ /dev/null
@@ -1,30 +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.cache.local;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-/**
- * Tests private cache interface on local cache.
- */
-public class GridCacheExLocalFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    @Override protected CacheMode cacheMode() {
-        return CacheMode.LOCAL;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index 369e041..5fa3283 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -130,12 +130,6 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiNodeFullApiSelfTest.class);
 
-        // Private cache API.
-        suite.addTestSuite(GridCacheExLocalFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheExReplicatedFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheExNearFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheExColocatedFullApiSelfTest.class);
-
 //        Multithreaded // TODO: GG-708
 //        suite.addTestSuite(GridCacheLocalFullApiMultithreadedSelfTest.class);
 //        suite.addTestSuite(GridCacheReplicatedFullApiMultithreadedSelfTest.class);


[43/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-883_1' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-883_1' into ignite-sprint-5


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

Branch: refs/heads/ignite-633
Commit: e3fe8ce581b20d3e378876e3c5b94ff6e22baff8
Parents: abeddc9 eb0e2db
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 9 13:44:56 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 9 13:44:56 2015 +0300

----------------------------------------------------------------------
 .../processors/affinity/GridAffinityAssignment.java   | 12 ++++++++++++
 .../affinity/GridAffinityAssignmentCache.java         |  4 ++--
 .../dht/atomic/GridNearAtomicUpdateFuture.java        |  6 +++++-
 .../preloader/GridDhtPartitionsExchangeFuture.java    | 14 +++++++++-----
 4 files changed, 28 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[23/50] incubator-ignite git commit: # ignite-sprint-5 javadoc warning

Posted by sb...@apache.org.
# ignite-sprint-5 javadoc warning


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

Branch: refs/heads/ignite-633
Commit: a5b5ec703dbcb593a945c8eb1d37e1261e55e834
Parents: 59db4a5
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 11:32:14 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 11:32:14 2015 +0300

----------------------------------------------------------------------
 modules/core/src/main/java/org/apache/ignite/IgniteCache.java | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a5b5ec70/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index 07f4f3f..a54adc9 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -325,7 +325,12 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
     @IgniteAsyncSupported
     @Override public boolean containsKey(K key);
 
-    /** {@inheritDoc} */
+    /**
+     * Determines if the {@link Cache} contains entries for the specified keys.
+     *
+     * @param keys Key whose presence in this cache is to be tested.
+     * @return {@code True} if this cache contains a mapping for the specified keys.
+     */
     @IgniteAsyncSupported
     public boolean containsKeys(Set<? extends K> keys);
 


[18/50] incubator-ignite git commit: # ignite-883 javadoc

Posted by sb...@apache.org.
# ignite-883 javadoc


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

Branch: refs/heads/ignite-633
Commit: 50b0b49987a99fd10691346244f7c60b8d2b6ddd
Parents: 75006b8
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 8 10:05:39 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 8 10:05:39 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/spi/IgniteSpiTimeoutObject.java     | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50b0b499/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiTimeoutObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiTimeoutObject.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiTimeoutObject.java
index f7fbd27f..b3fc28e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiTimeoutObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiTimeoutObject.java
@@ -20,7 +20,11 @@ package org.apache.ignite.spi;
 import org.apache.ignite.lang.*;
 
 /**
- *
+ * Provides possibility to schedule delayed execution,
+ * see {@link IgniteSpiContext#addTimeoutObject(IgniteSpiTimeoutObject)}.
+ * <p>
+ * Note: all timeout objects are executed in single dedicated thread, so implementation
+ * of {@link #onTimeout()} should not use time consuming and blocking method.
  */
 public interface IgniteSpiTimeoutObject {
     /**


[48/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5


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

Branch: refs/heads/ignite-633
Commit: e0426f0f92d8700fee7db667c5fccf7db89d81ad
Parents: ac908e9 928be42
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 10 10:10:35 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 10 10:10:35 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/managers/indexing/GridIndexingManager.java   | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------