You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/06/02 00:19:13 UTC

[01/50] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-943

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-389 c527a0447 -> d0157d4ef


Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-943


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

Branch: refs/heads/ignite-389
Commit: 3dea5f148e3fc017bf9643e69f2661a132cfc96c
Parents: 110573b 581f4d9
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 17:19:27 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 17:19:27 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  1 -
 .../processors/cache/GridCacheAdapter.java      | 15 ++++++++-
 .../cache/GridCacheConcurrentMap.java           | 21 ++++++++++---
 .../processors/cache/GridCacheProcessor.java    | 32 +++++++++++---------
 .../processors/cache/GridCacheProxyImpl.java    | 12 ++++++++
 .../processors/cache/IgniteInternalCache.java   |  5 +++
 .../cache/query/GridCacheQueryAdapter.java      |  2 ++
 .../cache/query/GridCacheQueryErrorFuture.java  |  2 ++
 .../cache/query/GridCacheQueryManager.java      |  2 +-
 .../continuous/CacheContinuousQueryManager.java | 24 ++++++++++++---
 .../cacheobject/IgniteCacheObjectProcessor.java |  5 ++-
 .../IgniteCacheObjectProcessorImpl.java         |  2 +-
 12 files changed, 93 insertions(+), 30 deletions(-)
----------------------------------------------------------------------



[19/50] [abbrv] incubator-ignite git commit: # ignite-709

Posted by ag...@apache.org.
# ignite-709


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

Branch: refs/heads/ignite-389
Commit: 07b6cb598ddb8f6c8ca76245ffbe6c434e08f870
Parents: 2c3ce15
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 29 11:26:45 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 29 11:34:51 2015 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionMap.java      |  2 +-
 ...niteCacheClientNodeChangingTopologyTest.java |  6 ++
 .../junits/common/GridCommonAbstractTest.java   | 61 +++++++++++++++++---
 3 files changed, 60 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07b6cb59/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
index facf7e3..faa6cf6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
@@ -237,7 +237,7 @@ public class GridDhtPartitionMap implements Comparable<GridDhtPartitionMap>, Ext
      * @return Full string representation.
      */
     public String toFullString() {
-        return S.toString(GridDhtPartitionMap.class, this, "size", size(), "map", super.toString());
+        return S.toString(GridDhtPartitionMap.class, this, "size", size(), "map", map.toString());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07b6cb59/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index 47c1d7e..922e618 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -1556,6 +1556,9 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
                 catch (TimeoutException e) {
                     log.error("Failed to wait for update.");
 
+                    for (Ignite ignite : G.allGrids())
+                        dumpCacheDebugInfo(ignite);
+
                     U.dumpThreads(log);
 
                     CyclicBarrier barrier0 = updateBarrier;
@@ -1593,6 +1596,9 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
                 catch (TimeoutException e) {
                     log.error("Failed to wait for update.");
 
+                    for (Ignite ignite : G.allGrids())
+                        dumpCacheDebugInfo(ignite);
+
                     U.dumpThreads(log);
 
                     CyclicBarrier barrier0 = updateBarrier;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07b6cb59/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 1f4c7b6..cded7fa 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.processors.cache.local.*;
+import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
@@ -383,19 +384,31 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                             int actual = owners.size();
 
                             if (affNodes.size() != owners.size() || !affNodes.containsAll(owners)) {
-                                LT.warn(log(), null, "Waiting for topology map update [grid=" + g.name() +
-                                    ", cache=" + cfg.getName() + ", p=" + p + ", nodes=" + exp + ", owners=" + actual +
-                                    ", affNodes=" + affNodes + ", owners=" + owners +
-                                    ", locNode=" + g.cluster().localNode().id() + ']');
+                                LT.warn(log(), null, "Waiting for topology map update [" +
+                                    "grid=" + g.name() +
+                                    ", cache=" + cfg.getName() +
+                                    ", cacheId=" + dht.context().cacheId() +
+                                    ", p=" + p +
+                                    ", affNodesCnt=" + exp +
+                                    ", ownersCnt=" + actual +
+                                    ", affNodes=" + affNodes +
+                                    ", owners=" + owners +
+                                    ", locNode=" + g.cluster().localNode() + ']');
 
                                 if (i == 0)
                                     start = System.currentTimeMillis();
 
                                 if (System.currentTimeMillis() - start > 30_000)
-                                    throw new IgniteException("Timeout of waiting for topology map update [grid="
-                                        + g.name() + ", p=" + p + ", nodes=" + exp + ", owners=" + actual +
-                                            ", affNodes=" + affNodes + ", owners=" + owners + ", locNode="
-                                        + g.cluster().localNode().id() + ']');
+                                    throw new IgniteException("Timeout of waiting for topology map update [" +
+                                        "grid=" + g.name() +
+                                        ", cache=" + cfg.getName() +
+                                        ", cacheId=" + dht.context().cacheId() +
+                                        ", p=" + p +
+                                        ", affNodesCnt=" + exp +
+                                        ", ownersCnt=" + actual +
+                                        ", affNodes=" + affNodes +
+                                        ", owners=" + owners +
+                                        ", locNode=" + g.cluster().localNode() + ']');
 
                                 Thread.sleep(200); // Busy wait.
 
@@ -415,6 +428,38 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
     }
 
     /**
+     * @param ignite Node.
+     */
+    public void dumpCacheDebugInfo(Ignite ignite) {
+        GridKernalContext ctx = ((IgniteKernal)ignite).context();
+
+        log.error("Cache information update [node=" + ignite.name() +
+            ", client=" + ignite.configuration().isClientMode() + ']');
+
+        GridCacheSharedContext cctx = ctx.cache().context();
+
+        log.error("Pending transactions:");
+
+        for (IgniteInternalTx tx : cctx.tm().activeTransactions())
+            log.error(">>> " + tx);
+
+        log.error("Pending explicit locks:");
+
+        for (GridCacheExplicitLockSpan lockSpan : cctx.mvcc().activeExplicitLocks())
+            log.error(">>> " + lockSpan);
+
+        log.error("Pending cache futures:");
+
+        for (GridCacheFuture<?> fut : cctx.mvcc().activeFutures())
+            log.error(">>> " + fut);
+
+        log.error("Pending atomic cache futures:");
+
+        for (GridCacheFuture<?> fut : cctx.mvcc().atomicFutures())
+            log.error(">>> " + fut);
+    }
+
+    /**
      * @param cache Cache.
      * @return Affinity.
      */


[08/50] [abbrv] incubator-ignite git commit: #IGNITE-857 Fixed review notes.

Posted by ag...@apache.org.
#IGNITE-857 Fixed review notes.


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

Branch: refs/heads/ignite-389
Commit: 73c519e4fae403c4592cfdf75158ad1df5cdf8c0
Parents: 3a0b5da
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 18:12:59 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 18:12:59 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/mesos/IgniteScheduler.java   | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/73c519e4/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 263123c..6bd3aa7 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -104,8 +104,10 @@ public class IgniteScheduler implements Scheduler {
                     Collections.singletonList(task),
                     Protos.Filters.newBuilder().setRefuseSeconds(1).build());
             }
-            catch (Exception e) {
+            catch (RuntimeException e) {
                 log.error("Failed launch task. Task id: {}. Task info: {}", taskId, task);
+
+                throw e;
             }
 
             synchronized (mux) {


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

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-840' 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/9a16d190
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/9a16d190
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/9a16d190

Branch: refs/heads/ignite-389
Commit: 9a16d1906b74e918a377cd9d5da6d96c8aeeaaad
Parents: 00848cc 9f9a631
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 1 12:45:01 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 1 12:45:01 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheEntryEx.java    | 6 ++++++
 .../ignite/internal/processors/cache/GridCacheMapEntry.java   | 5 +++++
 .../cache/distributed/GridDistributedCacheEntry.java          | 7 -------
 .../processors/cache/distributed/dht/GridDhtCacheEntry.java   | 6 +-----
 .../internal/processors/datastreamer/DataStreamerImpl.java    | 2 ++
 .../internal/processors/cache/GridCacheTestEntryEx.java       | 4 ++++
 .../datastreamer/DataStreamerMultiThreadedSelfTest.java       | 2 --
 7 files changed, 18 insertions(+), 14 deletions(-)
----------------------------------------------------------------------



[14/50] [abbrv] incubator-ignite git commit: # ignite-709 Fix tests

Posted by ag...@apache.org.
# ignite-709 Fix tests


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

Branch: refs/heads/ignite-389
Commit: 2799c3a69c2ddfb0b7c5f8f1c88808948fa40664
Parents: cfe5621
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 19:26:44 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 19:26:44 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAbstractFullApiSelfTest.java        | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2799c3a6/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 70d8f9c..5c31396 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
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.swapspace.inmemory.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.transactions.*;
@@ -130,6 +131,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         if (memoryMode() == OFFHEAP_TIERED || memoryMode() == OFFHEAP_VALUES)
             cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
 


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

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-943' 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/3c0046e8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/3c0046e8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/3c0046e8

Branch: refs/heads/ignite-389
Commit: 3c0046e8e040ab53a5f39819b1c86487f645d25f
Parents: 9a16d19 ab4c18c
Author: sevdokimov <se...@gridgain.com>
Authored: Mon Jun 1 14:42:34 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Mon Jun 1 14:42:34 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   27 +-
 .../configuration/IgniteConfiguration.java      |    4 +-
 .../ignite/internal/ClusterMetricsSnapshot.java |   14 +
 .../internal/GridEventConsumeHandler.java       |  100 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../apache/ignite/internal/IgniteKernal.java    |   23 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   20 +-
 .../ignite/internal/MarshallerContextImpl.java  |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |   18 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   48 +
 .../discovery/GridDiscoveryManager.java         |  214 +-
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../cache/DynamicCacheChangeBatch.java          |   20 +-
 .../cache/DynamicCacheDescriptor.java           |    2 +
 .../processors/cache/GridCacheAdapter.java      |   17 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |    6 +-
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   71 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |   74 +-
 .../processors/cache/GridCacheProxyImpl.java    |   12 +
 .../cache/GridCacheSharedContext.java           |    1 +
 .../processors/cache/GridCacheUtils.java        |  234 +-
 .../processors/cache/IgniteInternalCache.java   |    5 +
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../dht/GridClientPartitionTopology.java        |    8 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   22 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   78 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    4 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  313 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   28 +-
 .../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      |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |   12 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   15 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   54 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  836 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../service/GridServiceProcessor.java           |  121 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    2 +-
 .../communication/tcp/TcpCommunicationSpi.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    | 4766 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  170 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5799 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   10 +-
 .../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 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 .../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 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    3 +
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../GridCacheReturnValueTransferSelfTest.java   |    3 +
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  190 +
 .../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 +
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   47 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |   25 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   93 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../igfs/IgfsClientCacheSelfTest.java           |    3 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |   16 +-
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../discovery/AbstractDiscoverySelfTest.java    |    8 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1171 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/common/GridCommonAbstractTest.java   |   59 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |   41 -
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    3 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 214 files changed, 17497 insertions(+), 10220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3c0046e8/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------


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

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


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

Branch: refs/heads/ignite-389
Commit: cfe56213e1dfc011de47e0425627b3ff73a57763
Parents: 9cddb6f b6fc8a9
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 18:02:47 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 18:02:47 2015 +0300

----------------------------------------------------------------------
 dev-tools/gradle/wrapper/gradle-wrapper.jar     | Bin 51017 -> 0 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |  18 +-
 dev-tools/gradlew                               | 163 ++++++++----
 dev-tools/slurp.sh                              |   2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  73 ++++--
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   5 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../org/apache/ignite/services/Service.java     |   5 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  20 +-
 .../service/ClosureServiceClientsNodesTest.java | 245 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   4 +
 .../query/h2/sql/BaseH2CompareQueryTest.java    |  16 ++
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 .../licenses/jcraft-revised-bsd.txt             |  28 ---
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 47 files changed, 498 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


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

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


[18/50] [abbrv] incubator-ignite git commit: # ignite-709 can not skip preload on server nodes

Posted by ag...@apache.org.
# ignite-709 can not skip preload on server nodes


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

Branch: refs/heads/ignite-389
Commit: 2c3ce158de266dfd02c91e3e6e377ce667fd598f
Parents: 581f4d9
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 29 10:35:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 29 10:35:24 2015 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        |   2 +-
 .../IgniteCacheClientNodeConcurrentStart.java   | 105 +++++++++++++++++++
 ...teCacheClientNodePartitionsExchangeTest.java |   6 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   1 +
 4 files changed, 110 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3ce158/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 162c7b3..145def8 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
@@ -517,7 +517,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
                         onDone(exchId.topologyVersion());
 
-                        skipPreload = true;
+                        skipPreload = cctx.kernalContext().clientNode();
 
                         return;
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3ce158/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java
new file mode 100644
index 0000000..bd74ece
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.*;
+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.testframework.junits.common.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheClientNodeConcurrentStart extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES_CNT = 5;
+
+    /** */
+    private Set<Integer> clientNodes;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        assertNotNull(clientNodes);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        boolean client = false;
+
+        for (Integer clientIdx : clientNodes) {
+            if (getTestGridName(clientIdx).equals(gridName)) {
+                client = true;
+
+                break;
+            }
+        }
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setBackups(0);
+        ccfg.setRebalanceMode(SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentStart() throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 3; i++) {
+            try {
+                clientNodes = new HashSet<>();
+
+                while (clientNodes.size() < 2)
+                    clientNodes.add(rnd.nextInt(0, NODES_CNT));
+
+                clientNodes.add(NODES_CNT - 1);
+
+                log.info("Test iteration [iter=" + i + ", clients=" + clientNodes + ']');
+
+                startGridsMultiThreaded(NODES_CNT, true);
+
+                for (int node : clientNodes) {
+                    Ignite ignite = grid(node);
+
+                    assertTrue(ignite.configuration().isClientMode());
+                }
+            }
+            finally {
+                stopAllGrids();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3ce158/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
index 162aa81..68ae211 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
@@ -158,11 +158,11 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
 
         Ignite ignite1 = startGrid(1);
 
-        assertFalse(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
 
         ignite1.close();
 
-        assertFalse(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
 
         ignite1 = startGrid(1);
 
@@ -178,7 +178,7 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
             }
         }, EventType.EVT_CACHE_REBALANCE_STARTED, EventType.EVT_CACHE_REBALANCE_STOPPED);
 
-        assertFalse(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
 
         client = false;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3ce158/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 4664c66..037af9d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -136,6 +136,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(IgniteCachePartitionMapUpdateTest.class));
         suite.addTest(new TestSuite(IgniteCacheClientNodePartitionsExchangeTest.class));
         suite.addTest(new TestSuite(IgniteCacheClientNodeChangingTopologyTest.class));
+        suite.addTest(new TestSuite(IgniteCacheClientNodeConcurrentStart.class));
 
         return suite;
     }


[47/50] [abbrv] incubator-ignite git commit: IGNITE-943 Fix javadoc problems.

Posted by ag...@apache.org.
IGNITE-943 Fix javadoc problems.


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

Branch: refs/heads/ignite-389
Commit: 97d0b04e268985bd5afc32a35cd432b04e5d30a8
Parents: 3c0046e
Author: sevdokimov <se...@gridgain.com>
Authored: Mon Jun 1 15:29:54 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Mon Jun 1 15:29:54 2015 +0300

----------------------------------------------------------------------
 modules/core/src/main/java/org/apache/ignite/Ignition.java       | 4 ++--
 .../org/apache/ignite/configuration/IgniteConfiguration.java     | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/97d0b04e/modules/core/src/main/java/org/apache/ignite/Ignition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignition.java b/modules/core/src/main/java/org/apache/ignite/Ignition.java
index 35e0b51..d5d59a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignition.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignition.java
@@ -143,7 +143,7 @@ public class Ignition {
      *
      * @param clientMode Client mode flag.
      * @see IgniteConfiguration#isClientMode()
-     * @see TcpDiscoverySpi#setClientMode(boolean)
+     * @see TcpDiscoverySpi#setForceServerMode(boolean)
      */
     public static void setClientMode(boolean clientMode) {
         IgnitionEx.setClientMode(clientMode);
@@ -158,7 +158,7 @@ public class Ignition {
      *
      * @return Client mode flag.
      * @see IgniteConfiguration#isClientMode()
-     * @see TcpDiscoverySpi#setClientMode(boolean)
+     * @see TcpDiscoverySpi#setForceServerMode(boolean)
      */
     public static boolean isClientMode() {
         return IgnitionEx.isClientMode();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/97d0b04e/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 38e1f2c..71298f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -1827,7 +1827,7 @@ public class IgniteConfiguration {
      * {@link DiscoverySpi} in client mode if this property is {@code true}.
      *
      * @return Client mode flag.
-     * @see TcpDiscoverySpi#setClientMode(boolean)
+     * @see TcpDiscoverySpi#setForceServerMode(boolean)
      */
     public Boolean isClientMode() {
         return clientMode;


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

Posted by ag...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite 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/6cd1a6e5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/6cd1a6e5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/6cd1a6e5

Branch: refs/heads/ignite-389
Commit: 6cd1a6e55880571e09785afc824e258070242284
Parents: 2d9a938 5df0668
Author: agura <ag...@gridgain.com>
Authored: Fri May 29 16:59:49 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Fri May 29 16:59:49 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/GridJavaProcess.java   | 30 ++++++++++++--------
 1 file changed, 18 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



[05/50] [abbrv] incubator-ignite git commit: # ignite-857 review

Posted by ag...@apache.org.
# ignite-857 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/9cddb6fe
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/9cddb6fe
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/9cddb6fe

Branch: refs/heads/ignite-389
Commit: 9cddb6fe669ef9f4ac6792a306a665fa2037f773
Parents: 3dea5f1
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu May 28 17:58:49 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu May 28 17:58:49 2015 +0300

----------------------------------------------------------------------
 .../managers/discovery/GridDiscoveryManager.java  |  3 ++-
 .../apache/ignite/spi/discovery/DiscoverySpi.java |  3 ++-
 .../ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 18 ++++++++++++------
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java   |  2 +-
 ...itionedClientOnlyNoPrimaryFullApiSelfTest.java |  3 ++-
 5 files changed, 19 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cddb6fe/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 a910950..4ef602e 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
@@ -285,7 +285,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** {@inheritDoc} */
     @Override protected void onKernalStart0() throws IgniteCheckedException {
         if (Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode())
-            ctx.performance().add("Enable client mode for TcpDiscoverySpi (set TcpDiscoverySpi.forceServerMode to true)");
+            ctx.performance().add("Enable client mode for TcpDiscoverySpi " +
+                    "(set TcpDiscoverySpi.forceServerMode to false)");
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cddb6fe/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index e7fbadc..b952087 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -78,7 +78,8 @@ public interface DiscoverySpi extends IgniteSpi {
     /**
      * Sets node attributes and node version which will be distributed in grid during
      * join process. Note that these attributes cannot be changed and set only once.
-     *  @param attrs Map of node attributes.
+     *
+     * @param attrs Map of node attributes.
      * @param ver Product version.
      */
     public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cddb6fe/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 52ea78c..64e6001 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -54,10 +54,12 @@ import java.util.concurrent.atomic.*;
  * done across it.
  * <p>
  * If node is configured as client node (see {@link IgniteConfiguration#clientMode})
- * TcpDiscoverySpi starts in client mode too. In this case node does not insert to the ring,
- * it connects to any node in the ring router and communicated with that node only.
- * Thereby slowing or shutdown of client node will not affect whole cluster. If you want to start TcpDiscoverySpi in
- * server mode regardless {@link IgniteConfiguration#clientMode} you can set {@link #forceSrvMode} to true.
+ * TcpDiscoverySpi starts in client mode as well. In this case node does not take its place in the ring,
+ * but it connects to random node in the ring (IP taken from IP finder configured) and
+ * use it as a router for discovery traffic.
+ * Therefore slow client node or its shutdown will not affect whole cluster. If TcpDiscoverySpi
+ * needs to be started in server mode regardless of {@link IgniteConfiguration#clientMode},
+ * {@link #forceSrvMode} should be set to true.
  * <p>
  * At startup SPI tries to send messages to random IP taken from
  * {@link TcpDiscoveryIpFinder} about self start (stops when send succeeds)
@@ -388,11 +390,15 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /**
-     * If {@code true} TcpDiscoverySpi will started in server mode regardless
-     * of {@link IgniteConfiguration#isClientMode()}
+     * Sets force server mode flag.
+     * <p>
+     * If {@code true} TcpDiscoverySpi is started in server mode regardless
+     * of {@link IgniteConfiguration#isClientMode()}.
      *
      * @param forceSrvMode forceServerMode flag.
+     * @return {@code this} for chaining.
      */
+    @IgniteSpiConfiguration(optional = true)
     public TcpDiscoverySpi setForceServerMode(boolean forceSrvMode) {
         this.forceSrvMode = forceSrvMode;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cddb6fe/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
index 6f2ea6f..f338fab 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
@@ -277,7 +277,7 @@ public interface TcpDiscoverySpiMBean extends IgniteSpiManagementMBean {
      * Whether or not discovery is started in client mode.
      *
      * @return {@code true} if node is in client mode.
-     * @throws IllegalStateException If discovery SPI has not started.
+     * @throws IllegalStateException If discovery SPI is not started.
      */
     @MXBeanDescription("Client mode.")
     public boolean isClientMode() throws IllegalStateException;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cddb6fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
index 931e347..6b7d1ad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
@@ -65,7 +65,8 @@ public class GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest extends Grid
     @Override protected IgniteClosure<Throwable, Throwable> errorHandler() {
         return new IgniteClosure<Throwable, Throwable>() {
             @Override public Throwable apply(Throwable e) {
-                if (e instanceof IgniteException || e instanceof IgniteCheckedException || X.hasCause(e, ClusterTopologyCheckedException.class)) {
+                if (e instanceof IgniteException || e instanceof IgniteCheckedException ||
+                    X.hasCause(e, ClusterTopologyCheckedException.class)) {
                     info("Discarding exception: " + e);
 
                     return null;


[25/50] [abbrv] incubator-ignite git commit: [IGNITE-958]: IGNITE-218 (Wrong staging permissions while running MR job under hadoop accelerator): IGFS part.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java
index 7dca049..f23c62c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java
@@ -81,6 +81,9 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
     /** IGFS name. */
     private final String igfs;
 
+    /** The user this out proc is performing on behalf of. */
+    private final String userName;
+
     /** Client log. */
     private final Log log;
 
@@ -100,8 +103,8 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
      * @param log Client logger.
      * @throws IOException If failed.
      */
-    public HadoopIgfsOutProc(String host, int port, String grid, String igfs, Log log) throws IOException {
-        this(host, port, grid, igfs, false, log);
+    public HadoopIgfsOutProc(String host, int port, String grid, String igfs, Log log, String user) throws IOException {
+        this(host, port, grid, igfs, false, log, user);
     }
 
     /**
@@ -113,8 +116,8 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
      * @param log Client logger.
      * @throws IOException If failed.
      */
-    public HadoopIgfsOutProc(int port, String grid, String igfs, Log log) throws IOException {
-        this(null, port, grid, igfs, true, log);
+    public HadoopIgfsOutProc(int port, String grid, String igfs, Log log, String user) throws IOException {
+        this(null, port, grid, igfs, true, log, user);
     }
 
     /**
@@ -128,7 +131,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
      * @param log Client logger.
      * @throws IOException If failed.
      */
-    private HadoopIgfsOutProc(String host, int port, String grid, String igfs, boolean shmem, Log log)
+    private HadoopIgfsOutProc(String host, int port, String grid, String igfs, boolean shmem, Log log, String user)
         throws IOException {
         assert host != null && !shmem || host == null && shmem :
             "Invalid arguments [host=" + host + ", port=" + port + ", shmem=" + shmem + ']';
@@ -138,6 +141,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         this.grid = grid;
         this.igfs = igfs;
         this.log = log;
+        this.userName = IgfsUtils.fixUserName(user);
 
         io = HadoopIgfsIpcIo.get(log, endpoint);
 
@@ -173,6 +177,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
 
         msg.command(INFO);
         msg.path(path);
+        msg.userName(userName);
 
         return io.send(msg).chain(FILE_RES).get();
     }
@@ -184,6 +189,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.command(UPDATE);
         msg.path(path);
         msg.properties(props);
+        msg.userName(userName);
 
         return io.send(msg).chain(FILE_RES).get();
     }
@@ -196,6 +202,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.path(path);
         msg.accessTime(accessTime);
         msg.modificationTime(modificationTime);
+        msg.userName(userName);
 
         return io.send(msg).chain(BOOL_RES).get();
     }
@@ -207,6 +214,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.command(RENAME);
         msg.path(src);
         msg.destinationPath(dest);
+        msg.userName(userName);
 
         return io.send(msg).chain(BOOL_RES).get();
     }
@@ -218,6 +226,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.command(DELETE);
         msg.path(path);
         msg.flag(recursive);
+        msg.userName(userName);
 
         return io.send(msg).chain(BOOL_RES).get();
     }
@@ -231,6 +240,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.path(path);
         msg.start(start);
         msg.length(len);
+        msg.userName(userName);
 
         return io.send(msg).chain(BLOCK_LOCATION_COL_RES).get();
     }
@@ -241,6 +251,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
 
         msg.command(PATH_SUMMARY);
         msg.path(path);
+        msg.userName(userName);
 
         return io.send(msg).chain(SUMMARY_RES).get();
     }
@@ -252,6 +263,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.command(MAKE_DIRECTORIES);
         msg.path(path);
         msg.properties(props);
+        msg.userName(userName);
 
         return io.send(msg).chain(BOOL_RES).get();
     }
@@ -262,6 +274,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
 
         msg.command(LIST_FILES);
         msg.path(path);
+        msg.userName(userName);
 
         return io.send(msg).chain(FILE_COL_RES).get();
     }
@@ -272,6 +285,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
 
         msg.command(LIST_PATHS);
         msg.path(path);
+        msg.userName(userName);
 
         return io.send(msg).chain(PATH_COL_RES).get();
     }
@@ -288,6 +302,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.command(OPEN_READ);
         msg.path(path);
         msg.flag(false);
+        msg.userName(userName);
 
         IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
 
@@ -303,6 +318,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.path(path);
         msg.flag(true);
         msg.sequentialReadsBeforePrefetch(seqReadsBeforePrefetch);
+        msg.userName(userName);
 
         IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
 
@@ -321,6 +337,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.properties(props);
         msg.replication(replication);
         msg.blockSize(blockSize);
+        msg.userName(userName);
 
         Long streamId = io.send(msg).chain(LONG_RES).get();
 
@@ -336,6 +353,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.path(path);
         msg.flag(create);
         msg.properties(props);
+        msg.userName(userName);
 
         Long streamId = io.send(msg).chain(LONG_RES).get();
 
@@ -471,4 +489,9 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
             }
         };
     }
+
+    /** {@inheritDoc} */
+    @Override public String user() {
+        return userName;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
index 1dada21..7d0db49 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
@@ -55,6 +55,9 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
     /** Logger. */
     private final Log log;
 
+    /** The user name this wrapper works on behalf of. */
+    private final String userName;
+
     /**
      * Constructor.
      *
@@ -63,13 +66,15 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
      * @param conf Configuration.
      * @param log Current logger.
      */
-    public HadoopIgfsWrapper(String authority, String logDir, Configuration conf, Log log) throws IOException {
+    public HadoopIgfsWrapper(String authority, String logDir, Configuration conf, Log log, String user)
+            throws IOException {
         try {
             this.authority = authority;
             this.endpoint = new HadoopIgfsEndpoint(authority);
             this.logDir = logDir;
             this.conf = conf;
             this.log = log;
+            this.userName = user;
         }
         catch (IgniteCheckedException e) {
             throw new IOException("Failed to parse endpoint: " + authority, e);
@@ -362,13 +367,14 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
                 HadoopIgfsEx hadoop = null;
 
                 try {
-                    hadoop = new HadoopIgfsInProc(igfs, log);
+                    hadoop = new HadoopIgfsInProc(igfs, log, userName);
 
                     curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
                 }
                 catch (IOException | IgniteCheckedException e) {
                     if (e instanceof HadoopIgfsCommunicationException)
-                        hadoop.close(true);
+                        if (hadoop != null)
+                            hadoop.close(true);
 
                     if (log.isDebugEnabled())
                         log.debug("Failed to connect to in-proc IGFS, fallback to IPC mode.", e);
@@ -384,7 +390,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
                 HadoopIgfsEx hadoop = null;
 
                 try {
-                    hadoop = new HadoopIgfsOutProc(endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
+                    hadoop = new HadoopIgfsOutProc(endpoint.port(), endpoint.grid(), endpoint.igfs(), log, userName);
 
                     curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
                 }
@@ -409,7 +415,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
 
                 try {
                     hadoop = new HadoopIgfsOutProc(LOCALHOST, endpoint.port(), endpoint.grid(), endpoint.igfs(),
-                        log);
+                        log, userName);
 
                     curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
                 }
@@ -430,7 +436,8 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
             HadoopIgfsEx hadoop = null;
 
             try {
-                hadoop = new HadoopIgfsOutProc(endpoint.host(), endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
+                hadoop = new HadoopIgfsOutProc(endpoint.host(), endpoint.port(), endpoint.grid(), endpoint.igfs(),
+                    log, userName);
 
                 curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/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 e9c859bd..dd18c66 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
@@ -239,9 +239,7 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
         Thread.currentThread().setContextClassLoader(jobConf().getClassLoader());
 
         try {
-            FileSystem fs = FileSystem.get(jobConf());
-
-            HadoopFileSystemsUtils.setUser(fs, jobConf().getUser());
+            FileSystem.get(jobConf());
 
             LocalFileSystem locFs = FileSystem.getLocal(jobConf());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
index d11cabb..9bcd5de 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.security.*;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
@@ -39,6 +40,7 @@ import org.jsr166.*;
 
 import java.io.*;
 import java.net.*;
+import java.security.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -58,6 +60,9 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
     /** Thread count for multithreaded tests. */
     private static final int THREAD_CNT = 8;
 
+    /** Secondary file system user. */
+    private static final String SECONDARY_FS_USER = "secondary-default";
+
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
@@ -255,7 +260,7 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         if (mode != PRIMARY)
             cfg.setSecondaryFileSystem(new IgniteHadoopIgfsSecondaryFileSystem(secondaryFileSystemUriPath(),
-                secondaryFileSystemConfigPath()));
+                secondaryFileSystemConfigPath(), SECONDARY_FS_USER));
 
         cfg.setIpcEndpointConfiguration(primaryIpcEndpointConfiguration(gridName));
         cfg.setManagementPort(-1);
@@ -278,11 +283,28 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         primaryFsCfg.addResource(U.resolveIgniteUrl(primaryFileSystemConfigPath()));
 
-        fs = AbstractFileSystem.get(primaryFsUri, primaryFsCfg);
+        UserGroupInformation ugi = UserGroupInformation.getBestUGI(null, getClientFsUser());
+
+        // Create Fs on behalf of the client user:
+        ugi.doAs(new PrivilegedExceptionAction<Object>() {
+            @Override public Object run() throws Exception {
+                fs = AbstractFileSystem.get(primaryFsUri, primaryFsCfg);
+
+                return null;
+            }
+        });
 
         barrier = new CyclicBarrier(THREAD_CNT);
     }
 
+    /**
+     * Gets the user the Fs client operates on bahalf of.
+     * @return The user the Fs client operates on bahalf of.
+     */
+    protected String getClientFsUser() {
+        return "foo";
+    }
+
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         try {
@@ -297,14 +319,17 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
     /** @throws Exception If failed. */
     public void testStatus() throws Exception {
+        Path file1 = new Path("/file1");
 
-        try (FSDataOutputStream file = fs.create(new Path("/file1"), EnumSet.noneOf(CreateFlag.class),
+        try (FSDataOutputStream file = fs.create(file1, EnumSet.noneOf(CreateFlag.class),
             Options.CreateOpts.perms(FsPermission.getDefault()))) {
             file.write(new byte[1024 * 1024]);
         }
 
         FsStatus status = fs.getFsStatus();
 
+        assertEquals(getClientFsUser(), fs.getFileStatus(file1).getOwner());
+
         assertEquals(4, grid(0).cluster().nodes().size());
 
         long used = 0, max = 0;
@@ -707,6 +732,8 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         os.close();
 
+        assertEquals(getClientFsUser(), fs.getFileStatus(file).getOwner());
+
         fs.setOwner(file, "aUser", "aGroup");
 
         assertEquals("aUser", fs.getFileStatus(file).getOwner());
@@ -796,20 +823,20 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         int cnt = 2 * 1024;
 
-        FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
+        try (FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()))) {
 
-        for (long i = 0; i < cnt; i++)
-            out.writeLong(i);
+            for (long i = 0; i < cnt; i++)
+                out.writeLong(i);
+        }
 
-        out.close();
+        assertEquals(getClientFsUser(), fs.getFileStatus(file).getOwner());
 
-        FSDataInputStream in = fs.open(file, 1024);
+        try (FSDataInputStream in = fs.open(file, 1024)) {
 
-        for (long i = 0; i < cnt; i++)
-            assertEquals(i, in.readLong());
-
-        in.close();
+            for (long i = 0; i < cnt; i++)
+                assertEquals(i, in.readLong());
+        }
     }
 
     /** @throws Exception If failed. */
@@ -1191,6 +1218,9 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         assertEquals(dirPerm, fs.getFileStatus(dir).getPermission());
         assertEquals(nestedDirPerm, fs.getFileStatus(nestedDir).getPermission());
+
+        assertEquals(getClientFsUser(), fs.getFileStatus(dir).getOwner());
+        assertEquals(getClientFsUser(), fs.getFileStatus(nestedDir).getOwner());
     }
 
     /** @throws Exception If failed. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
index 9e84c51..b089995 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
@@ -162,9 +162,9 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
             primaryConfFullPath = null;
 
         SecondaryFileSystemProvider provider =
-            new SecondaryFileSystemProvider(primaryFsUriStr, primaryConfFullPath, null);
+            new SecondaryFileSystemProvider(primaryFsUriStr, primaryConfFullPath);
 
-        primaryFs = provider.createFileSystem();
+        primaryFs = provider.createFileSystem(null);
 
         primaryFsUri = provider.uri();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
index d9a3c59..b828aad 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.security.*;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
@@ -43,6 +44,7 @@ import org.jsr166.*;
 import java.io.*;
 import java.lang.reflect.*;
 import java.net.*;
+import java.security.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -72,6 +74,9 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
     /** Secondary file system configuration path. */
     private static final String SECONDARY_CFG_PATH = "/work/core-site-test.xml";
 
+    /** Secondary file system user. */
+    private static final String SECONDARY_FS_USER = "secondary-default";
+
     /** Secondary endpoint configuration. */
     protected static final IgfsIpcEndpointConfiguration SECONDARY_ENDPOINT_CFG;
 
@@ -145,6 +150,14 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
         endpoint = skipLocShmem ? "127.0.0.1:10500" : "shmem:10500";
     }
 
+    /**
+     * Gets the user the Fs client operates on bahalf of.
+     * @return The user the Fs client operates on bahalf of.
+     */
+    protected String getClientFsUser() {
+        return "foo";
+    }
+
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         Configuration secondaryConf = configuration(SECONDARY_AUTHORITY, true, true);
@@ -235,7 +248,17 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
         primaryFsCfg = configuration(PRIMARY_AUTHORITY, skipEmbed, skipLocShmem);
 
-        fs = FileSystem.get(primaryFsUri, primaryFsCfg);
+        UserGroupInformation clientUgi = UserGroupInformation.getBestUGI(null, getClientFsUser());
+        assertNotNull(clientUgi);
+
+        // Create the Fs on behalf of the specific user:
+        clientUgi.doAs(new PrivilegedExceptionAction<Object>() {
+            @Override public Object run() throws Exception {
+                fs = FileSystem.get(primaryFsUri, primaryFsCfg);
+
+                return null;
+            }
+        });
 
         barrier = new CyclicBarrier(THREAD_CNT);
     }
@@ -324,7 +347,8 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
         cfg.setDefaultMode(mode);
 
         if (mode != PRIMARY)
-            cfg.setSecondaryFileSystem(new IgniteHadoopIgfsSecondaryFileSystem(SECONDARY_URI, SECONDARY_CFG_PATH));
+            cfg.setSecondaryFileSystem(new IgniteHadoopIgfsSecondaryFileSystem(
+                SECONDARY_URI, SECONDARY_CFG_PATH, SECONDARY_FS_USER));
 
         cfg.setIpcEndpointConfiguration(primaryIpcEndpointConfiguration(gridName));
 
@@ -870,6 +894,8 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
         os.close();
 
+        assertEquals(getClientFsUser(), fs.getFileStatus(file).getOwner());
+
         fs.setOwner(file, "aUser", "aGroup");
 
         assertEquals("aUser", fs.getFileStatus(file).getOwner());
@@ -1001,19 +1027,19 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
         int cnt = 2 * 1024;
 
-        FSDataOutputStream out = fs.create(file, true, 1024);
-
-        for (long i = 0; i < cnt; i++)
-            out.writeLong(i);
+        try (FSDataOutputStream out = fs.create(file, true, 1024)) {
 
-        out.close();
+            for (long i = 0; i < cnt; i++)
+                out.writeLong(i);
+        }
 
-        FSDataInputStream in = fs.open(file, 1024);
+        assertEquals(getClientFsUser(), fs.getFileStatus(file).getOwner());
 
-        for (long i = 0; i < cnt; i++)
-            assertEquals(i, in.readLong());
+        try (FSDataInputStream in = fs.open(file, 1024)) {
 
-        in.close();
+            for (long i = 0; i < cnt; i++)
+                assertEquals(i, in.readLong());
+        }
     }
 
     /** @throws Exception If failed. */
@@ -1344,7 +1370,7 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
         String path = fs.getFileStatus(file).getPath().toString();
 
-        assertTrue(path.endsWith("/user/" + System.getProperty("user.name", "anonymous") + "/file"));
+        assertTrue(path.endsWith("/user/" + getClientFsUser() + "/file"));
     }
 
     /** @throws Exception If failed. */
@@ -1374,7 +1400,7 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
     public void testGetWorkingDirectoryIfDefault() throws Exception {
         String path = fs.getWorkingDirectory().toString();
 
-        assertTrue(path.endsWith("/user/" + System.getProperty("user.name", "anonymous")));
+        assertTrue(path.endsWith("/user/" + getClientFsUser()));
     }
 
     /** @throws Exception If failed. */
@@ -1412,17 +1438,20 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
     @SuppressWarnings("OctalInteger")
     public void testMkdirs() throws Exception {
         Path fsHome = new Path(PRIMARY_URI);
-        Path dir = new Path(fsHome, "/tmp/staging");
-        Path nestedDir = new Path(dir, "nested");
+        final Path dir = new Path(fsHome, "/tmp/staging");
+        final Path nestedDir = new Path(dir, "nested");
 
-        FsPermission dirPerm = FsPermission.createImmutable((short)0700);
-        FsPermission nestedDirPerm = FsPermission.createImmutable((short)111);
+        final FsPermission dirPerm = FsPermission.createImmutable((short)0700);
+        final FsPermission nestedDirPerm = FsPermission.createImmutable((short)111);
 
         assertTrue(fs.mkdirs(dir, dirPerm));
         assertTrue(fs.mkdirs(nestedDir, nestedDirPerm));
 
         assertEquals(dirPerm, fs.getFileStatus(dir).getPermission());
         assertEquals(nestedDirPerm, fs.getFileStatus(nestedDir).getPermission());
+
+        assertEquals(getClientFsUser(), fs.getFileStatus(dir).getOwner());
+        assertEquals(getClientFsUser(), fs.getFileStatus(nestedDir).getOwner());
     }
 
     /** @throws Exception If failed. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
index b92b213..fcfd587 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
@@ -125,7 +125,7 @@ public class IgniteHadoopFileSystemClientSelfTest extends IgfsCommonAbstractTest
         try {
             switchHandlerErrorFlag(true);
 
-            HadoopIgfs client = new HadoopIgfsOutProc("127.0.0.1", 10500, getTestGridName(0), "igfs", LOG);
+            HadoopIgfs client = new HadoopIgfsOutProc("127.0.0.1", 10500, getTestGridName(0), "igfs", LOG, null);
 
             client.handshake(null);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
index e103c5f..2c17ba9 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
@@ -144,6 +144,8 @@ public class IgniteHadoopFileSystemIpcCacheSelfTest extends IgfsCommonAbstractTe
 
         Map<String, HadoopIgfsIpcIo> cache = (Map<String, HadoopIgfsIpcIo>)cacheField.get(null);
 
+        cache.clear(); // avoid influence of previous tests in the same process.
+
         String name = "igfs:" + getTestGridName(0) + "@";
 
         Configuration cfg = new Configuration();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
index 8cf31a2..5f90bd4 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
@@ -33,6 +33,7 @@ import java.util.concurrent.atomic.*;
  * Test file systems for the working directory multi-threading support.
  */
 public class HadoopFileSystemsTest extends HadoopAbstractSelfTest {
+    /** the number of threads */
     private static final int THREAD_COUNT = 3;
 
     /** {@inheritDoc} */
@@ -87,10 +88,6 @@ public class HadoopFileSystemsTest extends HadoopAbstractSelfTest {
                 try {
                     int curThreadNum = threadNum.getAndIncrement();
 
-                    FileSystem fs = FileSystem.get(uri, cfg);
-
-                    HadoopFileSystemsUtils.setUser(fs, "user" + curThreadNum);
-
                     if ("file".equals(uri.getScheme()))
                         FileSystem.get(uri, cfg).setWorkingDirectory(new Path("file:///user/user" + curThreadNum));
 
@@ -149,24 +146,6 @@ public class HadoopFileSystemsTest extends HadoopAbstractSelfTest {
     }
 
     /**
-     * Test IGFS multi-thread working directory.
-     *
-     * @throws Exception If fails.
-     */
-    public void testIgfs() throws Exception {
-        testFileSystem(URI.create(igfsScheme()));
-    }
-
-    /**
-     * Test HDFS multi-thread working directory.
-     *
-     * @throws Exception If fails.
-     */
-    public void testHdfs() throws Exception {
-        testFileSystem(URI.create("hdfs://localhost/"));
-    }
-
-    /**
      * Test LocalFS multi-thread working directory.
      *
      * @throws Exception If fails.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
index 8a046e0..89bf830 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
@@ -61,10 +61,10 @@ public class HadoopSkipListSelfTest extends HadoopAbstractMapTest {
 
             int sigma = max((int)ceil(precission * exp), 5);
 
-            X.println("Level: " + level + " exp: " + exp + " act: " + levelsCnts[level] + " precission: " + precission +
+            X.println("Level: " + level + " exp: " + exp + " act: " + levelsCnts[level] + " precision: " + precission +
                 " sigma: " + sigma);
 
-            assertTrue(abs(exp - levelsCnts[level]) <= sigma);
+            assertTrue(abs(exp - levelsCnts[level]) <= sigma); // Sometimes fails.
         }
     }
 


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

Posted by ag...@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/56bda960
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/56bda960
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/56bda960

Branch: refs/heads/ignite-389
Commit: 56bda960f47fc0b929c8cd5b2bfcb5a114140408
Parents: bac7f79 43f7aaf
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 19:21:06 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 19:21:06 2015 +0300

----------------------------------------------------------------------
 dev-tools/src/main/groovy/jiraslurp.groovy                        | 3 +++
 .../internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java    | 2 +-
 scripts/git-patch-prop.sh                                         | 2 +-
 3 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[27/50] [abbrv] incubator-ignite git commit: [IGNITE-958]: IGNITE-218 (Wrong staging permissions while running MR job under hadoop accelerator): IGFS part.

Posted by ag...@apache.org.
[IGNITE-958]: IGNITE-218 (Wrong staging permissions while running MR job under hadoop accelerator): IGFS part.


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

Branch: refs/heads/ignite-389
Commit: 8455c7a6ed6f7449c7ad31b1ef7b129705262e1b
Parents: 3538819
Author: iveselovskiy <iv...@gridgain.com>
Authored: Fri May 29 15:40:26 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Fri May 29 15:40:26 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/igfs/IgfsUserContext.java | 119 +++++++++++
 .../hadoop/fs/HadoopLazyConcurrentMap.java      | 204 +++++++++++++++++++
 2 files changed, 323 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8455c7a6/modules/core/src/main/java/org/apache/ignite/igfs/IgfsUserContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsUserContext.java b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsUserContext.java
new file mode 100644
index 0000000..5a65bdb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsUserContext.java
@@ -0,0 +1,119 @@
+/*
+ * 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.igfs;
+
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.concurrent.*;
+
+/**
+ * Provides ability to execute IGFS code in a context of a specific user.
+ */
+public abstract class IgfsUserContext {
+    /** Thread local to hold the current user context. */
+    private static final ThreadLocal<String> userStackThreadLocal = new ThreadLocal<>();
+
+    /**
+     * Executes given callable in the given user context.
+     * The main contract of this method is that {@link #currentUser()} method invoked
+     * inside closure always returns 'user' this callable executed with.
+     * @param user the user name to invoke closure on behalf of.
+     * @param clo the closure to execute
+     * @param <T> The type of closure result.
+     * @return the result of closure execution.
+     * @throws IllegalArgumentException if user name is null or empty String or if the closure is null.
+     */
+    public static <T> T doAs(String user, final IgniteOutClosure<T> clo) {
+        if (F.isEmpty(user))
+            throw new IllegalArgumentException("Failed to use null or empty user name.");
+
+        final String ctxUser = userStackThreadLocal.get();
+
+        if (F.eq(ctxUser, user))
+            return clo.apply(); // correct context is already there
+
+        userStackThreadLocal.set(user);
+
+        try {
+            return clo.apply();
+        }
+        finally {
+            userStackThreadLocal.set(ctxUser);
+        }
+    }
+
+    /**
+     * Same contract that {@link #doAs(String, IgniteOutClosure)} has, but accepts
+     * callable that throws checked Exception.
+     * The Exception is not ever wrapped anyhow.
+     * If your Callable throws Some specific checked Exceptions, the recommended usage pattern is:
+     * <pre name="code" class="java">
+     *  public Foo myOperation() throws MyCheckedException1, MyCheckedException2 {
+     *      try {
+     *          return IgfsUserContext.doAs(user, new Callable<Foo>() {
+     *              &#64;Override public Foo call() throws MyCheckedException1, MyCheckedException2 {
+     *                  return makeSomeFoo(); // do the job
+     *              }
+     *          });
+     *      }
+     *      catch (MyCheckedException1 | MyCheckedException2 | RuntimeException | Error e) {
+     *          throw e;
+     *      }
+     *      catch (Exception e) {
+     *          throw new AssertionError("Must never go there.");
+     *      }
+     *  }
+     * </pre>
+     * @param user the user name to invoke closure on behalf of.
+     * @param clbl the Callable to execute
+     * @param <T> The type of callable result.
+     * @return the result of closure execution.
+     * @throws IllegalArgumentException if user name is null or empty String or if the closure is null.
+     */
+    public static <T> T doAs(String user, final Callable<T> clbl) throws Exception {
+        if (F.isEmpty(user))
+            throw new IllegalArgumentException("Failed to use null or empty user name.");
+
+        final String ctxUser = userStackThreadLocal.get();
+
+        if (F.eq(ctxUser, user))
+            return clbl.call(); // correct context is already there
+
+        userStackThreadLocal.set(user);
+
+        try {
+            return clbl.call();
+        }
+        finally {
+            userStackThreadLocal.set(ctxUser);
+        }
+    }
+
+    /**
+     * Gets the current context user.
+     * If this method is invoked outside of any {@link #doAs(String, IgniteOutClosure)} on the call stack, it will
+     * return null. Otherwise it will return the user name set in the most lower
+     * {@link #doAs(String, IgniteOutClosure)} call on the call stack.
+     * @return The current user, may be null.
+     */
+    @Nullable public static String currentUser() {
+        return userStackThreadLocal.get();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8455c7a6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
new file mode 100644
index 0000000..71b38c4
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
@@ -0,0 +1,204 @@
+/*
+ * 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.hadoop.fs;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.future.*;
+import org.jsr166.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.locks.*;
+
+/**
+ * Maps values by keys.
+ * Values are created lazily using {@link ValueFactory}.
+ *
+ * Despite of the name, does not depend on any Hadoop classes.
+ */
+public class HadoopLazyConcurrentMap<K, V extends Closeable> {
+    /** The map storing the actual values. */
+    private final ConcurrentMap<K, ValueWrapper> map = new ConcurrentHashMap8<>();
+
+    /** The factory passed in by the client. Will be used for lazy value creation. */
+    private final ValueFactory<K, V> factory;
+
+    /** Lock used to close the objects. */
+    private final ReadWriteLock closeLock = new ReentrantReadWriteLock();
+
+    /** Flag indicating that this map is closed and cleared. */
+    private boolean closed;
+
+    /**
+     * Constructor.
+     * @param factory the factory to create new values lazily.
+     */
+    public HadoopLazyConcurrentMap(ValueFactory<K, V> factory) {
+        this.factory = factory;
+    }
+
+    /**
+     * Gets cached or creates a new value of V.
+     * Never returns null.
+     * @param k the key to associate the value with.
+     * @return the cached or newly created value, never null.
+     * @throws IgniteException on error
+     */
+    public V getOrCreate(K k) {
+        ValueWrapper w = map.get(k);
+
+        if (w == null) {
+            closeLock.readLock().lock();
+
+            try {
+                if (closed)
+                    throw new IllegalStateException("Failed to create value for key [" + k
+                        + "]: the map is already closed.");
+
+                final ValueWrapper wNew = new ValueWrapper(k);
+
+                w = map.putIfAbsent(k, wNew);
+
+                if (w == null) {
+                    wNew.init();
+
+                    w = wNew;
+                }
+            }
+            finally {
+                closeLock.readLock().unlock();
+            }
+        }
+
+        try {
+            V v = w.getValue();
+
+            assert v != null;
+
+            return v;
+        }
+        catch (IgniteCheckedException ie) {
+            throw new IgniteException(ie);
+        }
+    }
+
+    /**
+     * Clears the map and closes all the values.
+     */
+    public void close() throws IgniteCheckedException {
+        closeLock.writeLock().lock();
+
+        try {
+            closed = true;
+
+            Exception err = null;
+
+            Set<K> keySet = map.keySet();
+
+            for (K key : keySet) {
+                V v = null;
+
+                try {
+                    v = map.get(key).getValue();
+                }
+                catch (IgniteCheckedException ignore) {
+                    // No-op.
+                }
+
+                if (v != null) {
+                    try {
+                        v.close();
+                    }
+                    catch (Exception err0) {
+                        if (err == null)
+                            err = err0;
+                    }
+                }
+            }
+
+            map.clear();
+
+            if (err != null)
+                throw new IgniteCheckedException(err);
+        }
+        finally {
+            closeLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Helper class that drives the lazy value creation.
+     */
+    private class ValueWrapper {
+        /** Future. */
+        private final GridFutureAdapter<V> fut = new GridFutureAdapter<>();
+
+        /** the key */
+        private final K key;
+
+        /**
+         * Creates new wrapper.
+         */
+        private ValueWrapper(K key) {
+            this.key = key;
+        }
+
+        /**
+         * Initializes the value using the factory.
+         */
+        private void init() {
+            try {
+                final V v0 = factory.createValue(key);
+
+                if (v0 == null)
+                    throw new IgniteException("Failed to create non-null value. [key=" + key + ']');
+
+                fut.onDone(v0);
+            }
+            catch (Throwable e) {
+                fut.onDone(e);
+            }
+        }
+
+        /**
+         * Gets the available value or blocks until the value is initialized.
+         * @return the value, never null.
+         * @throws IgniteCheckedException on error.
+         */
+        V getValue() throws IgniteCheckedException {
+            return fut.get();
+        }
+    }
+
+    /**
+     * Interface representing the factory that creates map values.
+     * @param <K> the type of the key.
+     * @param <V> the type of the value.
+     */
+    public interface ValueFactory <K, V> {
+        /**
+         * Creates the new value. Should never return null.
+         *
+         * @param key the key to create value for
+         * @return the value.
+         * @throws IgniteException on failure.
+         */
+        public V createValue(K key);
+    }
+}
\ No newline at end of file


[16/50] [abbrv] incubator-ignite git commit: # IGNITE-943 Bug fix: NPE in setAuthenticator

Posted by ag...@apache.org.
# IGNITE-943 Bug fix: NPE in setAuthenticator


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

Branch: refs/heads/ignite-389
Commit: 6b1265c2b103b4131b3ffc6deecd55d440fbe9b0
Parents: 2bc0795
Author: sevdokimov <se...@jetbrains.com>
Authored: Thu May 28 22:06:29 2015 +0300
Committer: sevdokimov <se...@jetbrains.com>
Committed: Thu May 28 22:06:29 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  5 -----
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 20 ++++++--------------
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  5 -----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  5 ++++-
 4 files changed, 10 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b1265c2/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 2171085..e672d64 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -319,11 +319,6 @@ class ClientImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
         if (segmented)
             throw new IgniteException("Failed to send custom message: client is disconnected");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b1265c2/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 b5c9519..57c13d6 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
@@ -115,9 +115,6 @@ class ServerImpl extends TcpDiscoveryImpl {
     /** Response on join request from coordinator (in case of duplicate ID or auth failure). */
     private final GridTuple<TcpDiscoveryAbstractMessage> joinRes = F.t1();
 
-    /** Node authenticator. */
-    private DiscoverySpiNodeAuthenticator nodeAuth;
-
     /** Mutex. */
     private final Object mux = new Object();
 
@@ -631,11 +628,6 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator nodeAuth) {
-        this.nodeAuth = nodeAuth;
-    }
-
-    /** {@inheritDoc} */
     @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
         try {
             msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, spi.marsh.marshal(evt)));
@@ -680,10 +672,10 @@ class ServerImpl extends TcpDiscoveryImpl {
                 if (log.isDebugEnabled())
                     log.debug("Join request message has not been sent (local node is the first in the topology).");
 
-                if (nodeAuth != null) {
+                if (spi.nodeAuth != null) {
                     // Authenticate local node.
                     try {
-                        SecurityContext subj = nodeAuth.authenticateNode(locNode, locCred);
+                        SecurityContext subj = spi.nodeAuth.authenticateNode(locNode, locCred);
 
                         if (subj == null)
                             throw new IgniteSpiException("Authentication failed for local node: " + locNode.id());
@@ -2469,12 +2461,12 @@ class ServerImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
-                if (nodeAuth != null) {
+                if (spi.nodeAuth != null) {
                     // Authenticate node first.
                     try {
                         SecurityCredentials cred = unmarshalCredentials(node);
 
-                        SecurityContext subj = nodeAuth.authenticateNode(node, cred);
+                        SecurityContext subj = spi.nodeAuth.authenticateNode(node, cred);
 
                         if (subj == null) {
                             // Node has not pass authentication.
@@ -2843,7 +2835,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
-                if (!isLocalNodeCoordinator() && nodeAuth != null && nodeAuth.isGlobalNodeAuthentication()) {
+                if (!isLocalNodeCoordinator() && spi.nodeAuth != null && spi.nodeAuth.isGlobalNodeAuthentication()) {
                     boolean authFailed = true;
 
                     try {
@@ -2861,7 +2853,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                             authFailed = false;
                         }
                         else {
-                            SecurityContext subj = nodeAuth.authenticateNode(node, cred);
+                            SecurityContext subj = spi.nodeAuth.authenticateNode(node, cred);
 
                             SecurityContext coordSubj = spi.ignite().configuration().getMarshaller().unmarshal(
                                 node.<byte[]>attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b1265c2/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index 4836911..f285279 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -113,11 +113,6 @@ abstract class TcpDiscoveryImpl {
     public abstract void disconnect() throws IgniteSpiException;
 
     /**
-     * @param auth Auth.
-     */
-    public abstract void setAuthenticator(DiscoverySpiNodeAuthenticator auth);
-
-    /**
      * @param msg Message.
      */
     public abstract void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b1265c2/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 64e6001..1b8424c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -294,6 +294,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
     protected long ipFinderCleanFreq = DFLT_IP_FINDER_CLEAN_FREQ;
 
+    /** Node authenticator. */
+    protected DiscoverySpiNodeAuthenticator nodeAuth;
+
     /** Context initialization latch. */
     @GridToStringExclude
     private final CountDownLatch ctxInitLatch = new CountDownLatch(1);
@@ -353,7 +356,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
 
     /** {@inheritDoc} */
     @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
-        impl.setAuthenticator(auth);
+        nodeAuth = auth;
     }
 
     /** {@inheritDoc} */


[31/50] [abbrv] incubator-ignite git commit: ignite-866 NPE during clean up

Posted by ag...@apache.org.
ignite-866 NPE during clean up


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

Branch: refs/heads/ignite-389
Commit: 2d9a938a281887fdae804ac68a89e93e5ad1b02b
Parents: 8455c7a
Author: agura <ag...@gridgain.com>
Authored: Thu May 14 14:40:38 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Fri May 29 16:43:38 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheMetricsImpl.java      | 62 +++++++++++++++-----
 .../processors/cache/GridCacheAdapter.java      | 12 +++-
 2 files changed, 55 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2d9a938a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index 560de97..af19077 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.store.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -118,7 +119,9 @@ public class CacheMetricsImpl implements CacheMetrics {
     /** {@inheritDoc} */
     @Override public long getOverflowSize() {
         try {
-            return cctx.cache().overflowSize();
+            GridCacheAdapter<?, ?> cache = cctx.cache();
+
+            return cache != null ? cache.overflowSize() : -1;
         }
         catch (IgniteCheckedException ignored) {
             return -1;
@@ -127,34 +130,47 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /** {@inheritDoc} */
     @Override public long getOffHeapEntriesCount() {
-        return cctx.cache().offHeapEntriesCount();
+        GridCacheAdapter<?, ?> cache = cctx.cache();
+
+        return cache != null ? cache.offHeapEntriesCount() : -1;
     }
 
     /** {@inheritDoc} */
     @Override public long getOffHeapAllocatedSize() {
-        return cctx.cache().offHeapAllocatedSize();
+        GridCacheAdapter<?, ?> cache = cctx.cache();
+
+        return cache != null ? cache.offHeapAllocatedSize() : -1;
     }
 
     /** {@inheritDoc} */
     @Override public int getSize() {
-        return cctx.cache().size();
+        GridCacheAdapter<?, ?> cache = cctx.cache();
+
+        return cache != null ? cache.size() : 0;
     }
 
     /** {@inheritDoc} */
     @Override public int getKeySize() {
-        return cctx.cache().size();
+        return getSize();
     }
 
     /** {@inheritDoc} */
     @Override public boolean isEmpty() {
-        return cctx.cache().isEmpty();
+        GridCacheAdapter<?, ?> cache = cctx.cache();
+
+        return cache == null || cache.isEmpty();
     }
 
     /** {@inheritDoc} */
     @Override public int getDhtEvictQueueCurrentSize() {
-        return cctx.isNear() ?
-                dhtCtx != null ? dhtCtx.evicts().evictQueueSize() : -1
-                : cctx.evicts().evictQueueSize();
+        GridCacheContext<?, ?> ctx = cctx.isNear() ? dhtCtx : cctx;
+
+        if (ctx == null)
+            return -1;
+
+        GridCacheEvictionManager evictMgr = ctx.evicts();
+
+        return evictMgr != null ? evictMgr.evictQueueSize() : -1;
     }
 
     /** {@inheritDoc} */
@@ -548,37 +564,51 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /** {@inheritDoc} */
     @Override public String getKeyType() {
-        return cctx.config().getKeyType().getName();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null ? ccfg.getKeyType().getName() : null;
     }
 
     /** {@inheritDoc} */
     @Override public String getValueType() {
-        return cctx.config().getValueType().getName();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null ? ccfg.getValueType().getName() : null;
     }
 
     /** {@inheritDoc} */
     @Override public boolean isReadThrough() {
-        return cctx.config().isReadThrough();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null && ccfg.isReadThrough();
     }
 
     /** {@inheritDoc} */
     @Override public boolean isWriteThrough() {
-        return cctx.config().isWriteThrough();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null && ccfg.isWriteThrough();
     }
 
     /** {@inheritDoc} */
     @Override public boolean isStoreByValue() {
-        return cctx.config().isStoreByValue();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null && ccfg.isStoreByValue();
     }
 
     /** {@inheritDoc} */
     @Override public boolean isStatisticsEnabled() {
-        return cctx.config().isStatisticsEnabled();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null && ccfg.isStatisticsEnabled();
     }
 
     /** {@inheritDoc} */
     @Override public boolean isManagementEnabled() {
-        return cctx.config().isManagementEnabled();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null && ccfg.isManagementEnabled();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2d9a938a/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 d390037..c975961 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
@@ -3249,7 +3249,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public long overflowSize() throws IgniteCheckedException {
-        return ctx.swap().swapSize();
+        GridCacheSwapManager swapMgr = ctx.swap();
+
+        return swapMgr != null ? swapMgr.swapSize() : -1;
     }
 
     /**
@@ -3802,12 +3804,16 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public long offHeapEntriesCount() {
-        return ctx.swap().offHeapEntriesCount();
+        GridCacheSwapManager swapMgr = ctx.swap();
+
+        return swapMgr != null ? swapMgr.offHeapEntriesCount() : -1;
     }
 
     /** {@inheritDoc} */
     @Override public long offHeapAllocatedSize() {
-        return ctx.swap().offHeapAllocatedSize();
+        GridCacheSwapManager swapMgr = ctx.swap();
+
+        return swapMgr != null ? swapMgr.offHeapAllocatedSize() : -1;
     }
 
     /** {@inheritDoc} */


[42/50] [abbrv] incubator-ignite git commit: # ignite-sprint-5: more help info at git-format-patch.sh

Posted by ag...@apache.org.
# ignite-sprint-5: more help info at git-format-patch.sh


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

Branch: refs/heads/ignite-389
Commit: 52d64bb3bf13b531458889d4ea5d8650ca1789b7
Parents: 2859671
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 1 12:16:07 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 1 12:16:07 2015 +0300

----------------------------------------------------------------------
 scripts/git-format-patch.sh | 14 +++++++++-----
 1 file changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52d64bb3/scripts/git-format-patch.sh
----------------------------------------------------------------------
diff --git a/scripts/git-format-patch.sh b/scripts/git-format-patch.sh
index 970347f..5aa2075 100755
--- a/scripts/git-format-patch.sh
+++ b/scripts/git-format-patch.sh
@@ -20,9 +20,13 @@
 # Git patch-file maker.
 #
 echo 'Usage: scripts/git-format-patch.sh [-ih|--ignitehome <path>] [-idb|--ignitedefbranch <branch-name>] [-ph|--patchhome <path>]'
+echo 'It is a script to create patch between current and default branches. The script is safe and do not broke or lose your changes.'
 echo "It should be called from IGNITE_HOME directory."
-echo "Patch will be created at PATCHES_HOME between Master branch (IGNITE_DEFAULT_BRANCH) and Current branch."
+echo "Patch will be created at PATCHES_HOME (= IGNITE_HOME, by default) between Master branch (IGNITE_DEFAULT_BRANCH) and Current branch."
 echo "Note: you can use ${IGNITE_HOME}/scripts/git-patch-prop-local.sh to set your own local properties (to rewrite settings at git-patch-prop-local.sh). "
+echo 'Examples:'
+echo '- Basic (with all defaults and properties from git-patch-prop.sh):  ./scripts/git-format-patch.sh'
+echo '- Rewrite some defaults (see Usage):                                ./scripts/git-format-patch.sh -ph /home/user_name/patches'
 echo
 
 #
@@ -51,17 +55,17 @@ do
         IGNITE_HOME="$2"
         shift
         ;;
-        
+
         -idb|--ignitedefbranch)
         IGNITE_DEFAULT_BRANCH="$2"
         shift
         ;;
-        
+
         -ph|--patchhome)
         PATCHES_HOME="$2"
         shift
         ;;
-        
+
         *)
         echo "Unknown parameter: ${key}"
         ;;
@@ -84,4 +88,4 @@ echo
 
 requireCleanWorkTree ${IGNITE_HOME}
 
-formatPatch ${IGNITE_HOME} ${IGNITE_DEFAULT_BRANCH} ${IGNITE_CURRENT_BRANCH} .patch
\ No newline at end of file
+formatPatch ${IGNITE_HOME} ${IGNITE_DEFAULT_BRANCH} ${IGNITE_CURRENT_BRANCH} .patch


[06/50] [abbrv] incubator-ignite git commit: #IGNITE-857 Fixed review notes.

Posted by ag...@apache.org.
#IGNITE-857 Fixed review notes.


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

Branch: refs/heads/ignite-389
Commit: 3a0b5daf0e23b456f68a15b2faa7080af671fca2
Parents: 62d1540
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 18:02:28 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 18:02:28 2015 +0300

----------------------------------------------------------------------
 modules/mesos/README.txt                        | 26 +++++++++++++++++++-
 modules/mesos/pom.xml                           |  2 +-
 .../apache/ignite/mesos/ClusterProperties.java  |  8 +++---
 .../apache/ignite/mesos/IgniteScheduler.java    | 13 +++++-----
 4 files changed, 36 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3a0b5daf/modules/mesos/README.txt
----------------------------------------------------------------------
diff --git a/modules/mesos/README.txt b/modules/mesos/README.txt
index 084ccce..75a62f8 100644
--- a/modules/mesos/README.txt
+++ b/modules/mesos/README.txt
@@ -1,4 +1,28 @@
 Apache Ignite Mesos Module
 ------------------------
 
-TODO
\ No newline at end of file
+Apache Ignite Mesos module provides integration Apache Ignite with Apache Mesos.
+
+Importing Apache Ignite Mesos Module In Maven Project
+-------------------------------------
+
+If you are using Maven to manage dependencies of your project, you can add Cloud module
+dependency like this (replace '${ignite.version}' with actual Ignite version you are
+interested in):
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+                        http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    ...
+    <dependencies>
+        ...
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-mesos</artifactId>
+            <version>${ignite.version}</version>
+        </dependency>
+        ...
+    </dependencies>
+    ...
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3a0b5daf/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index d3a2cef..3cf9028 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -25,7 +25,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-mesos</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.1-SNAPSHOT</version>
 
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3a0b5daf/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index 785eddc..956dd6e 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -201,7 +201,7 @@ public class ClusterProperties {
     }
 
     /**
-     * Set CPU count limit.
+     * Sets CPU count limit.
      */
     public void cpus(double cpu) {
         this.cpu = cpu;
@@ -215,7 +215,7 @@ public class ClusterProperties {
     }
 
     /**
-     * Set CPU count limit.
+     * Sets CPU count limit.
      */
     public void cpusPerNode(double cpu) {
         this.cpuPerNode = cpu;
@@ -229,7 +229,7 @@ public class ClusterProperties {
     }
 
     /**
-     * Set mem limit.
+     * Sets mem limit.
      *
      * @param mem Memory.
      */
@@ -245,7 +245,7 @@ public class ClusterProperties {
     }
 
     /**
-     * Set mem limit.
+     * Sets mem limit.
      *
      * @param mem Memory.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3a0b5daf/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 64e2931..263123c 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -77,9 +77,10 @@ public class IgniteScheduler implements Scheduler {
         for (Protos.Offer offer : offers) {
             Protos.TaskID taskId;
             Protos.TaskInfo task;
+            IgniteTask igniteTask;
 
             synchronized (mux) {
-                IgniteTask igniteTask = checkOffer(offer);
+                igniteTask = checkOffer(offer);
 
                 // Decline offer which doesn't match by mem or cpu.
                 if (igniteTask == null) {
@@ -96,8 +97,6 @@ public class IgniteScheduler implements Scheduler {
 
                 // Create task to run.
                 task = createTask(offer, igniteTask, taskId);
-
-                tasks.put(taskId.getValue(), igniteTask);
             }
 
             try {
@@ -106,12 +105,12 @@ public class IgniteScheduler implements Scheduler {
                     Protos.Filters.newBuilder().setRefuseSeconds(1).build());
             }
             catch (Exception e) {
-                synchronized (mux) {
-                    tasks.remove(taskId.getValue());
-                }
-
                 log.error("Failed launch task. Task id: {}. Task info: {}", taskId, task);
             }
+
+            synchronized (mux) {
+                tasks.put(taskId.getValue(), igniteTask);
+            }
         }
     }
 


[49/50] [abbrv] incubator-ignite git commit: IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389

Posted by ag...@apache.org.
IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389


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

Branch: refs/heads/ignite-389
Commit: f8910f6bcfe415b33558055a63445e20cacf7981
Parents: c527a04 541b1e0
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 1 15:04:05 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 1 15:04:05 2015 -0700

----------------------------------------------------------------------
 dev-tools/gradle/wrapper/gradle-wrapper.jar     |  Bin 51017 -> 0 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |   18 +-
 dev-tools/gradlew                               |  163 +-
 dev-tools/slurp.sh                              |    2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  100 +-
 examples/pom.xml                                |    2 +-
 modules/aop/pom.xml                             |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/clients/pom.xml                         |    2 +-
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 modules/core/pom.xml                            |    2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    8 +-
 .../java/org/apache/ignite/IgniteServices.java  |    5 +-
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |  187 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   27 +-
 .../configuration/IgniteConfiguration.java      |   10 +-
 .../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    |   27 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   22 +-
 .../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   |   59 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   48 +
 .../discovery/GridDiscoveryManager.java         |  214 +-
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../processors/cache/CacheMetricsImpl.java      |  367 +-
 .../cache/CacheMetricsMXBeanImpl.java           |  100 +
 .../processors/cache/CacheMetricsSnapshot.java  |  380 +-
 .../cache/DynamicCacheChangeBatch.java          |   20 +-
 .../cache/DynamicCacheDescriptor.java           |    2 +
 .../processors/cache/GridCacheAdapter.java      |   29 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |    6 +-
 .../processors/cache/GridCacheEntryEx.java      |    6 +
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMapEntry.java     |    5 +
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   71 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |   74 +-
 .../processors/cache/GridCacheProxyImpl.java    |   12 +
 .../cache/GridCacheSharedContext.java           |    1 +
 .../processors/cache/GridCacheSwapManager.java  |  118 +-
 .../processors/cache/GridCacheUtils.java        |  234 +-
 .../processors/cache/IgniteInternalCache.java   |    5 +
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedCacheEntry.java  |    7 -
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../dht/GridClientPartitionTopology.java        |    8 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtCacheEntry.java      |    6 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   22 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   78 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    4 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  313 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   28 +-
 .../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      |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |    7 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   15 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   54 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  836 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../datastreamer/DataStreamerImpl.java          |    2 +
 .../internal/processors/hadoop/HadoopJob.java   |    2 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |    8 +-
 .../internal/processors/igfs/IgfsImpl.java      |    8 +-
 .../processors/igfs/IgfsIpcHandler.java         |  184 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |    9 +-
 .../internal/processors/igfs/IgfsServer.java    |    4 +-
 .../internal/processors/igfs/IgfsUtils.java     |   16 +
 .../service/GridServiceProcessor.java           |  121 +-
 .../ignite/internal/util/GridJavaProcess.java   |   30 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |    2 +-
 .../ignite/mxbean/CacheMetricsMXBean.java       |   80 +
 .../apache/ignite/plugin/PluginProvider.java    |   26 +-
 .../org/apache/ignite/services/Service.java     |    5 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   37 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   47 -
 .../communication/tcp/TcpCommunicationSpi.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    | 4766 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  170 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5799 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   10 +-
 .../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 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 .../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 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    3 +
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../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/IgniteDynamicCacheStartSelfTest.java  |   20 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   47 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |   25 +-
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java |  412 ++
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   93 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../DataStreamerMultiThreadedSelfTest.java      |    2 -
 .../igfs/IgfsClientCacheSelfTest.java           |    3 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |  251 +
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../discovery/AbstractDiscoverySelfTest.java    |    8 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1171 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../testframework/GridSpiTestContext.java       |   25 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/common/GridCommonAbstractTest.java   |   59 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    4 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |    1 +
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |   41 -
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    3 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |    2 +-
 modules/hadoop/pom.xml                          |    2 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  165 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  107 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   32 +-
 .../internal/processors/hadoop/HadoopUtils.java |   10 +-
 .../hadoop/SecondaryFileSystemProvider.java     |   53 +-
 .../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/v2/HadoopV2TaskContext.java          |    4 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   56 +-
 ...oopSecondaryFileSystemConfigurationTest.java |    4 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   63 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |    2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |    2 +
 .../hadoop/HadoopFileSystemsTest.java           |   23 +-
 .../collections/HadoopSkipListSelfTest.java     |    4 +-
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/pom.xml                        |    2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |    4 +
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 .../query/h2/sql/BaseH2CompareQueryTest.java    |   16 +
 modules/jcl/pom.xml                             |    2 +-
 modules/jta/pom.xml                             |    2 +-
 modules/log4j/pom.xml                           |    2 +-
 modules/mesos/README.txt                        |   28 +
 modules/mesos/licenses/apache-2.0.txt           |  202 +
 modules/mesos/licenses/jetty-epl-license.txt    |   69 +
 modules/mesos/pom.xml                           |  101 +
 .../apache/ignite/mesos/ClusterProperties.java  |  519 ++
 .../apache/ignite/mesos/IgniteFramework.java    |  119 +
 .../apache/ignite/mesos/IgniteScheduler.java    |  361 ++
 .../org/apache/ignite/mesos/IgniteTask.java     |   86 +
 .../org/apache/ignite/mesos/package-info.java   |   22 +
 .../ignite/mesos/resource/IgniteProvider.java   |  234 +
 .../ignite/mesos/resource/JettyServer.java      |   61 +
 .../ignite/mesos/resource/ResourceHandler.java  |  142 +
 .../ignite/mesos/resource/ResourceProvider.java |  120 +
 .../ignite/mesos/resource/package-info.java     |   22 +
 .../main/resources/ignite-default-config.xml    |   35 +
 .../org/apache/ignite/IgniteMesosTestSuite.java |   38 +
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |  464 ++
 modules/rest-http/pom.xml                       |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |    2 +-
 modules/schema-import/pom.xml                   |    2 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spring/pom.xml                          |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/tools/pom.xml                           |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 .../licenses/jcraft-revised-bsd.txt             |   28 -
 modules/visor-console/pom.xml                   |    2 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web/pom.xml                             |    2 +-
 modules/yardstick/pom.xml                       |    2 +-
 parent/pom.xml                                  |    4 +
 pom.xml                                         |   15 +-
 scripts/git-format-patch.sh                     |   14 +-
 scripts/git-patch-prop.sh                       |    2 +-
 332 files changed, 23238 insertions(+), 11214 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 3c4c7d4,3065a2f..b61e4f0
--- 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
@@@ -2633,9 -2637,9 +2653,9 @@@ public class GridCacheProcessor extend
  
          req.clientStartOnly(true);
  
 -        F.first(initiateCacheChanges(F.asList(req))).get();
 +        F.first(initiateCacheChanges(F.asList(req), false)).get();
  
-         IgniteCache cache = jCacheProxies.get(masked);
+         IgniteCacheProxy cache = jCacheProxies.get(masked);
  
          if (cache == null && failIfNotStarted)
              throw new IllegalArgumentException("Cache is not started: " + cacheName);

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 2f32faa,fab490f..6fc6436
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@@ -498,21 -469,17 +498,26 @@@ public class GridCacheQueryAdapter<T> i
       * @param prj Projection (optional).
       * @return Collection of data nodes in provided projection (if any).
       */
 -    private static Collection<ClusterNode> nodes(final GridCacheContext<?, ?> cctx, @Nullable final ClusterGroup prj) {
 +    private static Collection<ClusterNode> nodes(final GridCacheContext<?, ?> cctx,
 +        @Nullable final ClusterGroup prj, @Nullable final Integer part) {
          assert cctx != null;
  
 +        final AffinityTopologyVersion topVer = cctx.affinity().affinityTopologyVersion();
 +
+         Collection<ClusterNode> affNodes = CU.affinityNodes(cctx);
+ 
 -        if (prj == null)
++        if (prj == null && part == null)
+             return affNodes;
+ 
 +        final Set<ClusterNode> owners =
 +            part == null ? Collections.<ClusterNode>emptySet() : new HashSet<>(cctx.topology().owners(part, topVer));
 +
-         return F.view(CU.allNodes(cctx), new P1<ClusterNode>() {
+         return F.view(affNodes, new P1<ClusterNode>() {
              @Override public boolean apply(ClusterNode n) {
 -                return prj.node(n.id()) != null;
 +
 +                return cctx.discovery().cacheAffinityNode(n, cctx.name()) &&
 +                    (prj == null || prj.node(n.id()) != null) &&
 +                    (part == null || owners.contains(n));
              }
          });
      }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 652d62e,32e9d63..6e71ba7
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@@ -769,138 -768,98 +769,138 @@@ public abstract class GridCacheQueryMan
  
          final boolean backups = qry.includeBackups() || cctx.isReplicated();
  
 -        final GridCloseableIteratorAdapter<IgniteBiTuple<K, V>> heapIt = new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
 -            private IgniteBiTuple<K, V> next;
 +        final GridCloseableIteratorAdapter<IgniteBiTuple<K, V>> heapIt =
 +            new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
 +                private IgniteBiTuple<K, V> next;
  
 -            private IgniteCacheExpiryPolicy expiryPlc = cctx.cache().expiryPolicy(plc);
 +                private IgniteCacheExpiryPolicy expiryPlc = cctx.cache().expiryPolicy(plc);
  
 -            private Iterator<K> iter = backups ? prj.keySetx().iterator() : prj.primaryKeySet().iterator();
 +                private Iterator<K> iter;
  
 -            {
 -                advance();
 -            }
 +                private GridDhtLocalPartition locPart;
  
 -            @Override public boolean onHasNext() {
 -                return next != null;
 -            }
 +                {
 +                    Integer part = qry.partition();
  
 -            @Override public IgniteBiTuple<K, V> onNext() {
 -                if (next == null)
 -                    throw new NoSuchElementException();
 +                    if (part == null || dht == null)
-                         iter = backups ? prj.keySet().iterator() : prj.primaryKeySet().iterator();
++                        iter = backups ? prj.keySetx().iterator() : prj.primaryKeySet().iterator();
 +                    else if (part < 0 || part >= cctx.affinity().partitions())
 +                        iter = F.emptyIterator();
 +                    else {
 +                        AffinityTopologyVersion topVer = cctx.affinity().affinityTopologyVersion();
  
 -                IgniteBiTuple<K, V> next0 = next;
 +                        locPart = dht.topology().localPartition(part, topVer, false);
  
 -                advance();
 +                        if (locPart == null || (locPart.state() != OWNING && locPart.state() != RENTING) ||
 +                            !locPart.reserve())
 +                            throw new GridDhtInvalidPartitionException(part, "Partition can't be reserved");
  
 -                return next0;
 -            }
 +                        iter = new Iterator<K>() {
 +                            private Iterator<KeyCacheObject> iter0 = locPart.keySet().iterator();
  
 -            private void advance() {
 -                IgniteBiTuple<K, V> next0 = null;
 -
 -                while (iter.hasNext()) {
 -                    next0 = null;
 +                            @Override public boolean hasNext() {
 +                                return iter0.hasNext();
 +                            }
  
 -                    K key = iter.next();
 +                            @Override public K next() {
 +                                KeyCacheObject key = iter0.next();
  
 -                    V val;
 +                                return key.value(cctx.cacheObjectContext(), false);
 +                            }
  
 -                    try {
 -                        val = prj.localPeek(key, CachePeekModes.ONHEAP_ONLY, expiryPlc);
 +                            @Override public void remove() {
 +                                iter0.remove();
 +                            }
 +                        };
                      }
 -                    catch (IgniteCheckedException e) {
 -                        if (log.isDebugEnabled())
 -                            log.debug("Failed to peek value: " + e);
  
 -                        val = null;
 -                    }
 +                    advance();
 +                }
  
 -                    if (dht != null && expiryPlc != null && expiryPlc.readyToFlush(100)) {
 -                        dht.sendTtlUpdateRequest(expiryPlc);
 +                @Override public boolean onHasNext() {
 +                    return next != null;
 +                }
  
 -                        expiryPlc = cctx.cache().expiryPolicy(plc);
 -                    }
 +                @Override public IgniteBiTuple<K, V> onNext() {
 +                    if (next == null)
 +                        throw new NoSuchElementException();
  
 -                    if (val != null) {
 -                        next0 = F.t(key, val);
 +                    IgniteBiTuple<K, V> next0 = next;
  
 -                        if (checkPredicate(next0))
 -                            break;
 -                        else
 -                            next0 = null;
 -                    }
 +                    advance();
 +
 +                    return next0;
                  }
  
 -                next = next0 != null ?
 -                    new IgniteBiTuple<>(next0.getKey(), next0.getValue()) :
 -                    null;
 +                private void advance() {
 +                    IgniteBiTuple<K, V> next0 = null;
  
 -                if (next == null)
 -                    sendTtlUpdate();
 -            }
 +                    while (iter.hasNext()) {
 +                        next0 = null;
  
 -            @Override protected void onClose() {
 -                sendTtlUpdate();
 -            }
 +                        K key = iter.next();
 +
 +                        V val;
  
 -            private void sendTtlUpdate() {
 -                if (dht != null && expiryPlc != null) {
 -                    dht.sendTtlUpdateRequest(expiryPlc);
 +                        try {
 +                            val = prj.localPeek(key, CachePeekModes.ONHEAP_ONLY, expiryPlc);
 +                        }
 +                        catch (IgniteCheckedException e) {
 +                            if (log.isDebugEnabled())
 +                                log.debug("Failed to peek value: " + e);
 +
 +                            val = null;
 +                        }
  
 -                    expiryPlc = null;
 +                        if (dht != null && expiryPlc != null && expiryPlc.readyToFlush(100)) {
 +                            dht.sendTtlUpdateRequest(expiryPlc);
 +
 +                            expiryPlc = cctx.cache().expiryPolicy(plc);
 +                        }
 +
 +                        if (val != null) {
 +                            next0 = F.t(key, val);
 +
 +                            if (checkPredicate(next0))
 +                                break;
 +                            else
 +                                next0 = null;
 +                        }
 +                    }
 +
 +                    next = next0 != null ?
 +                        new IgniteBiTuple<>(next0.getKey(), next0.getValue()) :
 +                        null;
 +
 +                    if (next == null)
 +                        sendTtlUpdate();
                  }
 -            }
  
 -            private boolean checkPredicate(Map.Entry<K, V> e) {
 -                if (keyValFilter != null) {
 -                    Map.Entry<K, V> e0 = (Map.Entry<K, V>)cctx.unwrapPortableIfNeeded(e, qry.keepPortable());
 +                @Override protected void onClose() {
 +                    sendTtlUpdate();
  
 -                    return keyValFilter.apply(e0.getKey(), e0.getValue());
 +                    if (locPart != null)
 +                        locPart.release();
                  }
  
 -                return true;
 -            }
 -        };
 +                private void sendTtlUpdate() {
 +                    if (dht != null && expiryPlc != null) {
 +                        dht.sendTtlUpdateRequest(expiryPlc);
 +
 +                        expiryPlc = null;
 +                    }
 +                }
 +
 +                private boolean checkPredicate(Map.Entry<K, V> e) {
 +                    if (keyValFilter != null) {
 +                        Map.Entry<K, V> e0 = (Map.Entry<K, V>)cctx.unwrapPortableIfNeeded(e, qry.keepPortable());
 +
 +                        return keyValFilter.apply(e0.getKey(), e0.getValue());
 +                    }
 +
 +                    return true;
 +                }
 +            };
  
          final GridIterator<IgniteBiTuple<K, V>> it;
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------

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


[45/50] [abbrv] incubator-ignite git commit: # IGNITE-943 Move static class instances.

Posted by ag...@apache.org.
# IGNITE-943 Move static class instances.


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

Branch: refs/heads/ignite-389
Commit: ab4c18caca9f3bd5dbc3174eb49ed79c1744a79d
Parents: 6c837c0
Author: sevdokimov <se...@gridgain.com>
Authored: Mon Jun 1 14:41:08 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Mon Jun 1 14:41:08 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab4c18ca/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 8eff6bd..2e31b69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -60,12 +60,6 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     /** Time to wait before reassignment retries. */
     private static final long RETRY_TIMEOUT = 1000;
 
-    /** */
-    private static final ServiceDeploymentPredicate DEPLOYMENT_PREDICATE = new ServiceDeploymentPredicate();
-
-    /** */
-    private static final ServiceAssignmentsPredicate ASSIGNMENTS_PREDICATE = new ServiceAssignmentsPredicate();
-
     /** Local service instances. */
     private final Map<String, Collection<ServiceContextImpl>> locSvcs = new HashMap<>();
 
@@ -352,7 +346,8 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                 "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']'));
                         }
                         else {
-                            Iterator<Cache.Entry<Object, Object>> it = serviceEntries(ASSIGNMENTS_PREDICATE);
+                            Iterator<Cache.Entry<Object, Object>> it = serviceEntries(
+                                ServiceAssignmentsPredicate.INSTANCE);
 
                             while (it.hasNext()) {
                                 Cache.Entry<Object, Object> e = it.next();
@@ -448,7 +443,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     public IgniteInternalFuture<?> cancelAll() {
         Collection<IgniteInternalFuture<?>> futs = new ArrayList<>();
 
-        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(ServiceDeploymentPredicate.INSTANCE);
 
         while (it.hasNext()) {
             Cache.Entry<Object, Object> e = it.next();
@@ -471,7 +466,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     public Collection<ServiceDescriptor> serviceDescriptors() {
         Collection<ServiceDescriptor> descs = new ArrayList<>();
 
-        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(ServiceDeploymentPredicate.INSTANCE);
 
         while (it.hasNext()) {
             Cache.Entry<Object, Object> e = it.next();
@@ -1113,7 +1108,8 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                 ctx.cache().context().deploy().ignoreOwnership(true);
 
                             try {
-                                Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+                                Iterator<Cache.Entry<Object, Object>> it = serviceEntries(
+                                    ServiceDeploymentPredicate.INSTANCE);
 
                                 while (it.hasNext()) {
                                     Cache.Entry<Object, Object> e = it.next();
@@ -1332,6 +1328,9 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      */
     static class ServiceDeploymentPredicate implements IgniteBiPredicate<Object, Object> {
         /** */
+        static final ServiceDeploymentPredicate INSTANCE = new ServiceDeploymentPredicate();
+
+        /** */
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */
@@ -1350,6 +1349,9 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      */
     static class ServiceAssignmentsPredicate implements IgniteBiPredicate<Object, Object> {
         /** */
+        static final ServiceAssignmentsPredicate INSTANCE = new ServiceAssignmentsPredicate();
+
+        /** */
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */


[30/50] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix GridSegmentationSelfTest

Posted by ag...@apache.org.
# IGNITE-943 Fix GridSegmentationSelfTest


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

Branch: refs/heads/ignite-389
Commit: ccc0d4038a5d41a8d83e99508279dd17d90d06df
Parents: 99234d0
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 16:39:59 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 16:40:37 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java  | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccc0d403/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 1b8424c..af85050 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -1688,6 +1688,13 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /**
      * <strong>FOR TEST ONLY!!!</strong>
      */
+    void forceNextNodeFailure() {
+        ((ServerImpl)impl).forceNextNodeFailure();
+    }
+
+    /**
+     * <strong>FOR TEST ONLY!!!</strong>
+     */
     public void addSendMessageListener(IgniteInClosure<TcpDiscoveryAbstractMessage> lsnr) {
         sendMsgLsnrs.add(lsnr);
     }


[29/50] [abbrv] incubator-ignite git commit: # ignite-943 fix services on client

Posted by ag...@apache.org.
# ignite-943 fix services on client


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

Branch: refs/heads/ignite-389
Commit: 99234d0ac5fd2e1d1033b1930cb4d7d6d49e0f28
Parents: 477a215
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 29 16:39:00 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 29 16:39:00 2015 +0300

----------------------------------------------------------------------
 .../service/GridServiceProcessor.java           | 104 ++++++++++++++++++-
 .../service/ClosureServiceClientsNodesTest.java |  10 ++
 2 files changed, 110 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/99234d0a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index a9a15e9..aedac75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
@@ -59,6 +60,12 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     /** Time to wait before reassignment retries. */
     private static final long RETRY_TIMEOUT = 1000;
 
+    /** */
+    private static final ServiceDeploymentPredicate DEPLOYMENT_PREDICATE = new ServiceDeploymentPredicate();
+
+    /** */
+    private static final ServiceAssignmentsPredicate ASSIGNMENTS_PREDICATE = new ServiceAssignmentsPredicate();
+
     /** Local service instances. */
     private final Map<String, Collection<ServiceContextImpl>> locSvcs = new HashMap<>();
 
@@ -345,7 +352,11 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                 "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']'));
                         }
                         else {
-                            for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+                            Iterator<Cache.Entry<Object, Object>> it = serviceEntries(ASSIGNMENTS_PREDICATE);
+
+                            while (it.hasNext()) {
+                                Cache.Entry<Object, Object> e = it.next();
+
                                 if (e.getKey() instanceof GridServiceAssignmentsKey) {
                                     GridServiceAssignments assigns = (GridServiceAssignments)e.getValue();
 
@@ -437,7 +448,11 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     public IgniteInternalFuture<?> cancelAll() {
         Collection<IgniteInternalFuture<?>> futs = new ArrayList<>();
 
-        for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+
+        while (it.hasNext()) {
+            Cache.Entry<Object, Object> e = it.next();
+
             if (!(e.getKey() instanceof GridServiceDeploymentKey))
                 continue;
 
@@ -456,7 +471,11 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     public Collection<ServiceDescriptor> serviceDescriptors() {
         Collection<ServiceDescriptor> descs = new ArrayList<>();
 
-        for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+
+        while (it.hasNext()) {
+            Cache.Entry<Object, Object> e = it.next();
+
             if (!(e.getKey() instanceof GridServiceDeploymentKey))
                 continue;
 
@@ -904,6 +923,43 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param p Entry predicate used to execute query from client node.
+     * @return Service deployment entries.
+     */
+    @SuppressWarnings("unchecked")
+    private Iterator<Cache.Entry<Object, Object>> serviceEntries(IgniteBiPredicate<Object, Object> p) {
+        if (!cache.context().affinityNode()) {
+            ClusterNode oldestSrvNode =
+                CU.oldestAliveCacheServerNode(cache.context().shared(), AffinityTopologyVersion.NONE);
+
+            if (oldestSrvNode == null)
+                return F.emptyIterator();
+
+            GridCacheQueryManager qryMgr = cache.context().queries();
+
+            CacheQuery<Map.Entry<Object, Object>> qry = qryMgr.createScanQuery(p, false);
+
+            qry.keepAll(false);
+
+            qry.projection(ctx.cluster().get().forNode(oldestSrvNode));
+
+            return cache.context().itHolder().iterator(qry.execute(),
+                new CacheIteratorConverter<Object, Map.Entry<Object,Object>>() {
+                    @Override protected Object convert(Map.Entry<Object, Object> e) {
+                        return new CacheEntryImpl<>(e.getKey(), e.getValue());
+                    }
+
+                    @Override protected void remove(Object item) {
+                        throw new UnsupportedOperationException();
+                    }
+                }
+            );
+        }
+        else
+            return cache.entrySetx().iterator();
+    }
+
+    /**
      * Service deployment listener.
      */
     private class DeploymentListener implements CacheEntryUpdatedListener<Object, Object> {
@@ -1056,7 +1112,11 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                 ctx.cache().context().deploy().ignoreOwnership(true);
 
                             try {
-                                for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+                                Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+
+                                while (it.hasNext()) {
+                                    Cache.Entry<Object, Object> e = it.next();
+
                                     if (!(e.getKey() instanceof GridServiceDeploymentKey))
                                         continue;
 
@@ -1265,4 +1325,40 @@ public class GridServiceProcessor extends GridProcessorAdapter {
          */
         public abstract void run0();
     }
+
+    /**
+     *
+     */
+    static class ServiceDeploymentPredicate implements IgniteBiPredicate<Object, Object> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(Object key, Object val) {
+            return key instanceof GridServiceDeploymentKey;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ServiceDeploymentPredicate.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    static class ServiceAssignmentsPredicate implements IgniteBiPredicate<Object, Object> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(Object key, Object val) {
+            return key instanceof GridServiceAssignmentsKey;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ServiceAssignmentsPredicate.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/99234d0a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
index 59594be..faccc9a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
@@ -84,6 +84,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
             srvNames.add(getTestGridName(i));
 
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             Collection<String> res = ignite.compute().broadcast(new IgniteCallable<String>() {
@@ -109,6 +111,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
      */
     public void testClientClosure() throws Exception {
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             Collection<String> res = ignite.compute(ignite.cluster().forClients()).
@@ -134,6 +138,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
      */
     public void testCustomClosure() throws Exception {
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             Collection<String> res = ignite.compute(ignite.cluster().forPredicate(F.<ClusterNode>alwaysTrue())).
@@ -157,6 +163,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
         UUID clientNodeId = grid(0).cluster().localNode().id();
 
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             ignite.services().deployNodeSingleton(SINGLETON_NAME, new TestService());
@@ -190,6 +198,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
         UUID clientNodeId = grid(0).cluster().localNode().id();
 
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             ignite.services(ignite.cluster().forClients()).deployNodeSingleton(SINGLETON_NAME, new TestService());


[20/50] [abbrv] incubator-ignite git commit: ignite-943 Fix tests: add setForceServerMode(true)

Posted by ag...@apache.org.
ignite-943 Fix tests: add setForceServerMode(true)


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

Branch: refs/heads/ignite-389
Commit: 1b0a051d4f91adc827e84ac816e3082ada6081ef
Parents: 6b1265c
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 13:15:25 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 13:15:59 2015 +0300

----------------------------------------------------------------------
 .../internal/GridReleaseTypeSelfTest.java       | 33 ++++++--------------
 ...GridCacheMixedPartitionExchangeSelfTest.java |  2 +-
 .../GridCacheReturnValueTransferSelfTest.java   |  3 ++
 .../IgniteCacheAbstractStopBusySelfTest.java    |  6 +---
 .../cache/IgniteCacheAbstractTest.java          |  2 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |  5 +--
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  6 +---
 7 files changed, 18 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
index 79e8905..96d5f1a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
@@ -45,32 +45,19 @@ public class GridReleaseTypeSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi discoSpi;
-
-        if (clientMode) {
-            discoSpi = new TcpDiscoverySpi() {
-                @Override public void setNodeAttributes(Map<String, Object> attrs,
-                    IgniteProductVersion ver) {
-                    super.setNodeAttributes(attrs, ver);
+        if (clientMode)
+            cfg.setClientMode(true);
 
-                    attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
-                }
-            };
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi() {
+            @Override public void setNodeAttributes(Map<String, Object> attrs,
+                IgniteProductVersion ver) {
+                super.setNodeAttributes(attrs, ver);
 
-            cfg.setClientMode(true);
-        }
-        else {
-            discoSpi = new TcpDiscoverySpi() {
-                @Override public void setNodeAttributes(Map<String, Object> attrs,
-                    IgniteProductVersion ver) {
-                    super.setNodeAttributes(attrs, ver);
-
-                    attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
-                }
-            };
-        }
+                attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
+            }
+        };
 
-        discoSpi.setIpFinder(IP_FINDER);
+        discoSpi.setIpFinder(IP_FINDER).setForceServerMode(true);
 
         cfg.setDiscoverySpi(discoSpi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
index 266937f..2fad523 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
@@ -56,7 +56,7 @@ public class GridCacheMixedPartitionExchangeSelfTest extends GridCommonAbstractT
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
 
         if (cache)
             cfg.setCacheConfiguration(cacheConfiguration());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
index 3e5987a..7723319 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import javax.cache.processor.*;
@@ -62,6 +63,8 @@ public class GridCacheReturnValueTransferSelfTest extends GridCommonAbstractTest
 
         cfg.setCacheConfiguration(ccfg);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         if (!cache)
             cfg.setClientMode(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
index aa8e2f7..10b14cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
@@ -101,11 +101,7 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
 
         cfg.setCommunicationSpi(commSpi);
 
-        TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-        spi.setIpFinder(finder);
-
-        cfg.setDiscoverySpi(spi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(finder).setForceServerMode(true));
 
         cfg.setCacheConfiguration(cacheCfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
index 371e547..1a62d4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
@@ -82,7 +82,7 @@ public abstract class IgniteCacheAbstractTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+        TcpDiscoverySpi disco = new TcpDiscoverySpi().setForceServerMode(true);
 
         disco.setMaxMissedHeartbeats(Integer.MAX_VALUE);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
index d3440fc..c0f73af 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
@@ -73,10 +73,7 @@ public class IgfsNearOnlyMultiNodeSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER).setForceServerMode(true));
 
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index 1a60bbd..6224cb9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -99,11 +99,7 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
+        c.setDiscoverySpi(new TcpDiscoverySpi().setForceServerMode(true).setIpFinder(ipFinder));
 
         // Otherwise noop swap space will be chosen on Windows.
         c.setSwapSpaceSpi(new FileSwapSpaceSpi());


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

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


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

Branch: refs/heads/ignite-389
Commit: d10120d67d3d6e20b25e494b03b829b20f263b5b
Parents: d10fe3e 5c30f9c
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 18:57:20 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 18:57:20 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/CacheMetrics.java   | 187 +++++++--
 .../org/apache/ignite/igfs/IgfsUserContext.java | 119 ++++++
 .../igfs/secondary/IgfsSecondaryFileSystem.java |   7 +
 .../internal/igfs/common/IgfsMarshaller.java    |  35 +-
 .../igfs/common/IgfsPathControlRequest.java     |  22 +
 .../internal/managers/GridManagerAdapter.java   |  59 +--
 .../processors/cache/CacheMetricsImpl.java      | 367 ++++++++++++++++-
 .../cache/CacheMetricsMXBeanImpl.java           | 100 +++++
 .../processors/cache/CacheMetricsSnapshot.java  | 380 +++++++++++++----
 .../processors/cache/GridCacheAdapter.java      |  12 +-
 .../processors/cache/GridCacheSwapManager.java  | 118 ++++--
 .../internal/processors/hadoop/HadoopJob.java   |   2 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |   8 +-
 .../internal/processors/igfs/IgfsImpl.java      |   8 +-
 .../processors/igfs/IgfsIpcHandler.java         | 184 +++++----
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   9 +-
 .../internal/processors/igfs/IgfsServer.java    |   4 +-
 .../internal/processors/igfs/IgfsUtils.java     |  16 +
 .../ignite/internal/util/GridJavaProcess.java   |  30 +-
 .../ignite/mxbean/CacheMetricsMXBean.java       |  80 ++++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  35 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  47 ---
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   8 +-
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java | 412 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |  25 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |   1 +
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java | 165 +++++---
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    | 107 +++--
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  32 +-
 .../internal/processors/hadoop/HadoopUtils.java |  10 +-
 .../hadoop/SecondaryFileSystemProvider.java     |  53 ++-
 .../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/v2/HadoopV2TaskContext.java          |   4 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  56 ++-
 ...oopSecondaryFileSystemConfigurationTest.java |   4 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  63 ++-
 .../IgniteHadoopFileSystemClientSelfTest.java   |   2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   2 +
 .../hadoop/HadoopFileSystemsTest.java           |  23 +-
 .../collections/HadoopSkipListSelfTest.java     |   4 +-
 47 files changed, 2537 insertions(+), 805 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d10120d6/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------


[17/50] [abbrv] incubator-ignite git commit: Added javadoc group.

Posted by ag...@apache.org.
Added javadoc group.


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

Branch: refs/heads/ignite-389
Commit: 2859671f1ff9866b9fab4de7695fbea86f458f32
Parents: 56bda96
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Fri May 29 10:02:18 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri May 29 10:02:18 2015 +0300

----------------------------------------------------------------------
 parent/pom.xml | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2859671f/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 44cb523..a514e35 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -317,6 +317,10 @@
                                 <title>Spring Caching</title>
                                 <packages>org.apache.ignite.cache.spring</packages>
                             </group>
+                            <group>
+                                <title>Mesos Framework</title>
+                                <packages>org.apache.ignite.mesos*</packages>
+                            </group>
                         </groups>
                         <header>
                             <![CDATA[


[02/50] [abbrv] incubator-ignite git commit: # ignite-857 review

Posted by ag...@apache.org.
# ignite-857 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/a36214c7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/a36214c7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/a36214c7

Branch: refs/heads/ignite-389
Commit: a36214c77564372916cb345b1a08f8e095276832
Parents: 789106b
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu May 28 17:20:58 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu May 28 17:20:58 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/mesos/IgniteScheduler.java   | 12 ++++++++----
 .../apache/ignite/mesos/IgniteSchedulerSelfTest.java    |  8 +++-----
 2 files changed, 11 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a36214c7/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index e833025..9507642 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -40,14 +40,14 @@ public class IgniteScheduler implements Scheduler {
     /** Default port range. */
     public static final String DEFAULT_PORT = ":47500..47510";
 
-    /** Delimiter to use in IP names. */
+    /** Delimiter char. */
     public static final String DELIM = ",";
 
     /** Logger. */
     private static final Logger log = LoggerFactory.getLogger(IgniteScheduler.class);
 
     /** Mutex. */
-    private static final Object mux = new Object();
+    private final Object mux = new Object();
 
     /** ID generator. */
     private AtomicInteger taskIdGenerator = new AtomicInteger();
@@ -89,7 +89,7 @@ public class IgniteScheduler implements Scheduler {
                 Protos.TaskID taskId = Protos.TaskID.newBuilder()
                     .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
 
-                log.info("Launching task: [{}]", igniteTask);
+                log.info("Launching task: {}", igniteTask);
 
                 // Create task to run.
                 Protos.TaskInfo task = createTask(offer, igniteTask, taskId);
@@ -175,7 +175,9 @@ public class IgniteScheduler implements Scheduler {
     /**
      * @return Address running nodes.
      */
-    protected String getAddress(String address) {
+    private String getAddress(String address) {
+        assert Thread.holdsLock(mux);
+
         if (tasks.isEmpty()) {
             if (address != null && !address.isEmpty())
                 return address + DEFAULT_PORT;
@@ -198,6 +200,8 @@ public class IgniteScheduler implements Scheduler {
      * @return Ignite task description.
      */
     private IgniteTask checkOffer(Protos.Offer offer) {
+        assert Thread.holdsLock(mux);
+
         // Check limit on running nodes.
         if (clusterProps.instances() <= tasks.size())
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a36214c7/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
index 13855b5..d627553 100644
--- a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
+++ b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
@@ -341,16 +341,14 @@ public class IgniteSchedulerSelfTest extends TestCase {
      * No-op implementation.
      */
     public static class DriverMock implements SchedulerDriver {
-        /**
-         *
-         */
+        /** */
         Collection<Protos.TaskInfo> launchedTask;
 
         /** */
         Protos.OfferID declinedOffer;
 
         /**
-         * Clear launched task.
+         * Clears launched task.
          */
         public void clear() {
             launchedTask = null;
@@ -463,4 +461,4 @@ public class IgniteSchedulerSelfTest extends TestCase {
             return null;
         }
     }
-}
\ No newline at end of file
+}


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

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


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

Branch: refs/heads/ignite-389
Commit: eb81019604bedd80bf8101f2d65e278b0de88cc0
Parents: ac258a5 7ec4c82
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 14:32:27 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 14:32:27 2015 +0300

----------------------------------------------------------------------
 dev-tools/src/main/groovy/jiraslurp.groovy      |   3 +
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   2 +-
 modules/mesos/README.txt                        |  28 +
 modules/mesos/licenses/apache-2.0.txt           | 202 ++++++++
 modules/mesos/licenses/jetty-epl-license.txt    |  69 +++
 modules/mesos/pom.xml                           | 101 ++++
 .../apache/ignite/mesos/ClusterProperties.java  | 519 +++++++++++++++++++
 .../apache/ignite/mesos/IgniteFramework.java    | 119 +++++
 .../apache/ignite/mesos/IgniteScheduler.java    | 361 +++++++++++++
 .../org/apache/ignite/mesos/IgniteTask.java     |  86 +++
 .../org/apache/ignite/mesos/package-info.java   |  22 +
 .../ignite/mesos/resource/IgniteProvider.java   | 234 +++++++++
 .../ignite/mesos/resource/JettyServer.java      |  61 +++
 .../ignite/mesos/resource/ResourceHandler.java  | 142 +++++
 .../ignite/mesos/resource/ResourceProvider.java | 120 +++++
 .../ignite/mesos/resource/package-info.java     |  22 +
 .../main/resources/ignite-default-config.xml    |  35 ++
 .../org/apache/ignite/IgniteMesosTestSuite.java |  38 ++
 .../ignite/mesos/IgniteSchedulerSelfTest.java   | 464 +++++++++++++++++
 parent/pom.xml                                  |   4 +
 pom.xml                                         |   1 +
 scripts/git-patch-prop.sh                       |   2 +-
 22 files changed, 2633 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[38/50] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix example tests.

Posted by ag...@apache.org.
# IGNITE-943 Fix example tests.


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

Branch: refs/heads/ignite-389
Commit: f499cfeaf140cf50355be775a79c186102833dfa
Parents: d10120d
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 20:07:10 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 20:07:10 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java  | 10 ++++++++--
 .../ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java | 10 ++++++----
 2 files changed, 14 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f499cfea/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index af85050..56fb63f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -1614,8 +1614,14 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
         U.interrupt(sockTimeoutWorker);
         U.join(sockTimeoutWorker, log);
 
-        if (ipFinder != null)
-            ipFinder.close();
+        if (ipFinder != null) {
+            try {
+                ipFinder.close();
+            }
+            catch (Exception e) {
+                log.error("Failed to close ipFinder", e);
+            }
+        }
 
         unregisterMBean();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f499cfea/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
index 6cf06ab..45d0816 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
@@ -495,11 +495,13 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
 
     /** {@inheritDoc} */
     @Override public void close() {
-        for (AddressSender addrSnd : addrSnds)
-            U.interrupt(addrSnd);
+        if (addrSnds != null) {
+            for (AddressSender addrSnd : addrSnds)
+                U.interrupt(addrSnd);
 
-        for (AddressSender addrSnd : addrSnds)
-            U.join(addrSnd, log);
+            for (AddressSender addrSnd : addrSnds)
+                U.join(addrSnd, log);
+        }
     }
 
     /** {@inheritDoc} */


[39/50] [abbrv] incubator-ignite git commit: # ignite-943

Posted by ag...@apache.org.
# ignite-943


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

Branch: refs/heads/ignite-389
Commit: 6c837c0783f04874157ac414f7161273c8c4d9f8
Parents: f499cfe
Author: sboikov <se...@inria.fr>
Authored: Fri May 29 23:43:25 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Fri May 29 23:43:25 2015 +0300

----------------------------------------------------------------------
 .../processors/service/GridServiceProcessor.java         | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6c837c07/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index aedac75..8eff6bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -1101,11 +1101,12 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             try {
                 depExe.submit(new BusyRunnable() {
                     @Override public void run0() {
-                        long topVer = ((DiscoveryEvent)evt).topologyVersion();
+                        AffinityTopologyVersion topVer =
+                            new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion());
 
-                        ClusterNode oldest = U.oldest(ctx.discovery().nodes(topVer), null);
+                        ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer);
 
-                        if (oldest.isLocal()) {
+                        if (oldest != null && oldest.isLocal()) {
                             final Collection<GridServiceDeployment> retries = new ConcurrentLinkedQueue<>();
 
                             if (ctx.deploy().enabled())
@@ -1128,7 +1129,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                         ctx.cache().internalCache(UTILITY_CACHE_NAME).context().affinity().
                                             affinityReadyFuture(topVer).get();
 
-                                        reassign(dep, topVer);
+                                        reassign(dep, topVer.topologyVersion());
                                     }
                                     catch (IgniteCheckedException ex) {
                                         if (!(e instanceof ClusterTopologyCheckedException))
@@ -1145,7 +1146,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                             }
 
                             if (!retries.isEmpty())
-                                onReassignmentFailed(topVer, retries);
+                                onReassignmentFailed(topVer.topologyVersion(), retries);
                         }
 
                         // Clean up zombie assignments.


[03/50] [abbrv] incubator-ignite git commit: #IGNITE-857 Fixed review notes.

Posted by ag...@apache.org.
#IGNITE-857 Fixed review notes.


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

Branch: refs/heads/ignite-389
Commit: 1c04c8b454c5906bec83c0096e5157dec5ab4f85
Parents: 4884995
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 17:46:27 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 17:46:27 2015 +0300

----------------------------------------------------------------------
 modules/mesos/pom.xml                           | 13 ----
 .../apache/ignite/mesos/ClusterProperties.java  |  6 +-
 .../apache/ignite/mesos/IgniteFramework.java    |  4 +-
 .../apache/ignite/mesos/IgniteScheduler.java    | 68 ++++++++++++--------
 4 files changed, 46 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c04c8b4/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 9079c66..d3a2cef 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -30,7 +30,6 @@
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>
         <mesos.version>0.22.0</mesos.version>
-        <slf4j.version>1.7.12</slf4j.version>
         <log4j.version>2.0.2</log4j.version>
     </properties>
 
@@ -42,24 +41,12 @@
         </dependency>
 
         <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-api</artifactId>
-            <version>${slf4j.version}</version>
-        </dependency>
-
-        <dependency>
             <groupId>org.apache.logging.log4j</groupId>
             <artifactId>log4j-core</artifactId>
             <version>${log4j.version}</version>
         </dependency>
 
         <dependency>
-            <groupId>org.apache.logging.log4j</groupId>
-            <artifactId>log4j-slf4j-impl</artifactId>
-            <version>${log4j.version}</version>
-        </dependency>
-
-        <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-server</artifactId>
             <version>${jetty.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c04c8b4/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index 944735e..785eddc 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.mesos;
 
-import org.slf4j.*;
+import org.apache.logging.log4j.*;
 
 import java.io.*;
 import java.net.*;
@@ -29,7 +29,7 @@ import java.util.regex.*;
  */
 public class ClusterProperties {
     /** */
-    private static final Logger log = LoggerFactory.getLogger(ClusterProperties.class);
+    private static final Logger log = LogManager.getLogger(ClusterProperties.class);
 
     /** Unlimited. */
     public static final double UNLIMITED = Double.MAX_VALUE;
@@ -62,7 +62,7 @@ public class ClusterProperties {
     public static final String IGNITE_HTTP_SERVER_PORT = "IGNITE_HTTP_SERVER_PORT";
 
     /** */
-    public static final String DEFAULT_HTTP_SERVER_PORT = "47511";
+    public static final String DEFAULT_HTTP_SERVER_PORT = "48610";
 
     /** Http server host. */
     private int httpServerPort = Integer.valueOf(DEFAULT_HTTP_SERVER_PORT);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c04c8b4/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
index 154385b..c30d206 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -19,8 +19,8 @@ package org.apache.ignite.mesos;
 
 import com.google.protobuf.*;
 import org.apache.ignite.mesos.resource.*;
+import org.apache.logging.log4j.*;
 import org.apache.mesos.*;
-import org.slf4j.*;
 
 import java.net.*;
 
@@ -29,7 +29,7 @@ import java.net.*;
  */
 public class IgniteFramework {
     /** */
-    public static final Logger log = LoggerFactory.getLogger(IgniteFramework.class);
+    public static final Logger log = LogManager.getLogger(IgniteFramework.class);
 
     /** Framework name. */
     public static final String IGNITE_FRAMEWORK_NAME = "Ignite";

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c04c8b4/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index e833025..bca064f 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.mesos;
 
 import org.apache.ignite.mesos.resource.*;
+import org.apache.logging.log4j.*;
 import org.apache.mesos.*;
-import org.slf4j.*;
 
 import java.util.*;
 import java.util.concurrent.atomic.*;
@@ -44,7 +44,7 @@ public class IgniteScheduler implements Scheduler {
     public static final String DELIM = ",";
 
     /** Logger. */
-    private static final Logger log = LoggerFactory.getLogger(IgniteScheduler.class);
+    private static final Logger log = LogManager.getLogger(IgniteScheduler.class);
 
     /** Mutex. */
     private static final Object mux = new Object();
@@ -72,10 +72,13 @@ public class IgniteScheduler implements Scheduler {
 
     /** {@inheritDoc} */
     @Override public void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
-        synchronized (mux) {
-            log.debug("Offers resources: {} ", offers.size());
+        log.debug("Offers resources: {} ", offers.size());
 
-            for (Protos.Offer offer : offers) {
+        for (Protos.Offer offer : offers) {
+            Protos.TaskID taskId;
+            Protos.TaskInfo task;
+
+            synchronized (mux) {
                 IgniteTask igniteTask = checkOffer(offer);
 
                 // Decline offer which doesn't match by mem or cpu.
@@ -86,19 +89,28 @@ public class IgniteScheduler implements Scheduler {
                 }
 
                 // Generate a unique task ID.
-                Protos.TaskID taskId = Protos.TaskID.newBuilder()
+                taskId = Protos.TaskID.newBuilder()
                     .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
 
-                log.info("Launching task: [{}]", igniteTask);
+                log.info("Launching task: {}", igniteTask);
 
                 // Create task to run.
-                Protos.TaskInfo task = createTask(offer, igniteTask, taskId);
+                task = createTask(offer, igniteTask, taskId);
+
+                tasks.put(taskId.getValue(), igniteTask);
+            }
 
+            try {
                 schedulerDriver.launchTasks(Collections.singletonList(offer.getId()),
                     Collections.singletonList(task),
                     Protos.Filters.newBuilder().setRefuseSeconds(1).build());
+            }
+            catch (Exception e) {
+                synchronized (mux) {
+                    tasks.remove(taskId.getValue());
+                }
 
-                tasks.put(taskId.getValue(), igniteTask);
+                log.error("Failed launch task. Task id: {}. Task info: {}", taskId, task);
             }
         }
     }
@@ -274,34 +286,36 @@ public class IgniteScheduler implements Scheduler {
     @Override public void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
         final String taskId = taskStatus.getTaskId().getValue();
 
-        log.info("Received update event task: [{}] is in state: [{}]", taskId, taskStatus.getState());
+        log.info("Received update event task: {} is in state: {}", taskId, taskStatus.getState());
 
         if (taskStatus.getState().equals(Protos.TaskState.TASK_FAILED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_ERROR)
             || taskStatus.getState().equals(Protos.TaskState.TASK_FINISHED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_KILLED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_LOST)) {
+            IgniteTask failedTask;
+
             synchronized (mux) {
-                IgniteTask failedTask = tasks.remove(taskId);
+                failedTask = tasks.remove(taskId);
+            }
 
-                if (failedTask != null) {
-                    List<Protos.Request> requests = new ArrayList<>();
+            if (failedTask != null) {
+                List<Protos.Request> requests = new ArrayList<>();
 
-                    Protos.Request request = Protos.Request.newBuilder()
-                        .addResources(Protos.Resource.newBuilder()
-                            .setType(Protos.Value.Type.SCALAR)
-                            .setName(MEM)
-                            .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.mem())))
-                        .addResources(Protos.Resource.newBuilder()
-                            .setType(Protos.Value.Type.SCALAR)
-                            .setName(CPU)
-                            .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.cpuCores())))
-                        .build();
+                Protos.Request request = Protos.Request.newBuilder()
+                    .addResources(Protos.Resource.newBuilder()
+                        .setType(Protos.Value.Type.SCALAR)
+                        .setName(MEM)
+                        .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.mem())))
+                    .addResources(Protos.Resource.newBuilder()
+                        .setType(Protos.Value.Type.SCALAR)
+                        .setName(CPU)
+                        .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.cpuCores())))
+                    .build();
 
-                    requests.add(request);
+                requests.add(request);
 
-                    schedulerDriver.requestResources(requests);
-                }
+                schedulerDriver.requestResources(requests);
             }
         }
     }
@@ -316,7 +330,7 @@ public class IgniteScheduler implements Scheduler {
     /** {@inheritDoc} */
     @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
         Protos.MasterInfo masterInfo) {
-        log.info("Scheduler registered. Master: [{}:{}], framework=[{}]", masterInfo.getIp(), masterInfo.getPort(),
+        log.info("Scheduler registered. Master: {}:{}, framework={}", masterInfo.getIp(), masterInfo.getPort(),
             frameworkID);
     }
 


[50/50] [abbrv] incubator-ignite git commit: IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389

Posted by ag...@apache.org.
IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389


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

Branch: refs/heads/ignite-389
Commit: d0157d4ef6091105245aee8d2e6698028ce3b95f
Parents: f8910f6
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 1 15:18:40 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 1 15:18:40 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0157d4e/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 2e31b69..aeb48f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -932,7 +932,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
             GridCacheQueryManager qryMgr = cache.context().queries();
 
-            CacheQuery<Map.Entry<Object, Object>> qry = qryMgr.createScanQuery(p, false);
+            CacheQuery<Map.Entry<Object, Object>> qry = qryMgr.createScanQuery(p, null, false);
 
             qry.keepAll(false);
 


[21/50] [abbrv] incubator-ignite git commit: mesos parent fix

Posted by ag...@apache.org.
mesos parent fix


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

Branch: refs/heads/ignite-389
Commit: 7ec4c823785fc5a614eacc96ddd02d23ad8665ce
Parents: 2859671
Author: avinogradov <av...@gridgain.com>
Authored: Fri May 29 13:36:39 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Fri May 29 13:36:39 2015 +0300

----------------------------------------------------------------------
 modules/mesos/pom.xml | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ec4c823/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index eca4fa9..c2bacff 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -23,7 +23,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
-    <groupId>org.apache.ignite</groupId>
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
     <artifactId>ignite-mesos</artifactId>
     <version>1.1.1-SNAPSHOT</version>
 


[36/50] [abbrv] incubator-ignite git commit: # regenerated PDF

Posted by ag...@apache.org.
# regenerated PDF


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

Branch: refs/heads/ignite-389
Commit: 5c30f9cf5c490b0d6c065e89557a8b8f3040eda5
Parents: 6cd1a6e 3f012b7
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri May 29 18:42:16 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri May 29 18:42:16 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/CacheMetrics.java   | 187 +++++++--
 .../internal/managers/GridManagerAdapter.java   |  59 +--
 .../processors/cache/CacheMetricsImpl.java      | 305 +++++++++++++-
 .../cache/CacheMetricsMXBeanImpl.java           | 100 +++++
 .../processors/cache/CacheMetricsSnapshot.java  | 380 +++++++++++++----
 .../processors/cache/GridCacheSwapManager.java  | 118 ++++--
 .../ignite/mxbean/CacheMetricsMXBean.java       |  80 ++++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  35 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  47 ---
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   8 +-
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java | 412 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |  25 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |   1 +
 13 files changed, 1457 insertions(+), 300 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5c30f9cf/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index af19077,3dcda3c..4e6a447
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@@ -129,24 -161,162 +164,168 @@@ public class CacheMetricsImpl implement
      }
  
      /** {@inheritDoc} */
+     @Override public long getOffHeapGets() {
+         return offHeapGets.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getOffHeapPuts() {
+         return offHeapPuts.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getOffHeapRemovals() {
+         return offHeapRemoves.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getOffHeapEvictions() {
+         return offHeapEvicts.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getOffHeapHits() {
+         return offHeapHits.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public float getOffHeapHitPercentage() {
+         long hits0 = offHeapHits.get();
+         long gets0 = offHeapGets.get();
+ 
+         if (hits0 == 0)
+             return 0;
+ 
+         return (float) hits0 / gets0 * 100.0f;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getOffHeapMisses() {
+         return offHeapMisses.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public float getOffHeapMissPercentage() {
+         long misses0 = offHeapMisses.get();
+         long reads0 = offHeapGets.get();
+ 
+         if (misses0 == 0)
+             return 0;
+ 
+         return (float) misses0 / reads0 * 100.0f;
+     }
+ 
+     /** {@inheritDoc} */
      @Override public long getOffHeapEntriesCount() {
 -        return cctx.cache().offHeapEntriesCount();
 +        GridCacheAdapter<?, ?> cache = cctx.cache();
 +
 +        return cache != null ? cache.offHeapEntriesCount() : -1;
      }
  
      /** {@inheritDoc} */
+     @Override public long getOffHeapPrimaryEntriesCount() {
+         try {
+             return cctx.swap().offheapEntriesCount(true, false, NONE);
+         }
+         catch (IgniteCheckedException e) {
+             return 0;
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getOffHeapBackupEntriesCount() {
+         try {
+             return cctx.swap().offheapEntriesCount(false, true, NONE);
+         }
+         catch (IgniteCheckedException e) {
+             return 0;
+         }
+     }
+ 
+     /** {@inheritDoc} */
      @Override public long getOffHeapAllocatedSize() {
 -        return cctx.cache().offHeapAllocatedSize();
 +        GridCacheAdapter<?, ?> cache = cctx.cache();
 +
 +        return cache != null ? cache.offHeapAllocatedSize() : -1;
      }
  
      /** {@inheritDoc} */
+     @Override public long getOffHeapMaxSize() {
+         return cctx.config().getOffHeapMaxMemory();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getSwapGets() {
+         return swapGets.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getSwapPuts() {
+         return swapPuts.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getSwapRemovals() {
+         return swapRemoves.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getSwapHits() {
+         return swapHits.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getSwapMisses() {
+         return swapMisses.get();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getSwapEntriesCount() {
+         try {
+             return cctx.cache().swapKeys();
+         }
+         catch (IgniteCheckedException e) {
+             return 0;
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public long getSwapSize() {
+         try {
+             return cctx.cache().swapSize();
+         }
+         catch (IgniteCheckedException e) {
+             return 0;
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public float getSwapHitPercentage() {
+         long hits0 = swapHits.get();
+         long gets0 = swapGets.get();
+ 
+         if (hits0 == 0)
+             return 0;
+ 
+         return (float) hits0 / gets0 * 100.0f;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public float getSwapMissPercentage() {
+         long misses0 = swapMisses.get();
+         long reads0 = swapGets.get();
+ 
+         if (misses0 == 0)
+             return 0;
+ 
+         return (float) misses0 / reads0 * 100.0f;
+     }
+ 
+     /** {@inheritDoc} */
      @Override public int getSize() {
 -        return cctx.cache().size();
 +        GridCacheAdapter<?, ?> cache = cctx.cache();
 +
 +        return cache != null ? cache.size() : 0;
      }
  
      /** {@inheritDoc} */
@@@ -606,11 -769,111 +797,113 @@@
  
      /** {@inheritDoc} */
      @Override public boolean isManagementEnabled() {
 -        return cctx.config().isManagementEnabled();
 +        CacheConfiguration ccfg = cctx.config();
 +
 +        return ccfg != null && ccfg.isManagementEnabled();
      }
  
+     /**
+      * Off-heap read callback.
+      *
+      * @param hit Hit or miss flag.
+      */
+     public void onOffHeapRead(boolean hit) {
+         offHeapGets.incrementAndGet();
+ 
+         if (hit)
+             offHeapHits.incrementAndGet();
+         else
+             offHeapMisses.incrementAndGet();
+ 
+         if (delegate != null)
+             delegate.onOffHeapRead(hit);
+     }
+ 
+     /**
+      * Off-heap write callback.
+      */
+     public void onOffHeapWrite() {
+         offHeapPuts.incrementAndGet();
+ 
+         if (delegate != null)
+             delegate.onOffHeapWrite();
+     }
+ 
+     /**
+      * Off-heap remove callback.
+      */
+     public void onOffHeapRemove() {
+         offHeapRemoves.incrementAndGet();
+ 
+         if (delegate != null)
+             delegate.onOffHeapRemove();
+     }
+ 
+     /**
+      * Off-heap evict callback.
+      */
+     public void onOffHeapEvict() {
+         offHeapEvicts.incrementAndGet();
+ 
+         if (delegate != null)
+             delegate.onOffHeapRemove();
+     }
+ 
+     /**
+      * Swap read callback.
+      *
+      * @param hit Hit or miss flag.
+      */
+     public void onSwapRead(boolean hit) {
+         swapGets.incrementAndGet();
+ 
+         if (hit)
+             swapHits.incrementAndGet();
+         else
+             swapMisses.incrementAndGet();
+ 
+         if (delegate != null)
+             delegate.onSwapRead(hit);
+     }
+ 
+     /**
+      * Swap write callback.
+      */
+     public void onSwapWrite() {
+         onSwapWrite(1);
+     }
+ 
+     /**
+      * Swap write callback.
+      *
+      * @param cnt Amount of entries.
+      */
+     public void onSwapWrite(int cnt) {
+         swapPuts.addAndGet(cnt);
+ 
+         if (delegate != null)
+             delegate.onSwapWrite(cnt);
+     }
+ 
+     /**
+      * Swap remove callback.
+      */
+     public void onSwapRemove() {
+         onSwapRemove(1);
+     }
+ 
+     /**
+      * Swap remove callback.
+      *
+      * @param cnt Amount of entries.
+      */
+     public void onSwapRemove(int cnt) {
+         swapRemoves.addAndGet(cnt);
+ 
+         if (delegate != null)
+             delegate.onSwapRemove(cnt);
+     }
+ 
      /** {@inheritDoc} */
      @Override public String toString() {
          return S.toString(CacheMetricsImpl.class, this);


[09/50] [abbrv] incubator-ignite git commit: [IGNITE-765]: org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.GcWorker#cleanupResources incorrectly handles FileLockInterruptionException

Posted by ag...@apache.org.
[IGNITE-765]: org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.GcWorker#cleanupResources incorrectly handles FileLockInterruptionException


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

Branch: refs/heads/ignite-389
Commit: 98e392cd79183945c26e9c220787cdbd4f686c26
Parents: b6fc8a9
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu May 28 18:39:45 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Thu May 28 18:39:45 2015 +0300

----------------------------------------------------------------------
 .../internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/98e392cd/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
index 86a0886..5185856 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
@@ -592,7 +592,7 @@ public class IpcSharedMemoryServerEndpoint implements IpcServerEndpoint {
                 if (log.isDebugEnabled())
                     log.debug("Token directory is being processed concurrently: " + workTokDir.getAbsolutePath());
             }
-            catch (InterruptedIOException ignored) {
+            catch (FileLockInterruptionException ignored) {
                 Thread.currentThread().interrupt();
             }
             catch (IOException e) {


[22/50] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix TcpClientDiscoverySpiSelfTest.testMetrics

Posted by ag...@apache.org.
# IGNITE-943 Fix TcpClientDiscoverySpiSelfTest.testMetrics


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

Branch: refs/heads/ignite-389
Commit: 2477c0ebd2e8003b58e3a83b4389d1118b30b5fc
Parents: 1b0a051
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 13:44:29 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 13:44:29 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2477c0eb/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 be3474d..f7be340 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
@@ -681,7 +681,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
             Ignite g = G.ignite("server-" + i);
 
             for (ClusterNode n : g.cluster().nodes()) {
-                if (n.metrics().getTotalExecutedJobs() != execJobsCnt)
+                if (n.metrics().getTotalExecutedJobs() != (n.isClient() ? 0 : execJobsCnt))
                     return false;
             }
         }
@@ -690,7 +690,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
             Ignite g = G.ignite("client-" + i);
 
             for (ClusterNode n : g.cluster().nodes()) {
-                if (n.metrics().getTotalExecutedJobs() != execJobsCnt)
+                if (n.metrics().getTotalExecutedJobs() != (n.isClient() ? 0 : execJobsCnt))
                     return false;
             }
         }


[10/50] [abbrv] incubator-ignite git commit: #IGNITE-857 Removed log4j.

Posted by ag...@apache.org.
#IGNITE-857 Removed log4j.


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

Branch: refs/heads/ignite-389
Commit: bac7f79af721874e8d36a01471e0b0362c54e442
Parents: 73c519e
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 19:01:49 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 19:01:49 2015 +0300

----------------------------------------------------------------------
 modules/mesos/pom.xml                           |  7 --
 .../apache/ignite/mesos/ClusterProperties.java  |  7 +-
 .../apache/ignite/mesos/IgniteFramework.java    |  8 +-
 .../apache/ignite/mesos/IgniteScheduler.java    | 83 ++++++++------------
 modules/mesos/src/main/resources/log4j2.xml     | 35 ---------
 5 files changed, 40 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac7f79a/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 3cf9028..eca4fa9 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -30,7 +30,6 @@
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>
         <mesos.version>0.22.0</mesos.version>
-        <log4j.version>2.0.2</log4j.version>
     </properties>
 
     <dependencies>
@@ -41,12 +40,6 @@
         </dependency>
 
         <dependency>
-            <groupId>org.apache.logging.log4j</groupId>
-            <artifactId>log4j-core</artifactId>
-            <version>${log4j.version}</version>
-        </dependency>
-
-        <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-server</artifactId>
             <version>${jetty.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac7f79a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index 956dd6e..6663625 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -17,11 +17,10 @@
 
 package org.apache.ignite.mesos;
 
-import org.apache.logging.log4j.*;
-
 import java.io.*;
 import java.net.*;
 import java.util.*;
+import java.util.logging.*;
 import java.util.regex.*;
 
 /**
@@ -29,7 +28,7 @@ import java.util.regex.*;
  */
 public class ClusterProperties {
     /** */
-    private static final Logger log = LogManager.getLogger(ClusterProperties.class);
+    private static final Logger log = Logger.getLogger(ClusterProperties.class.getSimpleName());
 
     /** Unlimited. */
     public static final double UNLIMITED = Double.MAX_VALUE;
@@ -448,7 +447,7 @@ public class ClusterProperties {
                     prop.hostnameConstraint = Pattern.compile(pattern);
                 }
                 catch (PatternSyntaxException e) {
-                    log.warn("IGNITE_HOSTNAME_CONSTRAINT has invalid pattern. It will be ignore.", e);
+                    log.log(Level.WARNING, "IGNITE_HOSTNAME_CONSTRAINT has invalid pattern. It will be ignore.", e);
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac7f79a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
index c30d206..3d582d9 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -19,17 +19,17 @@ package org.apache.ignite.mesos;
 
 import com.google.protobuf.*;
 import org.apache.ignite.mesos.resource.*;
-import org.apache.logging.log4j.*;
 import org.apache.mesos.*;
 
 import java.net.*;
+import java.util.logging.*;
 
 /**
  * Ignite mesos framework.
  */
 public class IgniteFramework {
     /** */
-    public static final Logger log = LogManager.getLogger(IgniteFramework.class);
+    public static final Logger log = Logger.getLogger(IgniteFramework.class.getSimpleName());
 
     /** Framework name. */
     public static final String IGNITE_FRAMEWORK_NAME = "Ignite";
@@ -80,13 +80,13 @@ public class IgniteFramework {
             log.info("Enabling authentication for the framework");
 
             if (System.getenv("DEFAULT_PRINCIPAL") == null) {
-                log.error("Expecting authentication principal in the environment");
+                log.log(Level.SEVERE, "Expecting authentication principal in the environment");
 
                 System.exit(1);
             }
 
             if (System.getenv("DEFAULT_SECRET") == null) {
-                log.error("Expecting authentication secret in the environment");
+                log.log(Level.SEVERE, "Expecting authentication secret in the environment");
 
                 System.exit(1);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac7f79a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 6bd3aa7..fbb9994 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -18,11 +18,11 @@
 package org.apache.ignite.mesos;
 
 import org.apache.ignite.mesos.resource.*;
-import org.apache.logging.log4j.*;
 import org.apache.mesos.*;
 
 import java.util.*;
 import java.util.concurrent.atomic.*;
+import java.util.logging.*;
 
 /**
  * Ignite scheduler receives offers from Mesos and decides how many resources will be occupied.
@@ -44,10 +44,7 @@ public class IgniteScheduler implements Scheduler {
     public static final String DELIM = ",";
 
     /** Logger. */
-    private static final Logger log = LogManager.getLogger(IgniteScheduler.class);
-
-    /** Mutex. */
-    private final Object mux = new Object();
+    private static final Logger log = Logger.getLogger(IgniteScheduler.class.getSimpleName());
 
     /** ID generator. */
     private AtomicInteger taskIdGenerator = new AtomicInteger();
@@ -71,33 +68,27 @@ public class IgniteScheduler implements Scheduler {
     }
 
     /** {@inheritDoc} */
-    @Override public void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
-        log.debug("Offers resources: {} ", offers.size());
+    @Override public synchronized void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
+        log.log(Level.FINE, "Offers resources: {0}", offers.size());
 
         for (Protos.Offer offer : offers) {
-            Protos.TaskID taskId;
-            Protos.TaskInfo task;
-            IgniteTask igniteTask;
+            IgniteTask igniteTask = checkOffer(offer);
 
-            synchronized (mux) {
-                igniteTask = checkOffer(offer);
+            // Decline offer which doesn't match by mem or cpu.
+            if (igniteTask == null) {
+                schedulerDriver.declineOffer(offer.getId());
 
-                // Decline offer which doesn't match by mem or cpu.
-                if (igniteTask == null) {
-                    schedulerDriver.declineOffer(offer.getId());
+                continue;
+            }
 
-                    continue;
-                }
+            // Generate a unique task ID.
+            Protos.TaskID taskId = Protos.TaskID.newBuilder()
+                .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
 
-                // Generate a unique task ID.
-                taskId = Protos.TaskID.newBuilder()
-                    .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
+            log.log(Level.INFO, "Launching task: {0}", igniteTask);
 
-                log.info("Launching task: {}", igniteTask);
-
-                // Create task to run.
-                task = createTask(offer, igniteTask, taskId);
-            }
+            // Create task to run.
+            Protos.TaskInfo task = createTask(offer, igniteTask, taskId);
 
             try {
                 schedulerDriver.launchTasks(Collections.singletonList(offer.getId()),
@@ -105,14 +96,13 @@ public class IgniteScheduler implements Scheduler {
                     Protos.Filters.newBuilder().setRefuseSeconds(1).build());
             }
             catch (RuntimeException e) {
-                log.error("Failed launch task. Task id: {}. Task info: {}", taskId, task);
+                log.log(Level.SEVERE, "Failed launch task. Task id: {0}. Task info: {1}",
+                    new Object[]{taskId, task, e});
 
                 throw e;
             }
 
-            synchronized (mux) {
-                tasks.put(taskId.getValue(), igniteTask);
-            }
+            tasks.put(taskId.getValue(), igniteTask);
         }
     }
 
@@ -124,7 +114,7 @@ public class IgniteScheduler implements Scheduler {
      * @param taskId Task id.
      * @return Task.
      */
-    protected Protos.TaskInfo createTask(Protos.Offer offer, IgniteTask igniteTask, Protos.TaskID taskId) {
+    private Protos.TaskInfo createTask(Protos.Offer offer, IgniteTask igniteTask, Protos.TaskID taskId) {
         String cfgUrl = clusterProps.igniteConfigUrl() != null ?
             clusterProps.igniteConfigUrl() : resourceProvider.igniteConfigUrl();
 
@@ -189,8 +179,6 @@ public class IgniteScheduler implements Scheduler {
      * @return Address running nodes.
      */
     private String getAddress(String address) {
-        assert Thread.holdsLock(mux);
-
         if (tasks.isEmpty()) {
             if (address != null && !address.isEmpty())
                 return address + DEFAULT_PORT;
@@ -213,8 +201,6 @@ public class IgniteScheduler implements Scheduler {
      * @return Ignite task description.
      */
     private IgniteTask checkOffer(Protos.Offer offer) {
-        assert Thread.holdsLock(mux);
-
         // Check limit on running nodes.
         if (clusterProps.instances() <= tasks.size())
             return null;
@@ -234,24 +220,24 @@ public class IgniteScheduler implements Scheduler {
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
                     cpus = resource.getScalar().getValue();
                 else
-                    log.debug("Cpus resource was not a scalar: {}" + resource.getType());
+                    log.log(Level.FINE, "Cpus resource was not a scalar: {0}" + resource.getType());
             }
             else if (resource.getName().equals(MEM)) {
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
                     mem = resource.getScalar().getValue();
                 else
-                    log.debug("Mem resource was not a scalar: {}", resource.getType());
+                    log.log(Level.FINE, "Mem resource was not a scalar: {0}", resource.getType());
             }
             else if (resource.getName().equals(DISK))
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
                     disk = resource.getScalar().getValue();
                 else
-                    log.debug("Disk resource was not a scalar: {}", resource.getType());
+                    log.log(Level.FINE, "Disk resource was not a scalar: {0}", resource.getType());
         }
 
         // Check that slave satisfies min requirements.
         if (cpus < clusterProps.minCpuPerNode() || mem < clusterProps.minMemoryPerNode()) {
-            log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
+            log.log(Level.FINE, "Offer not sufficient for slave request: {0}", offer.getResourcesList());
 
             return null;
         }
@@ -273,7 +259,7 @@ public class IgniteScheduler implements Scheduler {
 
         if ((clusterProps.cpusPerNode() != ClusterProperties.UNLIMITED && clusterProps.cpusPerNode() != cpus)
             || (clusterProps.memoryPerNode() != ClusterProperties.UNLIMITED && clusterProps.memoryPerNode() != mem)) {
-            log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
+            log.log(Level.FINE, "Offer not sufficient for slave request: {0}", offer.getResourcesList());
 
             return null;
         }
@@ -281,28 +267,25 @@ public class IgniteScheduler implements Scheduler {
         if (cpus > 0 && mem > 0)
             return new IgniteTask(offer.getHostname(), cpus, mem, disk);
         else {
-            log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
+            log.log(Level.FINE, "Offer not sufficient for slave request: {0}", offer.getResourcesList());
 
             return null;
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
+    @Override public synchronized void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
         final String taskId = taskStatus.getTaskId().getValue();
 
-        log.info("Received update event task: {} is in state: {}", taskId, taskStatus.getState());
+        log.log(Level.INFO, "Received update event task: {0} is in state: {1}",
+            new Object[]{taskId, taskStatus.getState()});
 
         if (taskStatus.getState().equals(Protos.TaskState.TASK_FAILED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_ERROR)
             || taskStatus.getState().equals(Protos.TaskState.TASK_FINISHED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_KILLED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_LOST)) {
-            IgniteTask failedTask;
-
-            synchronized (mux) {
-                failedTask = tasks.remove(taskId);
-            }
+            IgniteTask failedTask = tasks.remove(taskId);
 
             if (failedTask != null) {
                 List<Protos.Request> requests = new ArrayList<>();
@@ -335,8 +318,8 @@ public class IgniteScheduler implements Scheduler {
     /** {@inheritDoc} */
     @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
         Protos.MasterInfo masterInfo) {
-        log.info("Scheduler registered. Master: {}:{}, framework={}", masterInfo.getIp(), masterInfo.getPort(),
-            frameworkID);
+        log.log(Level.INFO, "Scheduler registered. Master: {0}:{1}, framework={2}", new Object[]{masterInfo.getIp(),
+            masterInfo.getPort(), frameworkID});
     }
 
     /** {@inheritDoc} */
@@ -346,7 +329,7 @@ public class IgniteScheduler implements Scheduler {
 
     /** {@inheritDoc} */
     @Override public void error(SchedulerDriver schedulerDriver, String s) {
-        log.error("Failed. Error message: {}", s);
+        log.log(Level.SEVERE, "Failed. Error message: {0}", s);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac7f79a/modules/mesos/src/main/resources/log4j2.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/resources/log4j2.xml b/modules/mesos/src/main/resources/log4j2.xml
deleted file mode 100644
index d66a83f..0000000
--- a/modules/mesos/src/main/resources/log4j2.xml
+++ /dev/null
@@ -1,35 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<configuration strict="true" status="OFF" monitorInterval="30"  shutdownHook="disable">
-
-    <appenders>
-        <appender type="Console" name="STDOUT">
-            <layout type="PatternLayout" pattern="%d %p %C{1.} [%t] %m%n %ex"/>
-        </appender>
-    </appenders>
-
-    <loggers>
-        <root level="info">
-            <appender-ref ref="STDOUT"/>
-        </root>
-
-    </loggers>
-
-</configuration>
\ No newline at end of file


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

Posted by ag...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite 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/00848cce
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/00848cce
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/00848cce

Branch: refs/heads/ignite-389
Commit: 00848ccedd6ad1490461069d5039cf7306626377
Parents: 52d64bb 31fcc3e
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 1 12:16:44 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 1 12:16:44 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/CacheMetrics.java   | 187 +++++++--
 .../configuration/IgniteConfiguration.java      |   6 +
 .../org/apache/ignite/igfs/IgfsUserContext.java | 119 ++++++
 .../igfs/secondary/IgfsSecondaryFileSystem.java |   7 +
 .../internal/igfs/common/IgfsMarshaller.java    |  35 +-
 .../igfs/common/IgfsPathControlRequest.java     |  22 +
 .../internal/managers/GridManagerAdapter.java   |  59 +--
 .../processors/cache/CacheMetricsImpl.java      | 367 ++++++++++++++++-
 .../cache/CacheMetricsMXBeanImpl.java           | 100 +++++
 .../processors/cache/CacheMetricsSnapshot.java  | 380 +++++++++++++----
 .../processors/cache/GridCacheAdapter.java      |  12 +-
 .../processors/cache/GridCacheSwapManager.java  | 118 ++++--
 .../internal/processors/hadoop/HadoopJob.java   |   2 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |   8 +-
 .../internal/processors/igfs/IgfsImpl.java      |   8 +-
 .../processors/igfs/IgfsIpcHandler.java         | 184 +++++----
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   9 +-
 .../internal/processors/igfs/IgfsServer.java    |   4 +-
 .../internal/processors/igfs/IgfsUtils.java     |  16 +
 .../ignite/internal/util/GridJavaProcess.java   |  30 +-
 .../ignite/mxbean/CacheMetricsMXBean.java       |  80 ++++
 .../apache/ignite/plugin/PluginProvider.java    |  26 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  35 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  47 ---
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   8 +-
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java | 412 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |  25 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |   1 +
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java | 165 +++++---
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    | 107 +++--
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  32 +-
 .../internal/processors/hadoop/HadoopUtils.java |  10 +-
 .../hadoop/SecondaryFileSystemProvider.java     |  53 ++-
 .../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/v2/HadoopV2TaskContext.java          |   4 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  56 ++-
 ...oopSecondaryFileSystemConfigurationTest.java |   4 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  63 ++-
 .../IgniteHadoopFileSystemClientSelfTest.java   |   2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   2 +
 .../hadoop/HadoopFileSystemsTest.java           |  23 +-
 .../collections/HadoopSkipListSelfTest.java     |   4 +-
 modules/mesos/pom.xml                           |   8 +-
 50 files changed, 2569 insertions(+), 813 deletions(-)
----------------------------------------------------------------------



[40/50] [abbrv] incubator-ignite git commit: # ignite-sprint-5 Added javadoc.

Posted by ag...@apache.org.
# ignite-sprint-5 Added 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/31fcc3e8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/31fcc3e8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/31fcc3e8

Branch: refs/heads/ignite-389
Commit: 31fcc3e8b760c2773e475770d819d433e8d19f4a
Parents: 5c30f9c
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 1 09:39:57 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 1 09:39:57 2015 +0300

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |  6 +++++
 .../apache/ignite/plugin/PluginProvider.java    | 26 ++++++++++++++------
 2 files changed, 25 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/31fcc3e8/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index ebe2b8e..3d4b64b 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -2188,15 +2188,21 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Gets plugin configurations.
+     *
      * @return Plugin configurations.
+     * @see PluginProvider
      */
     public PluginConfiguration[] getPluginConfigurations() {
         return pluginCfgs;
     }
 
     /**
+     * Sets plugin configurations.
+     *
      * @param pluginCfgs Plugin configurations.
      * @return {@code this} for chaining.
+     * @see PluginProvider
      */
     public IgniteConfiguration setPluginConfigurations(PluginConfiguration... pluginCfgs) {
         this.pluginCfgs = pluginCfgs;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/31fcc3e8/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java b/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
index 17bbc36..f064fde 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
@@ -19,13 +19,22 @@ package org.apache.ignite.plugin;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.util.*;
 
 /**
- * Pluggable ignite component.
+ * Pluggable Ignite component.
+ * <p>
+ * Ignite plugins are loaded using JDK {@link ServiceLoader}.
+ * First method called to initialize plugin is {@link PluginProvider#initExtensions(PluginContext, ExtensionRegistry)}.
+ * If plugin requires configuration it can be set in {@link IgniteConfiguration} using
+ * {@link IgniteConfiguration#setPluginConfigurations(PluginConfiguration...)}.
+ *
+ * @see IgniteConfiguration#setPluginConfigurations(PluginConfiguration...)
+ * @see PluginContext
  */
 public interface PluginProvider<C extends PluginConfiguration> {
     /**
@@ -49,18 +58,21 @@ public interface PluginProvider<C extends PluginConfiguration> {
     public <T extends IgnitePlugin> T plugin();
 
     /**
+     * Registers extensions.
+     *
      * @param ctx Plugin context.
-     * @param cls Ignite component class.
-     * @return Ignite component or {@code null} if component is not supported.
+     * @param registry Extension registry.
      */
-    @Nullable public <T> T createComponent(PluginContext ctx, Class<T> cls);
+    public void initExtensions(PluginContext ctx, ExtensionRegistry registry);
 
     /**
-     * Register extensions.
+     * Creates Ignite component.
+     *
      * @param ctx Plugin context.
-     * @param registry Extension registry.
+     * @param cls Ignite component class.
+     * @return Ignite component or {@code null} if component is not supported.
      */
-    public void initExtensions(PluginContext ctx, ExtensionRegistry registry);
+    @Nullable public <T> T createComponent(PluginContext ctx, Class<T> cls);
 
     /**
      * Starts grid component.


[32/50] [abbrv] incubator-ignite git commit: #gg-10369: Fix exception when we start ignite and gridgain nodes.

Posted by ag...@apache.org.
#gg-10369: Fix exception when we start ignite and gridgain nodes.


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

Branch: refs/heads/ignite-389
Commit: 5df06682c517731b3811ca4d0daabaa504b732f3
Parents: 8455c7a
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri May 29 16:57:30 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri May 29 16:57:30 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/GridJavaProcess.java   | 30 ++++++++++++--------
 1 file changed, 18 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5df06682/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
index bff26ec..42fe089 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
@@ -128,25 +128,31 @@ public final class GridJavaProcess {
         gjProc.log = log;
         gjProc.procKilledC = procKilledC;
 
-        String javaBin = System.getProperty("java.home") + File.separator + "bin" + File.separator + "java";
-        String classpath = System.getProperty("java.class.path");
-        String sfcp = System.getProperty("surefire.test.class.path");
-
-        if (sfcp != null)
-            classpath += System.getProperty("path.separator") + sfcp;
-
-        if (cp != null)
-            classpath += System.getProperty("path.separator") + cp;
-
         List<String> procParams = params == null || params.isEmpty() ?
             Collections.<String>emptyList() : Arrays.asList(params.split(" "));
 
         List<String> procCommands = new ArrayList<>();
 
+        String javaBin = System.getProperty("java.home") + File.separator + "bin" + File.separator + "java";
+
         procCommands.add(javaBin);
         procCommands.addAll(jvmArgs == null ? U.jvmArgs() : jvmArgs);
-        procCommands.add("-cp");
-        procCommands.add(classpath);
+
+        if (!jvmArgs.contains("-cp") && !jvmArgs.contains("-classpath")) {
+            String classpath = System.getProperty("java.class.path");
+
+            String sfcp = System.getProperty("surefire.test.class.path");
+
+            if (sfcp != null)
+                classpath += System.getProperty("path.separator") + sfcp;
+
+            if (cp != null)
+                classpath += System.getProperty("path.separator") + cp;
+
+            procCommands.add("-cp");
+            procCommands.add(classpath);
+        }
+
         procCommands.add(clsName);
         procCommands.addAll(procParams);
 


[23/50] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix TcpClientDiscoverySpiSelfTest.testMetrics

Posted by ag...@apache.org.
# IGNITE-943 Fix TcpClientDiscoverySpiSelfTest.testMetrics


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

Branch: refs/heads/ignite-389
Commit: ac258a5e06d3c8c9cca82b6883fc3af505a5e5dd
Parents: 2477c0e
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 13:47:00 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 13:47:00 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManagerAliveCacheSelfTest.java      | 2 +-
 .../processors/service/ClosureServiceClientsNodesTest.java     | 6 +-----
 2 files changed, 2 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac258a5e/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index af4dd0f..f2afb07 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -52,7 +52,7 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
     private static final int TMP_NODES_CNT = 3;
 
     /** */
-    private static final int ITERATIONS = 20;
+    private static final int ITERATIONS = 10;
 
     /** */
     private int gridCntr;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac258a5e/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
index 761f00f..59594be 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
@@ -53,11 +53,7 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
 
         cfg.setMarshaller(new OptimizedMarshaller(false));
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(ipFinder);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder).setForceServerMode(true));
 
         cfg.setCacheConfiguration();
 


[04/50] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-857' into ignite-857

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


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

Branch: refs/heads/ignite-389
Commit: 62d15409cbe2af2f91092c9bd248cb1109218812
Parents: 1c04c8b a36214c
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 17:48:06 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 17:48:06 2015 +0300

----------------------------------------------------------------------
 dev-tools/gradle/wrapper/gradle-wrapper.jar     | Bin 51017 -> 0 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |  18 +-
 dev-tools/gradlew                               | 163 ++++++++----
 dev-tools/slurp.sh                              |   2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  73 ++++--
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   5 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../org/apache/ignite/services/Service.java     |   5 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  20 +-
 .../service/ClosureServiceClientsNodesTest.java | 245 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   4 +
 .../query/h2/sql/BaseH2CompareQueryTest.java    |  16 ++
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 .../apache/ignite/mesos/IgniteScheduler.java    |  10 +-
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |   8 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 .../licenses/jcraft-revised-bsd.txt             |  28 ---
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 49 files changed, 508 insertions(+), 167 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/62d15409/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --cc modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index bca064f,9507642..64e2931
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@@ -44,10 -44,10 +44,10 @@@ public class IgniteScheduler implement
      public static final String DELIM = ",";
  
      /** Logger. */
 -    private static final Logger log = LoggerFactory.getLogger(IgniteScheduler.class);
 +    private static final Logger log = LogManager.getLogger(IgniteScheduler.class);
  
      /** Mutex. */
-     private static final Object mux = new Object();
+     private final Object mux = new Object();
  
      /** ID generator. */
      private AtomicInteger taskIdGenerator = new AtomicInteger();


[41/50] [abbrv] incubator-ignite git commit: # ignite-840 try evict partition on data load

Posted by ag...@apache.org.
# ignite-840 try evict partition on data 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/9f9a6315
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/9f9a6315
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/9f9a6315

Branch: refs/heads/ignite-389
Commit: 9f9a6315524f7500476704ec0321a58f43428fd0
Parents: 31fcc3e
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 1 10:28:51 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 1 10:28:51 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheEntryEx.java    | 6 ++++++
 .../ignite/internal/processors/cache/GridCacheMapEntry.java   | 5 +++++
 .../cache/distributed/GridDistributedCacheEntry.java          | 7 -------
 .../processors/cache/distributed/dht/GridDhtCacheEntry.java   | 6 +-----
 .../internal/processors/datastreamer/DataStreamerImpl.java    | 2 ++
 .../internal/processors/cache/GridCacheTestEntryEx.java       | 4 ++++
 .../datastreamer/DataStreamerMultiThreadedSelfTest.java       | 2 --
 7 files changed, 18 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9f9a6315/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 8d3d089..3857b35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.eviction.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.processors.dr.*;
@@ -943,4 +944,9 @@ public interface GridCacheEntryEx {
      * @return {@code True} if value was removed, {@code false} otherwise.
      */
     public <V> boolean removeMeta(UUID name, V val);
+
+    /**
+     * Calls {@link GridDhtLocalPartition#onUnlock()} for this entry's partition.
+     */
+    public void onUnlock();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9f9a6315/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 92035af..03638a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -4100,6 +4100,11 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     }
 
     /** {@inheritDoc} */
+    @Override public void onUnlock() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         // Identity comparison left on purpose.
         return o == this;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9f9a6315/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
index b79f9d5..bd72764 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
@@ -327,13 +327,6 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
     }
 
     /**
-     *
-     */
-    public void onUnlock() {
-        // No-op.
-    }
-
-    /**
      * Unlocks local lock.
      *
      * @return Removed candidate, or <tt>null</tt> if thread still holds the lock.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9f9a6315/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index c9a7af8..89b85c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -292,12 +292,8 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
         return ret;
     }
 
-    /**
-     * Calls {@link GridDhtLocalPartition#onUnlock()} for this entry's partition.
-     */
+    /** {@inheritDoc} */
     @Override public void onUnlock() {
-        super.onUnlock();
-
         locPart.onUnlock();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9f9a6315/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 db3d350..d16167a 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
@@ -1422,6 +1422,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     cctx.evicts().touch(entry, topVer);
 
                     CU.unwindEvicts(cctx);
+
+                    entry.onUnlock();
                 }
                 catch (GridDhtInvalidPartitionException | GridCacheEntryRemovedException ignored) {
                     // No-op.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9f9a6315/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 9a24109..eaa6e13 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -843,4 +843,8 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         return null;
     }
 
+    /** {@inheritDoc} */
+    @Override public void onUnlock() {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9f9a6315/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 bea3809..2382a66 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
@@ -68,8 +68,6 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStartStopIgnites() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-840");
-
         for (int attempt = 0; attempt < 3; ++attempt) {
             log.info("Iteration: " + attempt);
             


[11/50] [abbrv] incubator-ignite git commit: # ignite-456: add comment

Posted by ag...@apache.org.
# ignite-456: add comment


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

Branch: refs/heads/ignite-389
Commit: 3d77af8cb13552dfb8a183029959bb7b148a4018
Parents: e16cb03
Author: ashutak <as...@gridgain.com>
Authored: Thu May 28 19:18:10 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu May 28 19:18:10 2015 +0300

----------------------------------------------------------------------
 dev-tools/src/main/groovy/jiraslurp.groovy | 3 +++
 scripts/git-patch-prop.sh                  | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d77af8c/dev-tools/src/main/groovy/jiraslurp.groovy
----------------------------------------------------------------------
diff --git a/dev-tools/src/main/groovy/jiraslurp.groovy b/dev-tools/src/main/groovy/jiraslurp.groovy
index 3547337..8498cf0 100644
--- a/dev-tools/src/main/groovy/jiraslurp.groovy
+++ b/dev-tools/src/main/groovy/jiraslurp.groovy
@@ -420,6 +420,9 @@ def runAllTestBuilds = {builds, jiraNum ->
             else {
                 postData = "<build>" +
                         "  <buildType id='$it'/>" +
+                        "  <comment>" +
+                        "    <text>Auto triggered build to validate last attached patch file at $jiraNum.</text>" +
+                        "  </comment>" +
                         "  <properties>" +
                         "    <property name='env.JIRA_NUM' value='$jiraNum'/>" +
                         "  </properties>" +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d77af8c/scripts/git-patch-prop.sh
----------------------------------------------------------------------
diff --git a/scripts/git-patch-prop.sh b/scripts/git-patch-prop.sh
index 9c52583..c856fb4 100644
--- a/scripts/git-patch-prop.sh
+++ b/scripts/git-patch-prop.sh
@@ -19,6 +19,6 @@
 #
 # Git patch-file maker/applier properties.
 #
-IGNITE_DEFAULT_BRANCH='ignite-sprint-3'
+IGNITE_DEFAULT_BRANCH='ignite-sprint-5'
 
 PATCHES_HOME=${IGNITE_HOME}


[28/50] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-943

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-943


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

Branch: refs/heads/ignite-389
Commit: 477a2150ddc597b79846f862759dd6aed2cf4294
Parents: eb810196 07b6cb5
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 29 16:07:40 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 29 16:07:40 2015 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionMap.java      |   2 +-
 .../GridDhtPartitionsExchangeFuture.java        |   2 +-
 ...niteCacheClientNodeChangingTopologyTest.java |   6 ++
 .../IgniteCacheClientNodeConcurrentStart.java   | 105 +++++++++++++++++++
 ...teCacheClientNodePartitionsExchangeTest.java |   6 +-
 .../junits/common/GridCommonAbstractTest.java   |  61 +++++++++--
 .../testsuites/IgniteCacheTestSuite2.java       |   1 +
 7 files changed, 170 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/477a2150/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/477a2150/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
----------------------------------------------------------------------


[34/50] [abbrv] incubator-ignite git commit: # IGNITE-943 Increase test timeout.

Posted by ag...@apache.org.
# IGNITE-943 Increase test timeout.


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

Branch: refs/heads/ignite-389
Commit: b3d8e1e13bfc0049de5832ca1291ed5511e3f191
Parents: ccc0d40
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 17:11:44 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 17:11:44 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3d8e1e1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
index f5de96f..1d4d2f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
@@ -32,7 +32,7 @@ import java.util.concurrent.atomic.*;
 public class CacheRemoveAllSelfTest extends GridCacheAbstractSelfTest {
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
-        return 60000;
+        return 2 * 60 * 1000;
     }
 
     /** {@inheritDoc} */


[48/50] [abbrv] incubator-ignite git commit: # IGNITE-883 Don't create FileSwapSpi for clients.

Posted by ag...@apache.org.
# IGNITE-883 Don't create FileSwapSpi for clients.


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

Branch: refs/heads/ignite-389
Commit: 541b1e0f67089c3e6038ef787fd145977c84084b
Parents: 97d0b04
Author: sevdokimov <se...@gridgain.com>
Authored: Mon Jun 1 20:49:06 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Mon Jun 1 20:49:06 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/541b1e0f/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 6e4efb5..21f6652 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1844,7 +1844,7 @@ public class IgnitionEx {
             if (cfg.getSwapSpaceSpi() == null) {
                 boolean needSwap = false;
 
-                if (cfg.getCacheConfiguration() != null) {
+                if (cfg.getCacheConfiguration() != null && !Boolean.TRUE.equals(cfg.isClientMode())) {
                     for (CacheConfiguration c : cfg.getCacheConfiguration()) {
                         if (c.isSwapEnabled()) {
                             needSwap = true;


[26/50] [abbrv] incubator-ignite git commit: [IGNITE-958]: IGNITE-218 (Wrong staging permissions while running MR job under hadoop accelerator): IGFS part.

Posted by ag...@apache.org.
[IGNITE-958]: IGNITE-218 (Wrong staging permissions while running MR job under hadoop accelerator): IGFS part.


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

Branch: refs/heads/ignite-389
Commit: 353881951fcdcc16c3dc31d808d3af6c263f74ce
Parents: 7ec4c82
Author: iveselovskiy <iv...@gridgain.com>
Authored: Fri May 29 15:31:35 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Fri May 29 15:31:35 2015 +0300

----------------------------------------------------------------------
 .../igfs/secondary/IgfsSecondaryFileSystem.java |   7 +
 .../internal/igfs/common/IgfsMarshaller.java    |  35 +---
 .../igfs/common/IgfsPathControlRequest.java     |  22 +++
 .../internal/processors/hadoop/HadoopJob.java   |   2 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |   8 +-
 .../internal/processors/igfs/IgfsImpl.java      |   8 +-
 .../processors/igfs/IgfsIpcHandler.java         | 184 ++++++++++---------
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   9 +-
 .../internal/processors/igfs/IgfsServer.java    |   4 +-
 .../internal/processors/igfs/IgfsUtils.java     |  16 ++
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java | 165 ++++++++++++-----
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    | 107 +++++++----
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  32 +++-
 .../internal/processors/hadoop/HadoopUtils.java |  10 +-
 .../hadoop/SecondaryFileSystemProvider.java     |  53 +++---
 .../hadoop/fs/HadoopDistributedFileSystem.java  |  91 ---------
 .../hadoop/fs/HadoopFileSystemsUtils.java       |  17 --
 .../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/v2/HadoopV2TaskContext.java          |   4 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  56 ++++--
 ...oopSecondaryFileSystemConfigurationTest.java |   4 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  63 +++++--
 .../IgniteHadoopFileSystemClientSelfTest.java   |   2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   2 +
 .../hadoop/HadoopFileSystemsTest.java           |  23 +--
 .../collections/HadoopSkipListSelfTest.java     |   4 +-
 30 files changed, 684 insertions(+), 474 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
index 9026eac..cb69352 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
@@ -198,4 +198,11 @@ public interface IgfsSecondaryFileSystem {
      * @return Map of properties.
      */
     public Map<String,String> properties();
+
+
+    /**
+     * Closes the secondary file system.
+     * @throws IgniteException in case of an error.
+     */
+    public void close() throws IgniteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
index 11af716..6a6f22a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
@@ -73,6 +73,7 @@ public class IgfsMarshaller {
     }
 
     /**
+     * Serializes the message and sends it into the given output stream.
      * @param msg Message.
      * @param hdr Message header.
      * @param out Output.
@@ -119,6 +120,7 @@ public class IgfsMarshaller {
 
                     IgfsPathControlRequest req = (IgfsPathControlRequest)msg;
 
+                    U.writeString(out, req.userName());
                     writePath(out, req.path());
                     writePath(out, req.destinationPath());
                     out.writeBoolean(req.flag());
@@ -236,6 +238,7 @@ public class IgfsMarshaller {
                 case OPEN_CREATE: {
                     IgfsPathControlRequest req = new IgfsPathControlRequest();
 
+                    req.userName(U.readString(in));
                     req.path(readPath(in));
                     req.destinationPath(readPath(in));
                     req.flag(in.readBoolean());
@@ -298,8 +301,6 @@ public class IgfsMarshaller {
                 }
             }
 
-            assert msg != null;
-
             msg.command(cmd);
 
             return msg;
@@ -341,34 +342,4 @@ public class IgfsMarshaller {
 
         return null;
     }
-
-    /**
-     * Writes string to output.
-     *
-     * @param out Data output.
-     * @param str String.
-     * @throws IOException If write failed.
-     */
-    private void writeString(DataOutput out, @Nullable String str) throws IOException {
-        out.writeBoolean(str != null);
-
-        if (str != null)
-            out.writeUTF(str);
-    }
-
-    /**
-     * Reads string from input.
-     *
-     * @param in Data input.
-     * @return Read string.
-     * @throws IOException If read failed.
-     */
-    @Nullable private String readString(DataInput in) throws IOException {
-        boolean hasStr = in.readBoolean();
-
-        if (hasStr)
-            return in.readUTF();
-
-        return null;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsPathControlRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsPathControlRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsPathControlRequest.java
index 7ed1619..2f6e6e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsPathControlRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsPathControlRequest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.igfs.common;
 
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -63,6 +64,9 @@ public class IgfsPathControlRequest extends IgfsMessage {
     /** Last modification time. */
     private long modificationTime;
 
+    /** The user name this control request is made on behalf of. */
+    private String userName;
+
     /**
      * @param path Path.
      */
@@ -235,4 +239,22 @@ public class IgfsPathControlRequest extends IgfsMessage {
     @Override public String toString() {
         return S.toString(IgfsPathControlRequest.class, this, "cmd", command());
     }
+
+    /**
+     * Getter for the user name.
+     * @return user name.
+     */
+    public final String userName() {
+        assert userName != null;
+
+        return userName;
+    }
+
+    /**
+     * Setter for the user name.
+     * @param userName the user name.
+     */
+    public final void userName(String userName) {
+        this.userName = IgfsUtils.fixUserName(userName);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
index 65cb48d..5fd6c81 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
@@ -98,5 +98,5 @@ public interface HadoopJob {
     /**
      * Cleans up the job staging directory.
      */
-    void cleanupStagingDirectory();
+    public void cleanupStagingDirectory();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
index 7c1a837..361f75f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
@@ -48,8 +48,12 @@ public interface IgfsEx extends IgniteFileSystem {
     /** Property name for URI of file system. */
     public static final String SECONDARY_FS_URI = "SECONDARY_FS_URI";
 
-    /** Property name for user name of file system. */
-    public static final String SECONDARY_FS_USER_NAME = "SECONDARY_FS_USER_NAME";
+    /** Property name for default user name of file system.
+     * NOTE: for secondary file system this is just a default user name, which is used
+     * when the 2ndary filesystem is used outside of any user context.
+     * If another user name is set in the context, 2ndary file system will work on behalf
+     * of that user, which is different from the default. */
+     public static final String SECONDARY_FS_USER_NAME = "SECONDARY_FS_USER_NAME";
 
     /**
      * Stops IGFS cleaning all used resources.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 34636d2..c3495e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -245,8 +245,12 @@ public final class IgfsImpl implements IgfsEx {
             for (IgfsFileWorkerBatch batch : workerMap.values())
                 batch.cancel();
 
-            if (secondaryFs instanceof AutoCloseable)
-                U.closeQuiet((AutoCloseable)secondaryFs);
+            try {
+                secondaryFs.close();
+            }
+            catch (Exception e) {
+                log.error("Failed to close secondary file system.", e);
+            }
         }
 
         igfsCtx.kernalContext().io().removeMessageListener(topic, delMsgLsnr);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
index 8a8b858..cfe6ed4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
@@ -51,10 +51,10 @@ class IgfsIpcHandler implements IgfsServerHandler {
     private final int bufSize; // Buffer size. Must not be less then file block size.
 
     /** IGFS instance for this handler. */
-    private IgfsEx igfs;
+    private final IgfsEx igfs;
 
     /** Resource ID generator. */
-    private AtomicLong rsrcIdGen = new AtomicLong();
+    private final AtomicLong rsrcIdGen = new AtomicLong();
 
     /** Stopping flag. */
     private volatile boolean stopping;
@@ -241,138 +241,148 @@ class IgfsIpcHandler implements IgfsServerHandler {
      * @return Response message.
      * @throws IgniteCheckedException If failed.
      */
-    private IgfsMessage processPathControlRequest(IgfsClientSession ses, IgfsIpcCommand cmd,
+    private IgfsMessage processPathControlRequest(final IgfsClientSession ses, final IgfsIpcCommand cmd,
         IgfsMessage msg) throws IgniteCheckedException {
-        IgfsPathControlRequest req = (IgfsPathControlRequest)msg;
+        final IgfsPathControlRequest req = (IgfsPathControlRequest)msg;
 
         if (log.isDebugEnabled())
             log.debug("Processing path control request [igfsName=" + igfs.name() + ", req=" + req + ']');
 
-        IgfsControlResponse res = new IgfsControlResponse();
+        final IgfsControlResponse res = new IgfsControlResponse();
+
+        final String userName = req.userName();
+
+        assert userName != null;
 
         try {
-            switch (cmd) {
-                case EXISTS:
-                    res.response(igfs.exists(req.path()));
+            IgfsUserContext.doAs(userName, new IgniteOutClosure<Object>() {
+                @Override public Void apply() {
+                    switch (cmd) {
+                        case EXISTS:
+                            res.response(igfs.exists(req.path()));
 
-                    break;
+                            break;
 
-                case INFO:
-                    res.response(igfs.info(req.path()));
+                        case INFO:
+                            res.response(igfs.info(req.path()));
 
-                    break;
+                            break;
 
-                case PATH_SUMMARY:
-                    res.response(igfs.summary(req.path()));
+                        case PATH_SUMMARY:
+                            res.response(igfs.summary(req.path()));
 
-                    break;
+                            break;
 
-                case UPDATE:
-                    res.response(igfs.update(req.path(), req.properties()));
+                        case UPDATE:
+                            res.response(igfs.update(req.path(), req.properties()));
 
-                    break;
+                            break;
 
-                case RENAME:
-                    igfs.rename(req.path(), req.destinationPath());
+                        case RENAME:
+                            igfs.rename(req.path(), req.destinationPath());
 
-                    res.response(true);
+                            res.response(true);
 
-                    break;
+                            break;
 
-                case DELETE:
-                    res.response(igfs.delete(req.path(), req.flag()));
+                        case DELETE:
+                            res.response(igfs.delete(req.path(), req.flag()));
 
-                    break;
+                            break;
 
-                case MAKE_DIRECTORIES:
-                    igfs.mkdirs(req.path(), req.properties());
+                        case MAKE_DIRECTORIES:
+                            igfs.mkdirs(req.path(), req.properties());
 
-                    res.response(true);
+                            res.response(true);
 
-                    break;
+                            break;
 
-                case LIST_PATHS:
-                    res.paths(igfs.listPaths(req.path()));
+                        case LIST_PATHS:
+                            res.paths(igfs.listPaths(req.path()));
 
-                    break;
+                            break;
 
-                case LIST_FILES:
-                    res.files(igfs.listFiles(req.path()));
+                        case LIST_FILES:
+                            res.files(igfs.listFiles(req.path()));
 
-                    break;
+                            break;
 
-                case SET_TIMES:
-                    igfs.setTimes(req.path(), req.accessTime(), req.modificationTime());
+                        case SET_TIMES:
+                            igfs.setTimes(req.path(), req.accessTime(), req.modificationTime());
 
-                    res.response(true);
+                            res.response(true);
 
-                    break;
+                            break;
 
-                case AFFINITY:
-                    res.locations(igfs.affinity(req.path(), req.start(), req.length()));
+                        case AFFINITY:
+                            res.locations(igfs.affinity(req.path(), req.start(), req.length()));
 
-                    break;
+                            break;
 
-                case OPEN_READ: {
-                    IgfsInputStreamAdapter igfsIn = !req.flag() ? igfs.open(req.path(), bufSize) :
-                        igfs.open(req.path(), bufSize, req.sequentialReadsBeforePrefetch());
+                        case OPEN_READ: {
+                            IgfsInputStreamAdapter igfsIn = !req.flag() ? igfs.open(req.path(), bufSize) :
+                                igfs.open(req.path(), bufSize, req.sequentialReadsBeforePrefetch());
 
-                    long streamId = registerResource(ses, igfsIn);
+                            long streamId = registerResource(ses, igfsIn);
 
-                    if (log.isDebugEnabled())
-                        log.debug("Opened IGFS input stream for file read [igfsName=" + igfs.name() + ", path=" +
-                            req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
+                            if (log.isDebugEnabled())
+                                log.debug("Opened IGFS input stream for file read [igfsName=" + igfs.name() + ", path=" +
+                                    req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
 
-                    IgfsFileInfo info = new IgfsFileInfo(igfsIn.fileInfo(), null,
-                        igfsIn.fileInfo().modificationTime());
+                            IgfsFileInfo info = new IgfsFileInfo(igfsIn.fileInfo(), null,
+                                igfsIn.fileInfo().modificationTime());
 
-                    res.response(new IgfsInputStreamDescriptor(streamId, info.length()));
+                            res.response(new IgfsInputStreamDescriptor(streamId, info.length()));
 
-                    break;
-                }
+                            break;
+                        }
 
-                case OPEN_CREATE: {
-                    long streamId = registerResource(ses, igfs.create(
-                        req.path(),       // Path.
-                        bufSize,          // Buffer size.
-                        req.flag(),       // Overwrite if exists.
-                        affinityKey(req), // Affinity key based on replication factor.
-                        req.replication(),// Replication factor.
-                        req.blockSize(),  // Block size.
-                        req.properties()  // File properties.
-                    ));
+                        case OPEN_CREATE: {
+                            long streamId = registerResource(ses, igfs.create(
+                                req.path(),       // Path.
+                                bufSize,          // Buffer size.
+                                req.flag(),       // Overwrite if exists.
+                                affinityKey(req), // Affinity key based on replication factor.
+                                req.replication(),// Replication factor.
+                                req.blockSize(),  // Block size.
+                                req.properties()  // File properties.
+                            ));
 
-                    if (log.isDebugEnabled())
-                        log.debug("Opened IGFS output stream for file create [igfsName=" + igfs.name() + ", path=" +
-                            req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
+                            if (log.isDebugEnabled())
+                                log.debug("Opened IGFS output stream for file create [igfsName=" + igfs.name() + ", path=" +
+                                    req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
 
-                    res.response(streamId);
+                            res.response(streamId);
 
-                    break;
-                }
+                            break;
+                        }
 
-                case OPEN_APPEND: {
-                    long streamId = registerResource(ses, igfs.append(
-                        req.path(),        // Path.
-                        bufSize,           // Buffer size.
-                        req.flag(),        // Create if absent.
-                        req.properties()   // File properties.
-                    ));
+                        case OPEN_APPEND: {
+                            long streamId = registerResource(ses, igfs.append(
+                                req.path(),        // Path.
+                                bufSize,           // Buffer size.
+                                req.flag(),        // Create if absent.
+                                req.properties()   // File properties.
+                            ));
 
-                    if (log.isDebugEnabled())
-                        log.debug("Opened IGFS output stream for file append [igfsName=" + igfs.name() + ", path=" +
-                            req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
+                            if (log.isDebugEnabled())
+                                log.debug("Opened IGFS output stream for file append [igfsName=" + igfs.name() + ", path=" +
+                                    req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
 
-                    res.response(streamId);
+                            res.response(streamId);
 
-                    break;
-                }
+                            break;
+                        }
 
-                default:
-                    assert false : "Unhandled path control request command: " + cmd;
+                        default:
+                            assert false : "Unhandled path control request command: " + cmd;
 
-                    break;
-            }
+                            break;
+                    }
+
+                    return null;
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/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 683b317..b8095b8 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
@@ -30,14 +30,14 @@ import java.util.*;
  */
 class IgfsSecondaryFileSystemImpl implements IgfsSecondaryFileSystem {
     /** Delegate. */
-    private final IgfsImpl igfs;
+    private final IgfsEx igfs;
 
     /**
      * Constructor.
      *
      * @param igfs Delegate.
      */
-    IgfsSecondaryFileSystemImpl(IgfsImpl igfs) {
+    IgfsSecondaryFileSystemImpl(IgfsEx igfs) {
         this.igfs = igfs;
     }
 
@@ -118,4 +118,9 @@ class IgfsSecondaryFileSystemImpl implements IgfsSecondaryFileSystem {
     @Override public Map<String, String> properties() {
         return Collections.emptyMap();
     }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IgniteException {
+        igfs.stop(true);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java
index 253d5be..caa6866 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java
@@ -239,13 +239,13 @@ public class IgfsServer {
      */
     private class ClientWorker extends GridWorker {
         /** Connected client endpoint. */
-        private IpcEndpoint endpoint;
+        private final IpcEndpoint endpoint;
 
         /** Data output stream. */
         private final IgfsDataOutputStream out;
 
         /** Client session object. */
-        private IgfsClientSession ses;
+        private final IgfsClientSession ses;
 
         /** Queue node for fast unlink. */
         private ConcurrentLinkedDeque8.Node<ClientWorker> node;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
index 4b0234f..8026a44 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
@@ -18,9 +18,11 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.jetbrains.annotations.*;
 
 import java.lang.reflect.*;
 
@@ -88,4 +90,18 @@ public class IgfsUtils {
     private IgfsUtils() {
         // No-op.
     }
+
+    /**
+     * Provides non-null user name.
+     * If the user name is null or empty string, defaults to {@link FileSystemConfiguration#DFLT_USER_NAME},
+     * which is the current process owner user.
+     * @param user a user name to be fixed.
+     * @return non-null interned user name.
+     */
+    public static String fixUserName(@Nullable String user) {
+        if (F.isEmpty(user))
+           user = FileSystemConfiguration.DFLT_USER_NAME;
+
+        return user;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
index ba891f8..6a630fb 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
@@ -20,15 +20,16 @@ package org.apache.ignite.hadoop.fs;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.ipc.*;
 import org.apache.ignite.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
 import org.apache.ignite.internal.processors.hadoop.igfs.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.jetbrains.annotations.*;
+import org.apache.ignite.internal.processors.hadoop.fs.HadoopLazyConcurrentMap.*;
 
 import java.io.*;
 import java.net.*;
@@ -37,15 +38,45 @@ import java.util.*;
 import static org.apache.ignite.internal.processors.igfs.IgfsEx.*;
 
 /**
- * Adapter to use any Hadoop file system {@link FileSystem} as  {@link IgfsSecondaryFileSystem}.
+ * Adapter to use any Hadoop file system {@link FileSystem} as {@link IgfsSecondaryFileSystem}.
+ * In fact, this class deals with different FileSystems depending on the user context,
+ * see {@link IgfsUserContext#currentUser()}.
  */
-public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem, AutoCloseable {
-    /** Hadoop file system. */
-    private final FileSystem fileSys;
-
-    /** Properties of file system */
+public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem {
+    /** Properties of file system, see {@link #properties()}
+     *
+     * See {@link IgfsEx#SECONDARY_FS_CONFIG_PATH}
+     * See {@link IgfsEx#SECONDARY_FS_URI}
+     * See {@link IgfsEx#SECONDARY_FS_USER_NAME}
+     * */
     private final Map<String, String> props = new HashMap<>();
 
+    /** Secondary file system provider. */
+    private final SecondaryFileSystemProvider secProvider;
+
+    /** The default user name. It is used if no user context is set. */
+    private final String dfltUserName;
+
+    /** FileSystem instance created for the default user.
+     * Stored outside the fileSysLazyMap due to performance reasons. */
+    private final FileSystem dfltFs;
+
+    /** Lazy per-user cache for the file systems. It is cleared and nulled in #close() method. */
+    private final HadoopLazyConcurrentMap<String, FileSystem> fileSysLazyMap = new HadoopLazyConcurrentMap<>(
+        new ValueFactory<String, FileSystem>() {
+            @Override public FileSystem createValue(String key) {
+                try {
+                    assert !F.isEmpty(key);
+
+                    return secProvider.createFileSystem(key);
+                }
+                catch (IOException ioe) {
+                    throw new IgniteException(ioe);
+                }
+            }
+        }
+    );
+
     /**
      * Simple constructor that is to be used by default.
      *
@@ -77,8 +108,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
      * @throws IgniteCheckedException In case of error.
      */
     public IgniteHadoopIgfsSecondaryFileSystem(@Nullable String uri, @Nullable String cfgPath,
-        @Nullable String userName)
-            throws IgniteCheckedException {
+        @Nullable String userName) throws IgniteCheckedException {
         // Treat empty uri and userName arguments as nulls to improve configuration usability:
         if (F.isEmpty(uri))
             uri = null;
@@ -89,27 +119,31 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
         if (F.isEmpty(userName))
             userName = null;
 
+        this.dfltUserName = IgfsUtils.fixUserName(userName);
+
         try {
-            SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(uri, cfgPath, userName);
+            this.secProvider = new SecondaryFileSystemProvider(uri, cfgPath);
 
-            fileSys = secProvider.createFileSystem();
+            // File system creation for the default user name.
+            // The value is *not* stored in the 'fileSysLazyMap' cache, but saved in field:
+            this.dfltFs = secProvider.createFileSystem(dfltUserName);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
 
-            uri = secProvider.uri().toString();
+        assert dfltFs != null;
 
-            if (!uri.endsWith("/"))
-                uri += "/";
+        uri = secProvider.uri().toString();
 
-            if (cfgPath != null)
-                props.put(SECONDARY_FS_CONFIG_PATH, cfgPath);
+        if (!uri.endsWith("/"))
+            uri += "/";
 
-            if (userName != null)
-                props.put(SECONDARY_FS_USER_NAME, userName);
+        if (cfgPath != null)
+            props.put(SECONDARY_FS_CONFIG_PATH, cfgPath);
 
-            props.put(SECONDARY_FS_URI, uri);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
+        props.put(SECONDARY_FS_URI, uri);
+        props.put(SECONDARY_FS_USER_NAME, dfltUserName);
     }
 
     /**
@@ -119,7 +153,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
      * @return Hadoop path.
      */
     private Path convert(IgfsPath path) {
-        URI uri = fileSys.getUri();
+        URI uri = fileSysForUser().getUri();
 
         return new Path(uri.getScheme(), uri.getAuthority(), path.toString());
     }
@@ -131,14 +165,9 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
      * @param detailMsg Detailed error message.
      * @return Appropriate exception.
      */
-    @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "unchecked"})
     private IgfsException handleSecondaryFsError(IOException e, String detailMsg) {
-        boolean wrongVer = X.hasCause(e, RemoteException.class) ||
-            (e.getMessage() != null && e.getMessage().contains("Failed on local"));
-
-        return !wrongVer ? cast(detailMsg, e) :
-            new IgfsInvalidHdfsVersionException("HDFS version you are connecting to differs from local " +
-                "version.", e);    }
+        return cast(detailMsg, e);
+    }
 
     /**
      * Cast IO exception to IGFS exception.
@@ -178,7 +207,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public boolean exists(IgfsPath path) {
         try {
-            return fileSys.exists(convert(path));
+            return fileSysForUser().exists(convert(path));
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to check file existence [path=" + path + "]");
@@ -189,6 +218,8 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     @Nullable @Override public IgfsFile update(IgfsPath path, Map<String, String> props) {
         HadoopIgfsProperties props0 = new HadoopIgfsProperties(props);
 
+        final FileSystem fileSys = fileSysForUser();
+
         try {
             if (props0.userName() != null || props0.groupName() != null)
                 fileSys.setOwner(convert(path), props0.userName(), props0.groupName());
@@ -208,7 +239,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     @Override public void rename(IgfsPath src, IgfsPath dest) {
         // Delegate to the secondary file system.
         try {
-            if (!fileSys.rename(convert(src), convert(dest)))
+            if (!fileSysForUser().rename(convert(src), convert(dest)))
                 throw new IgfsException("Failed to rename (secondary file system returned false) " +
                     "[src=" + src + ", dest=" + dest + ']');
         }
@@ -220,7 +251,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public boolean delete(IgfsPath path, boolean recursive) {
         try {
-            return fileSys.delete(convert(path), recursive);
+            return fileSysForUser().delete(convert(path), recursive);
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to delete file [path=" + path + ", recursive=" + recursive + "]");
@@ -230,7 +261,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public void mkdirs(IgfsPath path) {
         try {
-            if (!fileSys.mkdirs(convert(path)))
+            if (!fileSysForUser().mkdirs(convert(path)))
                 throw new IgniteException("Failed to make directories [path=" + path + "]");
         }
         catch (IOException e) {
@@ -241,7 +272,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) {
         try {
-            if (!fileSys.mkdirs(convert(path), new HadoopIgfsProperties(props).permission()))
+            if (!fileSysForUser().mkdirs(convert(path), new HadoopIgfsProperties(props).permission()))
                 throw new IgniteException("Failed to make directories [path=" + path + ", props=" + props + "]");
         }
         catch (IOException e) {
@@ -252,7 +283,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public Collection<IgfsPath> listPaths(IgfsPath path) {
         try {
-            FileStatus[] statuses = fileSys.listStatus(convert(path));
+            FileStatus[] statuses = fileSysForUser().listStatus(convert(path));
 
             if (statuses == null)
                 throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path);
@@ -275,7 +306,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public Collection<IgfsFile> listFiles(IgfsPath path) {
         try {
-            FileStatus[] statuses = fileSys.listStatus(convert(path));
+            FileStatus[] statuses = fileSysForUser().listStatus(convert(path));
 
             if (statuses == null)
                 throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path);
@@ -302,13 +333,13 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
 
     /** {@inheritDoc} */
     @Override public IgfsSecondaryFileSystemPositionedReadable open(IgfsPath path, int bufSize) {
-        return new HadoopIgfsSecondaryFileSystemPositionedReadable(fileSys, convert(path), bufSize);
+        return new HadoopIgfsSecondaryFileSystemPositionedReadable(fileSysForUser(), convert(path), bufSize);
     }
 
     /** {@inheritDoc} */
     @Override public OutputStream create(IgfsPath path, boolean overwrite) {
         try {
-            return fileSys.create(convert(path), overwrite);
+            return fileSysForUser().create(convert(path), overwrite);
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", overwrite=" + overwrite + "]");
@@ -322,8 +353,8 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
             new HadoopIgfsProperties(props != null ? props : Collections.<String, String>emptyMap());
 
         try {
-            return fileSys.create(convert(path), props0.permission(), overwrite, bufSize, (short)replication, blockSize,
-                null);
+            return fileSysForUser().create(convert(path), props0.permission(), overwrite, bufSize,
+                (short)replication, blockSize, null);
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", props=" + props +
@@ -336,7 +367,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     @Override public OutputStream append(IgfsPath path, int bufSize, boolean create,
         @Nullable Map<String, String> props) {
         try {
-            return fileSys.append(convert(path), bufSize);
+            return fileSysForUser().append(convert(path), bufSize);
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to append file [path=" + path + ", bufSize=" + bufSize + "]");
@@ -346,7 +377,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public IgfsFile info(final IgfsPath path) {
         try {
-            final FileStatus status = fileSys.getFileStatus(convert(path));
+            final FileStatus status = fileSysForUser().getFileStatus(convert(path));
 
             if (status == null)
                 return null;
@@ -421,7 +452,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
         try {
             // We don't use FileSystem#getUsed() since it counts only the files
             // in the filesystem root, not all the files recursively.
-            return fileSys.getContentSummary(new Path("/")).getSpaceConsumed();
+            return fileSysForUser().getContentSummary(new Path("/")).getSpaceConsumed();
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to get used space size of file system.");
@@ -429,25 +460,57 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<String, String> properties() {
+    @Override public Map<String, String> properties() {
         return props;
     }
 
     /** {@inheritDoc} */
-    @Override public void close() throws IgniteCheckedException {
+    @Override public void close() throws IgniteException {
+        Exception e = null;
+
         try {
-            fileSys.close();
+            dfltFs.close();
         }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
+        catch (Exception e0) {
+            e = e0;
+        }
+
+        try {
+            fileSysLazyMap.close();
+        }
+        catch (IgniteCheckedException ice) {
+            if (e == null)
+                e = ice;
         }
+
+        if (e != null)
+            throw new IgniteException(e);
     }
 
     /**
      * Gets the underlying {@link FileSystem}.
+     * This method is used solely for testing.
      * @return the underlying Hadoop {@link FileSystem}.
      */
     public FileSystem fileSystem() {
-        return fileSys;
+        return fileSysForUser();
+    }
+
+    /**
+     * Gets the FileSystem for the current context user.
+     * @return the FileSystem instance, never null.
+     */
+    private FileSystem fileSysForUser() {
+        String user = IgfsUserContext.currentUser();
+
+        if (F.isEmpty(user))
+            user = dfltUserName; // default is never empty.
+
+        assert !F.isEmpty(user);
+
+        if (F.eq(user, dfltUserName))
+            return dfltFs; // optimization
+
+        return fileSysLazyMap.getOrCreate(user);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/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 1f53a06..c0a9ade 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
@@ -23,6 +23,7 @@ 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.*;
 import org.apache.ignite.igfs.*;
@@ -97,21 +98,8 @@ public class IgniteHadoopFileSystem extends FileSystem {
     /** Grid remote client. */
     private HadoopIgfsWrapper rmtClient;
 
-    /** User name for each thread. */
-    private final ThreadLocal<String> userName = new ThreadLocal<String>(){
-        /** {@inheritDoc} */
-        @Override protected String initialValue() {
-            return DFLT_USER_NAME;
-        }
-    };
-
-    /** Working directory for each thread. */
-    private final ThreadLocal<Path> workingDir = new ThreadLocal<Path>(){
-        /** {@inheritDoc} */
-        @Override protected Path initialValue() {
-            return getHomeDirectory();
-        }
-    };
+    /** working directory. */
+    private Path workingDir;
 
     /** Default replication factor. */
     private short dfltReplication;
@@ -129,6 +117,9 @@ public class IgniteHadoopFileSystem extends FileSystem {
     /** Secondary URI string. */
     private URI secondaryUri;
 
+    /** The user name this file system was created on behalf of. */
+    private String user;
+
     /** IGFS mode resolver. */
     private IgfsModeResolver modeRslvr;
 
@@ -182,6 +173,36 @@ public class IgniteHadoopFileSystem extends FileSystem {
     }
 
     /**
+     * Gets non-null and interned 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();
+        }
+
+        user = IgfsUtils.fixUserName(user);
+
+        assert user != null;
+
+        return user;
+    }
+
+    /**
      * Public setter that can be used by direct users of FS or Visor.
      *
      * @param colocateFileWrites Whether all ongoing file writes should be colocated.
@@ -221,7 +242,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             uriAuthority = uri.getAuthority();
 
-            setUser(cfg.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
+            user = getFsHadoopUser(cfg);
 
             // Override sequential reads before prefetch if needed.
             seqReadsBeforePrefetch = parameter(cfg, PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH, uriAuthority, 0);
@@ -244,7 +265,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             String logDir = logDirFile != null ? logDirFile.getAbsolutePath() : null;
 
-            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG);
+            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG, user);
 
             // Handshake.
             IgfsHandshakeResponse handshake = rmtClient.handshake(logDir);
@@ -289,13 +310,12 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
                 String secUri = props.get(SECONDARY_FS_URI);
                 String secConfPath = props.get(SECONDARY_FS_CONFIG_PATH);
-                String secUserName = props.get(SECONDARY_FS_USER_NAME);
 
                 try {
-                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath,
-                        secUserName);
+                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath);
+
+                    secondaryFs = secProvider.createFileSystem(user);
 
-                    secondaryFs = secProvider.createFileSystem();
                     secondaryUri = secProvider.uri();
                 }
                 catch (IOException e) {
@@ -306,6 +326,9 @@ public class IgniteHadoopFileSystem extends FileSystem {
                             "will have no effect): " + e.getMessage());
                 }
             }
+
+            // set working directory to the home directory of the current Fs user:
+            setWorkingDirectory(null);
         }
         finally {
             leaveBusy();
@@ -849,22 +872,11 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
     /** {@inheritDoc} */
     @Override public Path getHomeDirectory() {
-        Path path = new Path("/user/" + userName.get());
+        Path path = new Path("/user/" + user);
 
         return path.makeQualified(getUri(), null);
     }
 
-    /**
-     * Set user name and default working directory for current thread.
-     *
-     * @param userName User name.
-     */
-    public void setUser(String userName) {
-        this.userName.set(userName);
-
-        setWorkingDirectory(null);
-    }
-
     /** {@inheritDoc} */
     @Override public void setWorkingDirectory(Path newPath) {
         if (newPath == null) {
@@ -873,7 +885,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
             if (secondaryFs != null)
                 secondaryFs.setWorkingDirectory(toSecondary(homeDir));
 
-            workingDir.set(homeDir);
+            workingDir = homeDir;
         }
         else {
             Path fixedNewPath = fixRelativePart(newPath);
@@ -886,13 +898,13 @@ public class IgniteHadoopFileSystem extends FileSystem {
             if (secondaryFs != null)
                 secondaryFs.setWorkingDirectory(toSecondary(fixedNewPath));
 
-            workingDir.set(fixedNewPath);
+            workingDir = fixedNewPath;
         }
     }
 
     /** {@inheritDoc} */
     @Override public Path getWorkingDirectory() {
-        return workingDir.get();
+        return workingDir;
     }
 
     /** {@inheritDoc} */
@@ -1153,7 +1165,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
             return null;
 
         return path.isAbsolute() ? new IgfsPath(path.toUri().getPath()) :
-            new IgfsPath(convert(workingDir.get()), path.toUri().getPath());
+            new IgfsPath(convert(workingDir), path.toUri().getPath());
     }
 
     /**
@@ -1191,9 +1203,16 @@ public class IgniteHadoopFileSystem extends FileSystem {
      */
     @SuppressWarnings("deprecation")
     private FileStatus convert(IgfsFile file) {
-        return new FileStatus(file.length(), file.isDirectory(), getDefaultReplication(),
-            file.groupBlockSize(), file.modificationTime(), file.accessTime(), permission(file),
-            file.property(PROP_USER_NAME, DFLT_USER_NAME), file.property(PROP_GROUP_NAME, "users"),
+        return new FileStatus(
+            file.length(),
+            file.isDirectory(),
+            getDefaultReplication(),
+            file.groupBlockSize(),
+            file.modificationTime(),
+            file.accessTime(),
+            permission(file),
+            file.property(PROP_USER_NAME, user),
+            file.property(PROP_GROUP_NAME, "users"),
             convert(file.path())) {
             @Override public String toString() {
                 return "FileStatus [path=" + getPath() + ", isDir=" + isDir() + ", len=" + getLen() +
@@ -1247,4 +1266,12 @@ public class IgniteHadoopFileSystem extends FileSystem {
     @Override public String toString() {
         return S.toString(IgniteHadoopFileSystem.class, this);
     }
+
+    /**
+     * Returns the user name this File System is created on behalf of.
+     * @return the user name
+     */
+    public String user() {
+        return user;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/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 9cfb79b..f3fbe9c 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
@@ -22,7 +22,6 @@ import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.util.*;
 import org.apache.ignite.*;
 import org.apache.ignite.igfs.*;
@@ -40,6 +39,7 @@ import java.util.*;
 import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.configuration.FileSystemConfiguration.*;
+import static org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem.*;
 import static org.apache.ignite.igfs.IgfsMode.*;
 import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
 import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.*;
@@ -91,11 +91,14 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
     /** Grid remote client. */
     private HadoopIgfsWrapper rmtClient;
 
+    /** The name of the user this File System created on behalf of. */
+    private final String user;
+
     /** Working directory. */
     private IgfsPath workingDir;
 
     /** URI. */
-    private URI uri;
+    private final URI uri;
 
     /** Authority. */
     private String uriAuthority;
@@ -141,6 +144,8 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
         uri = name;
 
+        user = getFsHadoopUser(cfg);
+
         try {
             initialize(name, cfg);
         }
@@ -152,7 +157,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
             throw e;
         }
 
-        workingDir = new IgfsPath("/user/" + cfg.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
+        workingDir = new IgfsPath("/user/" + user);
     }
 
     /** {@inheritDoc} */
@@ -240,7 +245,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
             String logDir = logDirFile != null ? logDirFile.getAbsolutePath() : null;
 
-            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG);
+            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG, user);
 
             // Handshake.
             IgfsHandshakeResponse handshake = rmtClient.handshake(logDir);
@@ -284,13 +289,12 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
                 String secUri = props.get(SECONDARY_FS_URI);
                 String secConfPath = props.get(SECONDARY_FS_CONFIG_PATH);
-                String secUserName = props.get(SECONDARY_FS_USER_NAME);
 
                 try {
-                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath,
-                        secUserName);
+                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath);
+
+                    secondaryFs = secProvider.createAbstractFileSystem(user);
 
-                    secondaryFs = secProvider.createAbstractFileSystem();
                     secondaryUri = secProvider.uri();
                 }
                 catch (IOException e) {
@@ -929,7 +933,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
             file.modificationTime(),
             file.accessTime(),
             permission(file),
-            file.property(PROP_USER_NAME, DFLT_USER_NAME),
+            file.property(PROP_USER_NAME, user),
             file.property(PROP_GROUP_NAME, "users"),
             convert(file.path())) {
             @Override public String toString() {
@@ -983,4 +987,12 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
     @Override public String toString() {
         return S.toString(IgniteHadoopFileSystem.class, this);
     }
-}
+
+    /**
+     * Returns the user name this File System is created on behalf of.
+     * @return the user name
+     */
+    public String user() {
+        return user;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/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 00be422..d493bd4 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
@@ -126,11 +126,15 @@ public class HadoopUtils {
                 break;
 
             case PHASE_REDUCE:
-                assert status.totalReducerCnt() > 0;
-
+                // TODO: temporary fixed, but why PHASE_REDUCE could have 0 reducers?
+                // See https://issues.apache.org/jira/browse/IGNITE-764
                 setupProgress = 1;
                 mapProgress = 1;
-                reduceProgress = 1f - status.pendingReducerCnt() / (float)status.totalReducerCnt();
+
+                if (status.totalReducerCnt() > 0)
+                    reduceProgress = 1f - status.pendingReducerCnt() / (float)status.totalReducerCnt();
+                else
+                    reduceProgress = 1f;
 
                 break;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/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 27805f8..b1a057c 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
@@ -19,12 +19,15 @@ package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
+import org.apache.hadoop.security.*;
+import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.net.*;
+import java.security.*;
 
 /**
  * Encapsulates logic of secondary filesystem creation.
@@ -36,9 +39,6 @@ public class SecondaryFileSystemProvider {
     /** The secondary filesystem URI, never null. */
     private final URI uri;
 
-    /** Optional user name to log into secondary filesystem with. */
-    private @Nullable final String userName;
-
     /**
      * Creates new provider with given config parameters. The configuration URL is optional. The filesystem URI must be
      * specified either explicitly or in the configuration provided.
@@ -47,13 +47,10 @@ public class SecondaryFileSystemProvider {
      * property in the provided configuration.
      * @param secConfPath the secondary Fs path (file path on the local file system, optional).
      * See {@link IgniteUtils#resolveIgniteUrl(String)} on how the path resolved.
-     * @param userName User name.
      * @throws IOException
      */
     public SecondaryFileSystemProvider(final @Nullable String secUri,
-        final @Nullable String secConfPath, @Nullable String userName) throws IOException {
-        this.userName = userName;
-
+        final @Nullable String secConfPath) throws IOException {
         if (secConfPath != null) {
             URL url = U.resolveIgniteUrl(secConfPath);
 
@@ -88,20 +85,18 @@ public class SecondaryFileSystemProvider {
      * @return {@link org.apache.hadoop.fs.FileSystem}  instance for this secondary Fs.
      * @throws IOException
      */
-    public FileSystem createFileSystem() throws IOException {
+    public FileSystem createFileSystem(String userName) throws IOException {
+        userName = IgfsUtils.fixUserName(userName);
+
         final FileSystem fileSys;
 
-        if (userName == null)
-            fileSys = FileSystem.get(uri, cfg);
-        else {
-            try {
-                fileSys = FileSystem.get(uri, cfg, userName);
-            }
-            catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
+        try {
+           fileSys = FileSystem.get(uri, cfg, userName);
+        }
+        catch (InterruptedException e) {
+           Thread.currentThread().interrupt();
 
-                throw new IOException("Failed to create file system due to interrupt.", e);
-            }
+           throw new IOException("Failed to create file system due to interrupt.", e);
         }
 
         return fileSys;
@@ -109,10 +104,26 @@ public class SecondaryFileSystemProvider {
 
     /**
      * @return {@link org.apache.hadoop.fs.AbstractFileSystem} instance for this secondary Fs.
-     * @throws IOException
+     * @throws IOException in case of error.
      */
-    public AbstractFileSystem createAbstractFileSystem() throws IOException {
-        return AbstractFileSystem.get(uri, cfg);
+    public AbstractFileSystem createAbstractFileSystem(String userName) throws IOException {
+        userName = IgfsUtils.fixUserName(userName);
+
+        String ticketCachePath = cfg.get(CommonConfigurationKeys.KERBEROS_TICKET_CACHE_PATH);
+
+        UserGroupInformation ugi = UserGroupInformation.getBestUGI(ticketCachePath, userName);
+
+        try {
+            return ugi.doAs(new PrivilegedExceptionAction<AbstractFileSystem>() {
+                @Override public AbstractFileSystem run() throws IOException {
+                    return AbstractFileSystem.get(uri, cfg);
+                }
+            });
+        } catch (InterruptedException ie) {
+            Thread.currentThread().interrupt();
+
+            throw new IOException("Failed to create file system due to interrupt.", ie);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java
deleted file mode 100644
index 509f443..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java
+++ /dev/null
@@ -1,91 +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.hadoop.fs;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.hdfs.*;
-import org.apache.hadoop.mapreduce.*;
-
-import java.io.*;
-import java.net.*;
-
-import static org.apache.ignite.configuration.FileSystemConfiguration.*;
-
-/**
- * Wrapper of HDFS for support of separated working directory.
- */
-public class HadoopDistributedFileSystem extends DistributedFileSystem {
-    /** User name for each thread. */
-    private final ThreadLocal<String> userName = new ThreadLocal<String>() {
-        /** {@inheritDoc} */
-        @Override protected String initialValue() {
-            return DFLT_USER_NAME;
-        }
-    };
-
-    /** Working directory for each thread. */
-    private final ThreadLocal<Path> workingDir = new ThreadLocal<Path>() {
-        /** {@inheritDoc} */
-        @Override protected Path initialValue() {
-            return getHomeDirectory();
-        }
-    };
-
-    /** {@inheritDoc} */
-    @Override public void initialize(URI uri, Configuration conf) throws IOException {
-        super.initialize(uri, conf);
-
-        setUser(conf.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
-    }
-
-    /**
-     * Set user name and default working directory for current thread.
-     *
-     * @param userName User name.
-     */
-    public void setUser(String userName) {
-        this.userName.set(userName);
-
-        setWorkingDirectory(getHomeDirectory());
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getHomeDirectory() {
-        Path path = new Path("/user/" + userName.get());
-
-        return path.makeQualified(getUri(), null);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setWorkingDirectory(Path dir) {
-        Path fixedDir = fixRelativePart(dir);
-
-        String res = fixedDir.toUri().getPath();
-
-        if (!DFSUtil.isValidName(res))
-            throw new IllegalArgumentException("Invalid DFS directory name " + res);
-
-        workingDir.set(fixedDir);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getWorkingDirectory() {
-        return workingDir.get();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
index f3f51d4..d90bc28 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
@@ -19,8 +19,6 @@ package org.apache.ignite.internal.processors.hadoop.fs;
 
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
-import org.apache.hadoop.hdfs.protocol.*;
-import org.apache.ignite.hadoop.fs.v1.*;
 
 /**
  * Utilities for configuring file systems to support the separate working directory per each thread.
@@ -30,19 +28,6 @@ public class HadoopFileSystemsUtils {
     public static final String LOC_FS_WORK_DIR_PROP = "fs." + FsConstants.LOCAL_FS_URI.getScheme() + ".workDir";
 
     /**
-     * Set user name and default working directory for current thread if it's supported by file system.
-     *
-     * @param fs File system.
-     * @param userName User name.
-     */
-    public static void setUser(FileSystem fs, String userName) {
-        if (fs instanceof IgniteHadoopFileSystem)
-            ((IgniteHadoopFileSystem)fs).setUser(userName);
-        else if (fs instanceof HadoopDistributedFileSystem)
-            ((HadoopDistributedFileSystem)fs).setUser(userName);
-    }
-
-    /**
      * Setup wrappers of filesystems to support the separate working directory.
      *
      * @param cfg Config for setup.
@@ -51,7 +36,5 @@ public class HadoopFileSystemsUtils {
         cfg.set("fs." + FsConstants.LOCAL_FS_URI.getScheme() + ".impl", HadoopLocalFileSystemV1.class.getName());
         cfg.set("fs.AbstractFileSystem." + FsConstants.LOCAL_FS_URI.getScheme() + ".impl",
                 HadoopLocalFileSystemV2.class.getName());
-
-        cfg.set("fs." + HdfsConstants.HDFS_URI_SCHEME + ".impl", HadoopDistributedFileSystem.class.getName());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java
index 2f19226..b9c5113 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java
@@ -85,4 +85,10 @@ public interface HadoopIgfsEx extends HadoopIgfs {
      * @throws IOException If failed.
      */
     public void flush(HadoopIgfsStreamDelegate delegate) throws IOException;
+
+    /**
+     * The user this Igfs instance works on behalf of.
+     * @return the user name.
+     */
+    public String user();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java
index 44e531e..47ba0e8 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java
@@ -23,6 +23,7 @@ import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -46,25 +47,35 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     /** Logger. */
     private final Log log;
 
+    /** The user this Igfs works on behalf of. */
+    private final String user;
+
     /**
      * Constructor.
      *
      * @param igfs Target IGFS.
      * @param log Log.
      */
-    public HadoopIgfsInProc(IgfsEx igfs, Log log) {
+    public HadoopIgfsInProc(IgfsEx igfs, Log log, String userName) throws IgniteCheckedException {
+        this.user = IgfsUtils.fixUserName(userName);
+
         this.igfs = igfs;
+
         this.log = log;
 
         bufSize = igfs.configuration().getBlockSize() * 2;
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsHandshakeResponse handshake(String logDir) {
-        igfs.clientLogDirectory(logDir);
+    @Override public IgfsHandshakeResponse handshake(final String logDir) {
+        return IgfsUserContext.doAs(user, new IgniteOutClosure<IgfsHandshakeResponse>() {
+            @Override public IgfsHandshakeResponse apply() {
+                igfs.clientLogDirectory(logDir);
 
-        return new IgfsHandshakeResponse(igfs.name(), igfs.proxyPaths(), igfs.groupBlockSize(),
-            igfs.globalSampling());
+                return new IgfsHandshakeResponse(igfs.name(), igfs.proxyPaths(), igfs.groupBlockSize(),
+                    igfs.globalSampling());
+                }
+         });
     }
 
     /** {@inheritDoc} */
@@ -82,9 +93,13 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsFile info(IgfsPath path) throws IgniteCheckedException {
+    @Override public IgfsFile info(final IgfsPath path) throws IgniteCheckedException {
         try {
-            return igfs.info(path);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<IgfsFile>() {
+                @Override public IgfsFile apply() {
+                    return igfs.info(path);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -95,9 +110,13 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+    @Override public IgfsFile update(final IgfsPath path, final Map<String, String> props) throws IgniteCheckedException {
         try {
-            return igfs.update(path, props);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<IgfsFile>() {
+                @Override public IgfsFile apply() {
+                    return igfs.update(path, props);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -108,9 +127,15 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteCheckedException {
+    @Override public Boolean setTimes(final IgfsPath path, final long accessTime, final long modificationTime) throws IgniteCheckedException {
         try {
-            igfs.setTimes(path, accessTime, modificationTime);
+            IgfsUserContext.doAs(user, new IgniteOutClosure<Void>() {
+                @Override public Void apply() {
+                    igfs.setTimes(path, accessTime, modificationTime);
+
+                    return null;
+                }
+            });
 
             return true;
         }
@@ -124,9 +149,15 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean rename(IgfsPath src, IgfsPath dest) throws IgniteCheckedException {
+    @Override public Boolean rename(final IgfsPath src, final IgfsPath dest) throws IgniteCheckedException {
         try {
-            igfs.rename(src, dest);
+            IgfsUserContext.doAs(user, new IgniteOutClosure<Void>() {
+                @Override public Void apply() {
+                    igfs.rename(src, dest);
+
+                    return null;
+                }
+            });
 
             return true;
         }
@@ -139,9 +170,13 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean delete(IgfsPath path, boolean recursive) throws IgniteCheckedException {
+    @Override public Boolean delete(final IgfsPath path, final boolean recursive) throws IgniteCheckedException {
         try {
-            return igfs.delete(path, recursive);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<Boolean>() {
+                @Override public Boolean apply() {
+                    return igfs.delete(path, recursive);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -154,18 +189,32 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     /** {@inheritDoc} */
     @Override public IgfsStatus fsStatus() throws IgniteCheckedException {
         try {
-            return igfs.globalSpace();
+            return IgfsUserContext.doAs(user, new Callable<IgfsStatus>() {
+                @Override public IgfsStatus call() throws IgniteCheckedException {
+                    return igfs.globalSpace();
+                }
+            });
         }
         catch (IllegalStateException e) {
             throw new HadoopIgfsCommunicationException("Failed to get file system status because Grid is " +
                 "stopping.");
         }
+        catch (IgniteCheckedException | RuntimeException | Error e) {
+            throw e;
+        }
+        catch (Exception e) {
+            throw new AssertionError("Must never go there.");
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteCheckedException {
+    @Override public Collection<IgfsPath> listPaths(final IgfsPath path) throws IgniteCheckedException {
         try {
-            return igfs.listPaths(path);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<Collection<IgfsPath>>() {
+                @Override public Collection<IgfsPath> apply() {
+                    return igfs.listPaths(path);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -176,9 +225,13 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteCheckedException {
+    @Override public Collection<IgfsFile> listFiles(final IgfsPath path) throws IgniteCheckedException {
         try {
-            return igfs.listFiles(path);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<Collection<IgfsFile>>() {
+                @Override public Collection<IgfsFile> apply() {
+                    return igfs.listFiles(path);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -189,9 +242,15 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean mkdirs(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+    @Override public Boolean mkdirs(final IgfsPath path, final Map<String, String> props) throws IgniteCheckedException {
         try {
-            igfs.mkdirs(path, props);
+            IgfsUserContext.doAs(user, new IgniteOutClosure<Void>() {
+                @Override public Void apply() {
+                    igfs.mkdirs(path, props);
+
+                    return null;
+                }
+            });
 
             return true;
         }
@@ -205,9 +264,13 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsPathSummary contentSummary(IgfsPath path) throws IgniteCheckedException {
+    @Override public IgfsPathSummary contentSummary(final IgfsPath path) throws IgniteCheckedException {
         try {
-            return igfs.summary(path);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<IgfsPathSummary>() {
+                @Override public IgfsPathSummary apply() {
+                    return igfs.summary(path);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -219,10 +282,14 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len)
+    @Override public Collection<IgfsBlockLocation> affinity(final IgfsPath path, final long start, final long len)
         throws IgniteCheckedException {
         try {
-            return igfs.affinity(path, start, len);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<Collection<IgfsBlockLocation>>() {
+                @Override public Collection<IgfsBlockLocation> apply() {
+                    return igfs.affinity(path, start, len);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -233,11 +300,15 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate open(IgfsPath path) throws IgniteCheckedException {
+    @Override public HadoopIgfsStreamDelegate open(final IgfsPath path) throws IgniteCheckedException {
         try {
-            IgfsInputStreamAdapter stream = igfs.open(path, bufSize);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<HadoopIgfsStreamDelegate>() {
+                @Override public HadoopIgfsStreamDelegate apply() {
+                    IgfsInputStreamAdapter stream = igfs.open(path, bufSize);
 
-            return new HadoopIgfsStreamDelegate(this, stream, stream.fileInfo().length());
+                    return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream, stream.fileInfo().length());
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -248,12 +319,16 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate open(IgfsPath path, int seqReadsBeforePrefetch)
+    @Override public HadoopIgfsStreamDelegate open(final IgfsPath path, final int seqReadsBeforePrefetch)
         throws IgniteCheckedException {
         try {
-            IgfsInputStreamAdapter stream = igfs.open(path, bufSize, seqReadsBeforePrefetch);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<HadoopIgfsStreamDelegate>() {
+                @Override public HadoopIgfsStreamDelegate apply() {
+                    IgfsInputStreamAdapter stream = igfs.open(path, bufSize, seqReadsBeforePrefetch);
 
-            return new HadoopIgfsStreamDelegate(this, stream, stream.fileInfo().length());
+                    return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream, stream.fileInfo().length());
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -264,13 +339,17 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate create(IgfsPath path, boolean overwrite, boolean colocate,
-        int replication, long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException {
+    @Override public HadoopIgfsStreamDelegate create(final IgfsPath path, final boolean overwrite, final boolean colocate,
+        final int replication, final long blockSize, final @Nullable Map<String, String> props) throws IgniteCheckedException {
         try {
-            IgfsOutputStream stream = igfs.create(path, bufSize, overwrite,
-                colocate ? igfs.nextAffinityKey() : null, replication, blockSize, props);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<HadoopIgfsStreamDelegate>() {
+                @Override public HadoopIgfsStreamDelegate apply() {
+                    IgfsOutputStream stream = igfs.create(path, bufSize, overwrite,
+                        colocate ? igfs.nextAffinityKey() : null, replication, blockSize, props);
 
-            return new HadoopIgfsStreamDelegate(this, stream);
+                    return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -281,12 +360,16 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate append(IgfsPath path, boolean create,
-        @Nullable Map<String, String> props) throws IgniteCheckedException {
+    @Override public HadoopIgfsStreamDelegate append(final IgfsPath path, final boolean create,
+        final @Nullable Map<String, String> props) throws IgniteCheckedException {
         try {
-            IgfsOutputStream stream = igfs.append(path, bufSize, create, props);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<HadoopIgfsStreamDelegate>() {
+                @Override public HadoopIgfsStreamDelegate apply() {
+                    IgfsOutputStream stream = igfs.append(path, bufSize, create, props);
 
-            return new HadoopIgfsStreamDelegate(this, stream);
+                    return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -407,4 +490,9 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
         if (lsnr0 != null && log.isDebugEnabled())
             log.debug("Removed stream event listener [delegate=" + delegate + ']');
     }
+
+    /** {@inheritDoc} */
+    @Override public String user() {
+        return user;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35388195/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java
index 0264e7b..3561e95 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java
@@ -41,7 +41,7 @@ import java.util.concurrent.locks.*;
 @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
 public class HadoopIgfsIpcIo implements HadoopIgfsIo {
     /** Logger. */
-    private Log log;
+    private final Log log;
 
     /** Request futures map. */
     private ConcurrentMap<Long, HadoopIgfsFuture> reqMap =


[15/50] [abbrv] incubator-ignite git commit: # IGNITE-709 Fix IgfsOneClientNodeTest

Posted by ag...@apache.org.
# IGNITE-709 Fix IgfsOneClientNodeTest


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

Branch: refs/heads/ignite-389
Commit: 2bc07956a828ac836641f4fbe6ae0db7b3cc1793
Parents: 2799c3a
Author: sevdokimov <se...@jetbrains.com>
Authored: Thu May 28 21:47:48 2015 +0300
Committer: sevdokimov <se...@jetbrains.com>
Committed: Thu May 28 21:47:48 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 112 ++++++++-----------
 .../processors/igfs/IgfsOneClientNodeTest.java  |   8 +-
 2 files changed, 47 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2bc07956/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 59e25fc..b5c9519 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
@@ -810,84 +810,61 @@ class ServerImpl extends TcpDiscoveryImpl {
             boolean retry = false;
             Collection<Exception> errs = new ArrayList<>();
 
-            for (int j = 2; --j >= 0;) {
-                for (InetSocketAddress addr : addrs) {
-                    Socket sock = null;
-                    Exception ex = null;
+            for (InetSocketAddress addr : addrs) {
+                try {
+                    Integer res = sendMessageDirectly(joinReq, addr);
 
-                    try {
-                        sock = spi.openSocket(addr);
-                    }
-                    catch (Exception e) {
-                        if (j > 0)
-                            continue;
+                    assert res != null;
 
-                        ex = e;
-                    }
+                    noResAddrs.remove(addr);
 
-                    if (ex == null) {
-                        try {
-                            Integer res = sendMessageDirectly(joinReq, addr, sock);
+                    // Address is responsive, reset period start.
+                    noResStart = 0;
 
-                            assert res != null;
+                    switch (res) {
+                        case RES_WAIT:
+                            // Concurrent startup, try sending join request again or wait if no success.
+                            retry = true;
 
-                            noResAddrs.remove(addr);
+                            break;
+                        case RES_OK:
+                            if (log.isDebugEnabled())
+                                log.debug("Join request message has been sent to address [addr=" + addr +
+                                    ", req=" + joinReq + ']');
 
-                            // Address is responsive, reset period start.
-                            noResStart = 0;
+                            // Join request sending succeeded, wait for response from topology.
+                            return true;
 
-                            switch (res) {
-                                case RES_WAIT:
-                                    // Concurrent startup, try sending join request again or wait if no success.
+                        default:
+                            // Concurrent startup, try next node.
+                            if (res == RES_CONTINUE_JOIN) {
+                                if (!fromAddrs.contains(addr))
                                     retry = true;
+                            }
+                            else {
+                                if (log.isDebugEnabled())
+                                    log.debug("Unexpected response to join request: " + res);
 
-                                    break;
-                                case RES_OK:
-                                    if (log.isDebugEnabled())
-                                        log.debug("Join request message has been sent to address [addr=" + addr +
-                                            ", req=" + joinReq + ']');
-
-                                    // Join request sending succeeded, wait for response from topology.
-                                    return true;
-
-                                default:
-                                    // Concurrent startup, try next node.
-                                    if (res == RES_CONTINUE_JOIN) {
-                                        if (!fromAddrs.contains(addr))
-                                            retry = true;
-                                    }
-                                    else {
-                                        if (log.isDebugEnabled())
-                                            log.debug("Unexpected response to join request: " + res);
-
-                                        retry = true;
-                                    }
-
-                                    break;
+                                retry = true;
                             }
-                        }
-                        catch (IgniteSpiException e) {
-                            e.printStackTrace();
 
-                            ex = e;
-                        }
+                            break;
                     }
+                }
+                catch (IgniteSpiException e) {
+                    errs.add(e);
 
-                    if (ex != null) {
-                        errs.add(ex);
-
-                        if (log.isDebugEnabled()) {
-                            IOException ioe = X.cause(ex, IOException.class);
-
-                            log.debug("Failed to send join request message [addr=" + addr +
-                                ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']');
+                    if (log.isDebugEnabled()) {
+                        IOException ioe = X.cause(e, IOException.class);
 
-                            onException("Failed to send join request message [addr=" + addr +
-                                ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']', ioe);
-                        }
+                        log.debug("Failed to send join request message [addr=" + addr +
+                            ", msg=" + (ioe != null ? ioe.getMessage() : e.getMessage()) + ']');
 
-                        noResAddrs.add(addr);
+                        onException("Failed to send join request message [addr=" + addr +
+                            ", msg=" + (ioe != null ? ioe.getMessage() : e.getMessage()) + ']', ioe);
                     }
+
+                    noResAddrs.add(addr);
                 }
             }
 
@@ -950,7 +927,7 @@ class ServerImpl extends TcpDiscoveryImpl {
      * @return Response read from the recipient or {@code null} if no response is supposed.
      * @throws IgniteSpiException If an error occurs.
      */
-    @Nullable private Integer sendMessageDirectly(TcpDiscoveryAbstractMessage msg, InetSocketAddress addr, Socket sock)
+    @Nullable private Integer sendMessageDirectly(TcpDiscoveryAbstractMessage msg, InetSocketAddress addr)
         throws IgniteSpiException {
         assert msg != null;
         assert addr != null;
@@ -972,11 +949,12 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             boolean openSock = false;
 
+            Socket sock = null;
+
             try {
                 long tstamp = U.currentTimeMillis();
 
-                if (sock == null)
-                    sock = spi.openSocket(addr);
+                sock = spi.openSocket(addr);
 
                 openSock = true;
 
@@ -1060,8 +1038,6 @@ class ServerImpl extends TcpDiscoveryImpl {
             }
             finally {
                 U.closeQuiet(sock);
-
-                sock = null;
             }
         }
 
@@ -2718,7 +2694,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             for (InetSocketAddress addr : spi.getNodeAddresses(node, U.sameMacs(locNode, node))) {
                 try {
-                    sendMessageDirectly(msg, addr, null);
+                    sendMessageDirectly(msg, addr);
 
                     ex = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2bc07956/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
index 49ddb03..3498cd9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
@@ -52,11 +52,9 @@ public class IgfsOneClientNodeTest extends GridCommonAbstractTest {
 
         cfg.setClientMode(true);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi()
+            .setForceServerMode(true)
+            .setIpFinder(new TcpDiscoveryVmIpFinder(true)));
 
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 


[35/50] [abbrv] incubator-ignite git commit: # ignite-943 fixed client futures remap

Posted by ag...@apache.org.
# ignite-943 fixed client futures remap


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

Branch: refs/heads/ignite-389
Commit: d10fe3e9013bc9ecd27823ff00443983b7c066c2
Parents: b3d8e1e
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 29 17:29:28 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 29 17:29:28 2015 +0300

----------------------------------------------------------------------
 .../dht/colocated/GridDhtColocatedLockFuture.java | 18 ++++++++++++------
 .../near/GridNearOptimisticTxPrepareFuture.java   | 18 ++++++++++++------
 2 files changed, 24 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d10fe3e9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index 5a4c91d..c784948 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@ -549,7 +549,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
         }
 
         // Must get topology snapshot and map on that version.
-        mapOnTopology(false);
+        mapOnTopology(false, null);
     }
 
     /**
@@ -557,8 +557,9 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
      * will asynchronously wait for it's completeness and then try again.
      *
      * @param remap Remap flag.
+     * @param c Optional closure to run after map.
      */
-    private void mapOnTopology(final boolean remap) {
+    private void mapOnTopology(final boolean remap, @Nullable final Runnable c) {
         // We must acquire topology snapshot from the topology version future.
         cctx.topology().readLock();
 
@@ -597,12 +598,15 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
 
                 map(keys, remap);
 
+                if (c != null)
+                    c.run();
+
                 markInitialized();
             }
             else {
                 fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                        mapOnTopology(remap);
+                        mapOnTopology(remap, c);
                     }
                 });
             }
@@ -1366,9 +1370,11 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
             for (KeyCacheObject key : GridDhtColocatedLockFuture.this.keys)
                 cctx.mvcc().removeExplicitLock(threadId, key, lockVer);
 
-            mapOnTopology(true);
-
-            onDone(true);
+            mapOnTopology(true, new Runnable() {
+                @Override public void run() {
+                    onDone(true);
+                }
+            });
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d10fe3e9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 0a92112..44b7997 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -226,13 +226,14 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
             return;
         }
 
-        prepareOnTopology(false);
+        prepareOnTopology(false, null);
     }
 
     /**
      * @param remap Remap flag.
+     * @param c Optional closure to run after map.
      */
-    private void prepareOnTopology(final boolean remap) {
+    private void prepareOnTopology(final boolean remap, @Nullable final Runnable c) {
         GridDhtTopologyFuture topFut = topologyReadLock();
 
         try {
@@ -271,13 +272,16 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
                     tx.topologyVersion(topFut.topologyVersion());
 
                 prepare0(remap);
+
+                if (c != null)
+                    c.run();
             }
             else {
                 topFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
                         cctx.kernalContext().closure().runLocalSafe(new GridPlainRunnable() {
                             @Override public void run() {
-                                prepareOnTopology(remap);
+                                prepareOnTopology(remap, c);
                             }
                         });
                     }
@@ -796,9 +800,11 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
          *
          */
         private void remap() {
-            prepareOnTopology(true);
-
-            onDone(tx);
+            prepareOnTopology(true, new Runnable() {
+                @Override public void run() {
+                    onDone(tx);
+                }
+            });
         }
 
         /** {@inheritDoc} */


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

Posted by ag...@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/43f7aafc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/43f7aafc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/43f7aafc

Branch: refs/heads/ignite-389
Commit: 43f7aafc0cf7b8db7510e6e3966a713c79d3461d
Parents: 3d77af8 98e392c
Author: ashutak <as...@gridgain.com>
Authored: Thu May 28 19:18:29 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu May 28 19:18:29 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   5 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   2 +-
 .../org/apache/ignite/services/Service.java     |   5 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  20 +-
 .../service/ClosureServiceClientsNodesTest.java | 245 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   4 +
 .../query/h2/sql/BaseH2CompareQueryTest.java    |  16 ++
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 42 files changed, 320 insertions(+), 55 deletions(-)
----------------------------------------------------------------------