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

[01/19] incubator-ignite git commit: IGNITE-80 - Porting changes to a separate branch.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-471-2 1f4ddddf6 -> 19d3aac7c


IGNITE-80 - Porting changes to a separate branch.


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

Branch: refs/heads/ignite-471-2
Commit: dcda61b4fe2be3005544a3fc915b19ac3e4c9598
Parents: 1e53395
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Apr 29 14:08:05 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Apr 29 14:08:05 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    |  5 +--
 .../GridCachePartitionExchangeManager.java      |  4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  6 ++-
 .../dht/atomic/GridDhtAtomicCache.java          |  4 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 42 +++++++++++++++-----
 .../dht/atomic/GridNearAtomicUpdateRequest.java | 36 ++++++++++++++---
 .../colocated/GridDhtColocatedLockFuture.java   |  4 +-
 .../cache/transactions/IgniteTxManager.java     | 24 +++++++++++
 8 files changed, 101 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dcda61b4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index b8668e6..112330a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -146,9 +146,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             c = clsHandlers.get(new ListenerKey(cacheMsg.cacheId(), cacheMsg.getClass()));
 
         if (c == null) {
-            if (log.isDebugEnabled())
-                log.debug("Received message without registered handler (will ignore) [msg=" + cacheMsg +
-                    ", nodeId=" + nodeId + ']');
+            U.warn(log, "Received message without registered handler (will ignore) [msg=" + cacheMsg +
+                ", nodeId=" + nodeId + ']');
 
             return;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dcda61b4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 5f82ae2..e61168e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -409,10 +409,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * @param ver Topology version.
      * @return Future or {@code null} is future is already completed.
      */
-    public @Nullable IgniteInternalFuture<?> affinityReadyFuture(AffinityTopologyVersion ver) {
+    @Nullable public IgniteInternalFuture<?> affinityReadyFuture(AffinityTopologyVersion ver) {
         GridDhtPartitionsExchangeFuture lastInitializedFut0 = lastInitializedFut;
 
-        if (lastInitializedFut0 != null && lastInitializedFut0.topologyVersion().compareTo(ver) >= 0) {
+        if (lastInitializedFut0 != null && lastInitializedFut0.topologyVersion().compareTo(ver) == 0) {
             if (log.isDebugEnabled())
                 log.debug("Return lastInitializedFut for topology ready future " +
                     "[ver=" + ver + ", fut=" + lastInitializedFut0 + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dcda61b4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 1c46fd0..4d1db85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -645,8 +645,10 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
                     res.error(e);
                 }
 
-                res.invalidPartitions(fut.invalidPartitions(),
-                    new AffinityTopologyVersion(ctx.discovery().topologyVersion()));
+                if (!F.isEmpty(fut.invalidPartitions()))
+                    res.invalidPartitions(fut.invalidPartitions(), ctx.shared().exchange().readyAffinityVersion());
+                else
+                    res.invalidPartitions(fut.invalidPartitions(), req.topologyVersion());
 
                 try {
                     ctx.io().send(nodeId, res, ctx.ioPolicy());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dcda61b4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 905f7bf..a30f211 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -1041,7 +1041,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
                     // Do not check topology version for CLOCK versioning since
                     // partition exchange will wait for near update future.
-                    if (topology().topologyVersion().equals(req.topologyVersion()) ||
+                    // Also do not check topology version if topology was locked on near node by
+                    // external transaction or explicit lock.
+                    if (topology().topologyVersion().equals(req.topologyVersion()) || req.topologyLocked() ||
                         ctx.config().getAtomicWriteOrderMode() == CLOCK) {
                         ClusterNode node = ctx.discovery().node(nodeId);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dcda61b4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 072ab52..3dc89f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.processors.cache.dr.*;
+import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.tostring.*;
@@ -136,6 +137,9 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
     /** Task name hash. */
     private final int taskNameHash;
 
+    /** Topology locked flag. Set if atomic update is performed inside a TX or explicit lock. */
+    private boolean topLocked;
+
     /** Skip store flag. */
     private final boolean skipStore;
 
@@ -289,7 +293,23 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
      * @param waitTopFut Whether to wait for topology future.
      */
     public void map(boolean waitTopFut) {
-        mapOnTopology(keys, false, null, waitTopFut);
+        AffinityTopologyVersion topVer = null;
+
+        IgniteInternalTx tx = cctx.tm().anyActiveThreadTx();
+
+        if (tx != null && tx.topologyVersionSnapshot() != null)
+            topVer = tx.topologyVersionSnapshot();
+
+        if (topVer == null)
+            topVer = cctx.mvcc().lastExplicitLockTopologyVersion(Thread.currentThread().getId());
+
+        if (topVer == null)
+            mapOnTopology(keys, false, null, waitTopFut);
+        else {
+            topLocked = true;
+
+            map0(topVer, keys, false, null);
+        }
     }
 
     /** {@inheritDoc} */
@@ -430,15 +450,12 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                 }
 
                 topVer = fut.topologyVersion();
-
-                if (futVer == null)
-                    // Assign future version in topology read lock before first exception may be thrown.
-                    futVer = cctx.versions().next(topVer);
             }
             else {
                 if (waitTopFut) {
                     fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
+                        @Override
+                        public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
                             mapOnTopology(keys, remap, oldNodeId, waitTopFut);
                         }
                     });
@@ -448,9 +465,6 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
 
                 return;
             }
-
-            if (!remap && (cctx.config().getAtomicWriteOrderMode() == CLOCK || syncMode != FULL_ASYNC))
-                cctx.mvcc().addAtomicFuture(version(), this);
         }
         finally {
             cache.topology().readUnlock();
@@ -474,6 +488,7 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
     }
 
     /**
+     * @param topVer Topology version.
      * @param keys Keys to map.
      * @param remap Flag indicating if this is partial remap for this future.
      * @param oldNodeId Old node ID if was remap.
@@ -494,6 +509,13 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
             return;
         }
 
+        if (futVer == null)
+            // Assign future version in topology read lock before first exception may be thrown.
+            futVer = cctx.versions().next(topVer);
+
+        if (!remap && (cctx.config().getAtomicWriteOrderMode() == CLOCK || syncMode != FULL_ASYNC))
+            cctx.mvcc().addAtomicFuture(version(), this);
+
         CacheConfiguration ccfg = cctx.config();
 
         // Assign version on near node in CLOCK ordering mode even if fastMap is false.
@@ -579,6 +601,7 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                 fastMap,
                 updVer,
                 topVer,
+                topLocked,
                 syncMode,
                 op,
                 retval,
@@ -716,6 +739,7 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                             fastMap,
                             updVer,
                             topVer,
+                            topLocked,
                             syncMode,
                             op,
                             retval,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dcda61b4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
index e0e3e26..a96a666 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
@@ -64,6 +64,9 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
     /** Topology version. */
     private AffinityTopologyVersion topVer;
 
+    /** Topology locked flag. Set if atomic update is performed inside TX or explicit lock. */
+    private boolean topLocked;
+
     /** Write synchronization mode. */
     private CacheWriteSynchronizationMode syncMode;
 
@@ -162,6 +165,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
         boolean fastMap,
         @Nullable GridCacheVersion updateVer,
         @NotNull AffinityTopologyVersion topVer,
+        boolean topLocked,
         CacheWriteSynchronizationMode syncMode,
         GridCacheOperation op,
         boolean retval,
@@ -179,6 +183,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
         this.updateVer = updateVer;
 
         this.topVer = topVer;
+        this.topLocked = topLocked;
         this.syncMode = syncMode;
         this.op = op;
         this.retval = retval;
@@ -254,6 +259,13 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
     }
 
     /**
+     * @return Topology locked flag.
+     */
+    public boolean topologyLocked() {
+        return topLocked;
+    }
+
+    /**
      * @return Cache write synchronization mode.
      */
     public CacheWriteSynchronizationMode writeSynchronizationMode() {
@@ -664,18 +676,24 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 writer.incrementState();
 
             case 20:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeBoolean("topLocked", topLocked))
                     return false;
 
                 writer.incrementState();
 
             case 21:
-                if (!writer.writeMessage("updateVer", updateVer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
             case 22:
+                if (!writer.writeMessage("updateVer", updateVer))
+                    return false;
+
+                writer.incrementState();
+
+            case 23:
                 if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG))
                     return false;
 
@@ -842,7 +860,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 20:
-                topVer = reader.readMessage("topVer");
+                topLocked = reader.readBoolean("topLocked");
 
                 if (!reader.isLastRead())
                     return false;
@@ -850,7 +868,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 21:
-                updateVer = reader.readMessage("updateVer");
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -858,6 +876,14 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 22:
+                updateVer = reader.readMessage("updateVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 23:
                 vals = reader.readCollection("vals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -877,7 +903,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 23;
+        return 24;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dcda61b4/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 5b74b31..6292f2d 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
@@ -292,7 +292,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
                     false,
                     false);
 
-                cand.topologyVersion(new AffinityTopologyVersion(topVer.get().topologyVersion()));
+                cand.topologyVersion(topVer.get());
             }
         }
         else {
@@ -311,7 +311,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
                     false,
                     false);
 
-                cand.topologyVersion(new AffinityTopologyVersion(topVer.get().topologyVersion()));
+                cand.topologyVersion(topVer.get());
             }
             else
                 cand = cand.reenter();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dcda61b4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index c494602..874e640 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -639,6 +639,30 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @return Any transaction associated with the current thread.
+     */
+    public IgniteInternalTx anyActiveThreadTx() {
+        long threadId = Thread.currentThread().getId();
+
+        IgniteInternalTx tx = threadMap.get(threadId);
+
+        if (tx != null && tx.topologyVersionSnapshot() != null)
+            return tx;
+
+        for (GridCacheContext cacheCtx : cctx.cache().context().cacheContexts()) {
+            if (!cacheCtx.systemTx())
+                continue;
+
+            tx = sysThreadMap.get(new TxThreadKey(threadId, cacheCtx.cacheId()));
+
+            if (tx != null && tx.topologyVersionSnapshot() != null)
+                return tx;
+        }
+
+        return null;
+    }
+
+    /**
      * @return Local transaction.
      */
     @Nullable public IgniteInternalTx localTxx() {


[15/19] incubator-ignite git commit: Merge branch ignite-920 into ignite-sprint-5

Posted by sb...@apache.org.
Merge branch ignite-920 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/02d0acd1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/02d0acd1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/02d0acd1

Branch: refs/heads/ignite-471-2
Commit: 02d0acd1d412914efdaf13c3ac0d798df74083b2
Parents: 26a713c f1b5ecd
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue May 19 11:16:12 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue May 19 11:16:12 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |  26 +++-
 .../processors/cache/GridCacheIoManager.java    |   5 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../distributed/GridDistributedTxMapping.java   |   5 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   6 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   1 +
 .../dht/atomic/GridDhtAtomicCache.java          |   4 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  42 ++++--
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  36 ++++-
 .../colocated/GridDhtColocatedLockFuture.java   |   4 +-
 .../distributed/near/GridNearCacheEntry.java    |   2 +-
 .../cache/distributed/near/GridNearTxLocal.java |   5 +-
 .../near/GridNearTxPrepareResponse.java         |  28 +++-
 .../transactions/IgniteTxLocalAdapter.java      |   4 +-
 .../cache/transactions/IgniteTxManager.java     |  24 ++++
 .../cache/IgniteCacheNearLockValueSelfTest.java | 144 +++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 17 files changed, 306 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/02d0acd1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/02d0acd1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/02d0acd1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/02d0acd1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/02d0acd1/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 9fd4e88,159a8d8..3fa3d9d
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@@ -153,6 -159,295 +153,8 @@@ public class IgniteCacheTestSuite exten
  //        suite.addTestSuite(GridCacheNearTxExceptionSelfTest.class);
  //        suite.addTestSuite(GridCacheStopSelfTest.class); TODO IGNITE-257
  
 -        // Local cache.
 -        suite.addTestSuite(GridCacheLocalBasicApiSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalBasicStoreSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalAtomicBasicStoreSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalGetAndTransformStoreSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalAtomicGetAndTransformStoreSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalLoadAllSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalLockSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalMultithreadedSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalTxSingleThreadedSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalTxTimeoutSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalEventSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalEvictionEventSelfTest.class);
 -        suite.addTestSuite(GridCacheVariableTopologySelfTest.class);
 -        suite.addTestSuite(GridCacheLocalTxMultiThreadedSelfTest.class);
 -        suite.addTestSuite(GridCacheTransformEventSelfTest.class);
 -        suite.addTestSuite(GridCacheLocalIsolatedNodesSelfTest.class);
 -
 -        // Partitioned cache.
 -        suite.addTestSuite(GridCachePartitionedGetSelfTest.class);
 -        suite.addTest(new TestSuite(GridCachePartitionedBasicApiTest.class));
 -        suite.addTest(new TestSuite(GridCacheNearMultiGetSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheNearJobExecutionSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheNearOneNodeSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheNearMultiNodeSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheAtomicNearMultiNodeSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheNearReadersSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheAtomicNearReadersSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedAffinitySelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheRendezvousAffinityFunctionExcludeNeighborsSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheRendezvousAffinityClientSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedProjectionAffinitySelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedBasicOpSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedBasicStoreSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedGetAndTransformStoreSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedAtomicGetAndTransformStoreSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedBasicStoreMultiNodeSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedNearDisabledBasicStoreMultiNodeSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedEventSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedLockSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedMultiNodeLockSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedMultiNodeSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedMultiThreadedPutGetSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedNodeFailureSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedExplicitLockNodeFailureSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedTxSingleThreadedSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheColocatedTxSingleThreadedSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedTxTimeoutSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheFinishPartitionsSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtEntrySelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtInternalEntrySelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtMappingSelfTest.class));
 -//        suite.addTest(new TestSuite(GridCachePartitionedTxMultiThreadedSelfTest.class)); TODO-gg-4066
 -        suite.addTest(new TestSuite(GridCacheDhtPreloadSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtPreloadOffHeapSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtPreloadBigDataSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtPreloadPutGetSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtPreloadDisabledSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtPreloadMultiThreadedSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheColocatedPreloadRestartSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheNearPreloadRestartSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtPreloadStartStopSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtPreloadUnloadSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedAffinityFilterSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedPreloadLifecycleSelfTest.class));
 -        suite.addTest(new TestSuite(CacheLoadingConcurrentGridStartSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtPreloadDelayedSelfTest.class));
 -        suite.addTest(new TestSuite(GridPartitionedBackupLoadSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedLoadCacheSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionNotLoadedEventSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtEvictionsDisabledSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheNearEvictionEventSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheAtomicNearEvictionEventSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtEvictionSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheReplicatedEvictionSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtEvictionNearReadersSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheDhtAtomicEvictionNearReadersSelfTest.class));
 -//        suite.addTest(new TestSuite(GridCachePartitionedTopologyChangeSelfTest.class)); TODO-gg-5489
 -        suite.addTest(new TestSuite(GridCachePartitionedPreloadEventsSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedUnloadEventsSelfTest.class));
 -        suite.addTest(new TestSuite(GridCachePartitionedAffinityHashIdResolverSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheColocatedOptimisticTransactionSelfTest.class));
 -        suite.addTestSuite(GridCacheAtomicMessageCountSelfTest.class);
 -        suite.addTest(new TestSuite(GridCacheNearPartitionedClearSelfTest.class));
 -
 -        suite.addTest(new TestSuite(GridCacheDhtExpiredEntriesPreloadSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheNearExpiredEntriesPreloadSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheAtomicExpiredEntriesPreloadSelfTest.class));
 -
 -        suite.addTest(new TestSuite(GridCacheOffheapUpdateSelfTest.class));
 -
 -        // TODO: GG-7242, GG-7243: Enabled when fixed.
 -//        suite.addTest(new TestSuite(GridCacheDhtRemoveFailureTest.class));
 -//        suite.addTest(new TestSuite(GridCacheNearRemoveFailureTest.class));
 -        // TODO: GG-7201: Enable when fixed.
 -        //suite.addTest(new TestSuite(GridCacheDhtAtomicRemoveFailureTest.class));
 -
 -        suite.addTest(new TestSuite(GridCacheNearPrimarySyncSelfTest.class));
 -        suite.addTest(new TestSuite(GridCacheColocatedPrimarySyncSelfTest.class));
 -
 -        // Value consistency tests.
 -        suite.addTestSuite(GridCacheValueConsistencyAtomicSelfTest.class);
 -        suite.addTestSuite(GridCacheValueConsistencyAtomicPrimaryWriteOrderSelfTest.class);
 -        suite.addTestSuite(GridCacheValueConsistencyAtomicNearEnabledSelfTest.class);
 -        suite.addTestSuite(GridCacheValueConsistencyAtomicPrimaryWriteOrderNearEnabledSelfTest.class);
 -        suite.addTestSuite(GridCacheValueConsistencyTransactionalSelfTest.class);
 -        suite.addTestSuite(GridCacheValueConsistencyTransactionalNearEnabledSelfTest.class);
 -        suite.addTestSuite(GridCacheValueBytesPreloadingSelfTest.class);
 -
 -        // Replicated cache.
 -        suite.addTestSuite(GridCacheReplicatedBasicApiTest.class);
 -        suite.addTestSuite(GridCacheReplicatedBasicOpSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedBasicStoreSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedGetAndTransformStoreSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedAtomicGetAndTransformStoreSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedEventSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedSynchronousCommitTest.class);
 -
 -        // TODO: GG-7437.
 -        // suite.addTestSuite(GridCacheReplicatedInvalidateSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedLockSelfTest.class);
 -        // TODO: enable when GG-7437 is fixed.
 -        //suite.addTestSuite(GridCacheReplicatedMultiNodeLockSelfTest.class);
 -        //suite.addTestSuite(GridCacheReplicatedMultiNodeSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedNodeFailureSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedTxSingleThreadedSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedTxTimeoutSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedPreloadSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedPreloadOffHeapSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedPreloadLifecycleSelfTest.class);
 -        suite.addTestSuite(GridCacheSyncReplicatedPreloadSelfTest.class);
 -
 -        suite.addTestSuite(GridCacheDeploymentSelfTest.class);
 -        suite.addTestSuite(GridCacheDeploymentOffHeapSelfTest.class);
 -
 -        suite.addTestSuite(GridCachePutArrayValueSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedUnswapAdvancedSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedEvictionEventSelfTest.class);
 -        // TODO: GG-7569.
 -        // suite.addTestSuite(GridCacheReplicatedTxMultiThreadedSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedPreloadEventsSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedPreloadStartStopEventsSelfTest.class);
 -        // TODO: GG-7434
 -        // suite.addTestSuite(GridReplicatedTxPreloadTest.class);
 -
 -        suite.addTestSuite(IgniteTxReentryNearSelfTest.class);
 -        suite.addTestSuite(IgniteTxReentryColocatedSelfTest.class);
 -
 -        suite.addTestSuite(GridCacheOrderedPreloadingSelfTest.class);
 -
 -        // Test for byte array value special case.
 -//        suite.addTestSuite(GridCacheLocalByteArrayValuesSelfTest.class);
 -        suite.addTestSuite(GridCacheNearPartitionedP2PEnabledByteArrayValuesSelfTest.class);
 -        suite.addTestSuite(GridCacheNearPartitionedP2PDisabledByteArrayValuesSelfTest.class);
 -        suite.addTestSuite(GridCachePartitionedOnlyP2PEnabledByteArrayValuesSelfTest.class);
 -        suite.addTestSuite(GridCachePartitionedOnlyP2PDisabledByteArrayValuesSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedP2PEnabledByteArrayValuesSelfTest.class);
 -        suite.addTestSuite(GridCacheReplicatedP2PDisabledByteArrayValuesSelfTest.class);
 -
 -        // Near-only cache.
 -        suite.addTest(IgniteCacheNearOnlySelfTestSuite.suite());
 -
 -        // Test cache with daemon nodes.
 -        suite.addTestSuite(GridCacheDaemonNodeLocalSelfTest.class);
 -        suite.addTestSuite(GridCacheDaemonNodePartitionedSelfTest.class);
 -        suite.addTestSuite(GridCacheDaemonNodeReplicatedSelfTest.class);
 -
 -        // Write-behind.
 -        suite.addTest(IgniteCacheWriteBehindTestSuite.suite());
 -
 -        // Transform.
 -        suite.addTestSuite(GridCachePartitionedTransformWriteThroughBatchUpdateSelfTest.class);
 -
 -        suite.addTestSuite(GridCacheEntryVersionSelfTest.class);
 -        suite.addTestSuite(GridCacheVersionSelfTest.class);
 -
 -        // Memory leak tests.
 -        suite.addTestSuite(GridCacheReferenceCleanupSelfTest.class);
 -        suite.addTestSuite(GridCacheReloadSelfTest.class);
 -
 -        suite.addTestSuite(GridCacheMixedModeSelfTest.class);
 -
 -        // Cache metrics.
 -        suite.addTest(IgniteCacheMetricsSelfTestSuite.suite());
 -
 -        // Topology validator.
 -        suite.addTest(IgniteTopologyValidatorTestSuit.suite());
 -
 -        // Eviction.
 -        suite.addTest(IgniteCacheEvictionSelfTestSuite.suite());
 -
 -        // Iterators.
 -        suite.addTest(IgniteCacheIteratorsSelfTestSuite.suite());
 -
 -        // Cache interceptor tests.
 -        suite.addTest(IgniteCacheInterceptorSelfTestSuite.suite());
 -
 -        // Multi node update.
 -        suite.addTestSuite(GridCacheMultinodeUpdateSelfTest.class);
 -        // TODO: GG-5353.
 -        // suite.addTestSuite(GridCacheMultinodeUpdateNearEnabledSelfTest.class);
 -        // suite.addTestSuite(GridCacheMultinodeUpdateNearEnabledNoBackupsSelfTest.class);
 -        suite.addTestSuite(GridCacheMultinodeUpdateAtomicSelfTest.class);
 -        suite.addTestSuite(GridCacheMultinodeUpdateAtomicNearEnabledSelfTest.class);
 -
 -        suite.addTestSuite(IgniteCacheAtomicLoadAllTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicLocalLoadAllTest.class);
 -        suite.addTestSuite(IgniteCacheTxLoadAllTest.class);
 -        suite.addTestSuite(IgniteCacheTxLocalLoadAllTest.class);
 -
 -        suite.addTestSuite(IgniteCacheAtomicLoaderWriterTest.class);
 -        suite.addTestSuite(IgniteCacheTxLoaderWriterTest.class);
 -
 -        suite.addTestSuite(IgniteCacheAtomicStoreSessionTest.class);
 -        suite.addTestSuite(IgniteCacheTxStoreSessionTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicStoreSessionWriteBehindTest.class);
 -        suite.addTestSuite(IgniteCacheTxStoreSessionWriteBehindTest.class);
 -
 -        suite.addTestSuite(IgniteCacheAtomicNoReadThroughTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicNearEnabledNoReadThroughTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicLocalNoReadThroughTest.class);
 -        suite.addTestSuite(IgniteCacheTxNoReadThroughTest.class);
 -        suite.addTestSuite(IgniteCacheTxNearEnabledNoReadThroughTest.class);
 -        suite.addTestSuite(IgniteCacheTxLocalNoReadThroughTest.class);
 -
 -        suite.addTestSuite(IgniteCacheAtomicNoLoadPreviousValueTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicNearEnabledNoLoadPreviousValueTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicLocalNoLoadPreviousValueTest.class);
 -        suite.addTestSuite(IgniteCacheTxNoLoadPreviousValueTest.class);
 -        suite.addTestSuite(IgniteCacheTxNearEnabledNoLoadPreviousValueTest.class);
 -        suite.addTestSuite(IgniteCacheTxLocalNoLoadPreviousValueTest.class);
 -
 -        suite.addTestSuite(IgniteCacheAtomicNoWriteThroughTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicNearEnabledNoWriteThroughTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicLocalNoWriteThroughTest.class);
 -        suite.addTestSuite(IgniteCacheTxNoWriteThroughTest.class);
 -        suite.addTestSuite(IgniteCacheTxNearEnabledNoWriteThroughTest.class);
 -        suite.addTestSuite(IgniteCacheTxLocalNoWriteThroughTest.class);
 -
 -        suite.addTestSuite(IgniteCacheAtomicPeekModesTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicNearPeekModesTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicReplicatedPeekModesTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicLocalPeekModesTest.class);
 -        suite.addTestSuite(IgniteCacheTxPeekModesTest.class);
 -        suite.addTestSuite(IgniteCacheTxNearPeekModesTest.class);
 -        suite.addTestSuite(IgniteCacheTxLocalPeekModesTest.class);
 -        suite.addTestSuite(IgniteCacheTxReplicatedPeekModesTest.class);
 -
 -        // TODO: IGNITE-114.
 -        // suite.addTestSuite(IgniteCacheInvokeReadThroughTest.class);
 -        // suite.addTestSuite(GridCacheVersionMultinodeTest.class);
 -
 -        suite.addTestSuite(IgniteCacheNearReadCommittedTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicCopyOnReadDisabledTest.class);
 -        suite.addTestSuite(IgniteCacheTxCopyOnReadDisabledTest.class);
 -
 -        suite.addTestSuite(IgniteCacheTxPreloadNoWriteTest.class);
 -
 -        suite.addTestSuite(IgniteDynamicCacheStartSelfTest.class);
 -        suite.addTestSuite(IgniteCacheDynamicStopSelfTest.class);
 -        suite.addTestSuite(IgniteCacheConfigurationTemplateTest.class);
 -        suite.addTestSuite(IgniteCacheConfigurationDefaultTemplateTest.class);
 -
 -        suite.addTestSuite(GridCacheTxLoadFromStoreOnLockSelfTest.class);
 -
 -        suite.addTestSuite(GridCacheMarshallingNodeJoinSelfTest.class);
 -
 -        suite.addTestSuite(IgniteCacheJdbcBlobStoreNodeRestartTest.class);
 -
 -        suite.addTestSuite(IgniteCacheAtomicLocalStoreValueTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicStoreValueTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicNearEnabledStoreValueTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicPrimaryWriteOrderStoreValueTest.class);
 -        suite.addTestSuite(IgniteCacheAtomicPrimaryWriteOrderNearEnabledStoreValueTest.class);
 -        suite.addTestSuite(IgniteCacheTxLocalStoreValueTest.class);
 -        suite.addTestSuite(IgniteCacheTxStoreValueTest.class);
 -        suite.addTestSuite(IgniteCacheTxNearEnabledStoreValueTest.class);
 -
 -        suite.addTestSuite(IgniteCacheLockFailoverSelfTest.class);
 -        suite.addTestSuite(IgniteCacheMultiTxLockSelfTest.class);
 -
 -        suite.addTestSuite(IgniteInternalCacheTypesTest.class);
 -
 -        suite.addTestSuite(IgniteExchangeFutureHistoryTest.class);
 -
 -        suite.addTestSuite(CacheNoValueClassOnServerNodeTest.class);
 -
+         suite.addTestSuite(IgniteCacheNearLockValueSelfTest.class);
+ 
          return suite;
      }
  }


[19/19] incubator-ignite git commit: ignite-471-2: fixed grid messaging tests

Posted by sb...@apache.org.
ignite-471-2: fixed grid messaging 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/19d3aac7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/19d3aac7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/19d3aac7

Branch: refs/heads/ignite-471-2
Commit: 19d3aac7c878255639dbc30a163699b6883b2b00
Parents: 490a252
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed May 20 12:47:49 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed May 20 12:47:49 2015 +0300

----------------------------------------------------------------------
 ...GridMessagingNoPeerClassLoadingSelfTest.java | 70 ++++++++++++++------
 1 file changed, 48 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/19d3aac7/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingNoPeerClassLoadingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingNoPeerClassLoadingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingNoPeerClassLoadingSelfTest.java
index 610ce64..b6a775f 100644
--- a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingNoPeerClassLoadingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingNoPeerClassLoadingSelfTest.java
@@ -17,8 +17,10 @@
 
 package org.apache.ignite.messaging;
 
+import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.resources.*;
 import org.apache.ignite.testframework.config.*;
 
 import java.net.*;
@@ -54,37 +56,61 @@ public class GridMessagingNoPeerClassLoadingSelfTest extends GridMessagingSelfTe
 
         Class rcCls = extLdr.loadClass(EXT_RESOURCE_CLS_NAME);
 
+        MessageListener list = new MessageListener(ignite1);
+
+        ignite2.message().remoteListen("", list);
+
+        message(ignite1.cluster().forRemotes()).send(null, Collections.singleton(rcCls.newInstance()));
+
+        /*
+            We shouldn't get a message, because remote node won't be able to
+            unmarshal it (peer class loading is disabled.)
+         */
+        assertFalse(list.rcvLatch.await(3, TimeUnit.SECONDS));
+    }
+
+    /**
+     *
+     */
+    private static class MessageListener<UUID, Object> implements P2<UUID, Object> {
+        /** */
         final AtomicBoolean error = new AtomicBoolean(false); //to make it modifiable
 
+        /** */
         final CountDownLatch rcvLatch = new CountDownLatch(1);
 
-        ignite2.message().remoteListen("", new P2<UUID, Object>() {
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                try {
-                    log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
+        /** */
+        final Ignite sender;
 
-                    if (!nodeId.equals(ignite1.cluster().localNode().id())) {
-                        log.error("Unexpected sender node: " + nodeId);
+        /** */
+        @LoggerResource
+        private transient IgniteLogger log;
 
-                        error.set(true);
+        /**
+         * @param sender
+         */
+        private MessageListener(Ignite sender) {
+            this.sender = sender;
+        }
 
-                        return false;
-                    }
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID nodeId, Object msg) {
+            try {
+                log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
 
-                    return true;
-                }
-                finally {
-                    rcvLatch.countDown();
-                }
-            }
-        });
+                if (!nodeId.equals(sender.cluster().localNode().id())) {
+                    log.error("Unexpected sender node: " + nodeId);
 
-        message(ignite1.cluster().forRemotes()).send(null, Collections.singleton(rcCls.newInstance()));
+                    error.set(true);
 
-        /*
-            We shouldn't get a message, because remote node won't be able to
-            unmarshal it (peer class loading is disabled.)
-         */
-        assertFalse(rcvLatch.await(3, TimeUnit.SECONDS));
+                    return false;
+                }
+
+                return true;
+            }
+            finally {
+                rcvLatch.countDown();
+            }
+        }
     }
 }


[14/19] incubator-ignite git commit: GG-9614 Interop .Net: Implement GridEvents API. - done

Posted by sb...@apache.org.
GG-9614 Interop .Net: Implement GridEvents API. - done


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

Branch: refs/heads/ignite-471-2
Commit: 26a713c84b96bb0d89b802bd2ab3cd1319da0e2c
Parents: 3f7a80a
Author: ptupitsyn <pt...@gridgain.com>
Authored: Tue May 19 18:26:20 2015 +0300
Committer: ptupitsyn <pt...@gridgain.com>
Committed: Tue May 19 18:26:20 2015 +0300

----------------------------------------------------------------------
 .../internal/GridEventConsumeHandler.java       | 26 ++++++++++++++
 .../interop/InteropAwareEventFilter.java        | 37 ++++++++++++++++++++
 .../interop/InteropLocalEventListener.java      | 28 +++++++++++++++
 .../eventstorage/GridEventStorageManager.java   | 24 ++++++++++++-
 4 files changed, 114 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/26a713c8/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
index c60646e..505204d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.interop.*;
 import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
@@ -124,6 +125,9 @@ class GridEventConsumeHandler implements GridContinuousHandler {
         if (filter != null)
             ctx.resource().injectGeneric(filter);
 
+        if (filter instanceof InteropAwareEventFilter)
+            ((InteropAwareEventFilter)filter).initialize(ctx);
+
         final boolean loc = nodeId.equals(ctx.localNodeId());
 
         lsnr = new GridLocalEventListener() {
@@ -188,6 +192,28 @@ class GridEventConsumeHandler implements GridContinuousHandler {
 
         if (lsnr != null)
             ctx.event().removeLocalEventListener(lsnr, types);
+
+        RuntimeException err = null;
+
+        try {
+            if (filter instanceof InteropAwareEventFilter)
+                ((InteropAwareEventFilter)filter).close();
+        }
+        catch(RuntimeException ex) {
+            err = ex;
+        }
+
+        try {
+            if (cb instanceof InteropLocalEventListener)
+                ((InteropLocalEventListener)cb).close();
+        }
+        catch (RuntimeException ex) {
+            if (err == null)
+                err = ex;
+        }
+
+        if (err != null)
+            throw err;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/26a713c8/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropAwareEventFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropAwareEventFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropAwareEventFilter.java
new file mode 100644
index 0000000..8dbc73b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropAwareEventFilter.java
@@ -0,0 +1,37 @@
+/*
+ * 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.interop;
+
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.lang.*;
+
+/**
+ * Special version of predicate for events with initialize/close callbacks.
+ */
+public interface InteropAwareEventFilter<E extends Event> extends IgnitePredicate<E> {
+    /**
+     * Initializes the filter.
+     */
+    public void initialize(GridKernalContext ctx);
+
+    /**
+     * Closes the filter.
+     */
+    public void close();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/26a713c8/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropLocalEventListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropLocalEventListener.java b/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropLocalEventListener.java
new file mode 100644
index 0000000..180863b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropLocalEventListener.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.interop;
+
+/**
+ * Special version of listener for events with close callbacks.
+ */
+public interface InteropLocalEventListener {
+    /**
+     * Closes the listener.
+     */
+    public void close();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/26a713c8/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
index 10cc99a..95c5eb1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.events.*;
+import org.apache.ignite.internal.interop.*;
 import org.apache.ignite.internal.managers.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.deployment.*;
@@ -650,6 +651,14 @@ public class GridEventStorageManager extends GridManagerAdapter<EventStorageSpi>
             }
         }
 
+        if (lsnr instanceof UserListenerWrapper)
+        {
+            IgnitePredicate p = ((UserListenerWrapper)lsnr).listener();
+
+            if (p instanceof InteropLocalEventListener)
+                ((InteropLocalEventListener)p).close();
+        }
+
         return found;
     }
 
@@ -752,7 +761,20 @@ public class GridEventStorageManager extends GridManagerAdapter<EventStorageSpi>
     public <T extends Event> Collection<T> localEvents(IgnitePredicate<T> p) {
         assert p != null;
 
-        return getSpi().localEvents(p);
+        if (p instanceof InteropAwareEventFilter) {
+            InteropAwareEventFilter p0 = (InteropAwareEventFilter)p;
+
+            p0.initialize(ctx);
+
+            try {
+                return getSpi().localEvents(p0);
+            }
+            finally {
+                p0.close();
+            }
+        }
+        else
+            return getSpi().localEvents(p);
     }
 
     /**


[05/19] incubator-ignite git commit: GG-7190

Posted by sb...@apache.org.
GG-7190


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

Branch: refs/heads/ignite-471-2
Commit: 489323b0175f6eba897caf3f40f4f0fcb970df19
Parents: 04774b5f
Author: avinogradov <av...@gridgain.com>
Authored: Mon May 18 12:54:41 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon May 18 12:54:41 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489323b0/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 5738778..7fa0a03 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
@@ -64,6 +64,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridCacheNearMultiNodeSelfTest.class));
         suite.addTest(new TestSuite(GridCacheAtomicNearMultiNodeSelfTest.class));
         suite.addTest(new TestSuite(GridCacheNearReadersSelfTest.class));
+        suite.addTest(new TestSuite(GridCacheNearReaderPreloadSelfTest.class));
         suite.addTest(new TestSuite(GridCacheAtomicNearReadersSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedAffinitySelfTest.class));
         suite.addTest(new TestSuite(GridCacheRendezvousAffinityFunctionExcludeNeighborsSelfTest.class));


[09/19] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-4' into ignite-sprint-4

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


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

Branch: refs/heads/ignite-471-2
Commit: d3c056e153c17629c1b12db593877c3c1fde9798
Parents: b218e78 9c30fba
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon May 18 11:07:11 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon May 18 11:07:11 2015 -0700

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   6 -
 assembly/release-base.xml                       |   4 +-
 bin/ignite-schema-import.bat                    |   2 +-
 bin/ignite-schema-import.sh                     |   2 +-
 bin/ignite.bat                                  |   2 +-
 bin/ignite.sh                                   |   2 +-
 bin/ignitevisorcmd.bat                          |   2 +-
 bin/ignitevisorcmd.sh                           |   2 +-
 bin/include/build-classpath.bat                 |  46 ++
 bin/include/build-classpath.sh                  |  71 +++
 bin/include/target-classpath.bat                |  46 --
 bin/include/target-classpath.sh                 |  71 ---
 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 +-
 .../ignite/codegen/MessageCodeGenerator.java    |   4 +-
 modules/core/pom.xml                            |   2 +-
 .../communication/GridIoMessageFactory.java     |   4 +-
 .../cache/DynamicCacheDescriptor.java           |  16 +-
 .../processors/cache/GridCacheAdapter.java      | 518 +++++++++---------
 .../processors/cache/GridCacheMapEntry.java     |  18 +-
 .../GridCachePartitionExchangeManager.java      |   3 +
 .../processors/cache/GridCacheProcessor.java    | 189 ++++---
 .../processors/cache/GridCacheTtlManager.java   |  42 +-
 .../processors/cache/GridCacheUtils.java        |   5 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java | 434 ---------------
 ...idCacheOptimisticCheckPreparedTxRequest.java | 232 --------
 ...dCacheOptimisticCheckPreparedTxResponse.java | 179 -------
 .../distributed/GridCacheTxRecoveryFuture.java  | 506 ++++++++++++++++++
 .../distributed/GridCacheTxRecoveryRequest.java | 261 +++++++++
 .../GridCacheTxRecoveryResponse.java            | 182 +++++++
 .../GridDistributedTxRemoteAdapter.java         |   2 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   2 +-
 .../dht/GridPartitionedGetFuture.java           |   2 +-
 .../cache/query/GridCacheSqlQuery.java          |   2 +-
 .../cache/query/GridCacheTwoStepQuery.java      |  17 +
 .../cache/transactions/IgniteInternalTx.java    |   5 +-
 .../cache/transactions/IgniteTxAdapter.java     |   2 +-
 .../cache/transactions/IgniteTxHandler.java     |  38 +-
 .../transactions/IgniteTxLocalAdapter.java      |   2 +-
 .../cache/transactions/IgniteTxManager.java     | 173 ++----
 .../datastreamer/DataStreamerImpl.java          |   2 +
 .../processors/igfs/IgfsDataManager.java        |   3 +
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../internal/processors/igfs/IgfsUtils.java     |  11 +-
 .../internal/visor/query/VisorQueryArg.java     |  14 +-
 .../internal/visor/query/VisorQueryJob.java     |   2 +
 .../communication/tcp/TcpCommunicationSpi.java  |   2 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   8 +-
 .../resources/META-INF/classnames.properties    |  12 +-
 .../internal/GridUpdateNotifierSelfTest.java    |  21 +-
 .../processors/cache/CacheGetFromJobTest.java   | 110 ++++
 .../GridCacheAbstractFailoverSelfTest.java      |   4 +-
 .../GridCacheAbstractNodeRestartSelfTest.java   |  94 ++--
 ...xOriginatingNodeFailureAbstractSelfTest.java |   2 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   2 +-
 ...rDisabledPrimaryNodeFailureRecoveryTest.java |  31 ++
 ...rtitionedPrimaryNodeFailureRecoveryTest.java |  31 ++
 ...woBackupsPrimaryNodeFailureRecoveryTest.java |  37 ++
 ...ePrimaryNodeFailureRecoveryAbstractTest.java | 533 +++++++++++++++++++
 .../GridCachePartitionedNodeRestartTest.java    |   4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |   4 +-
 .../GridCacheReplicatedNodeRestartSelfTest.java |   2 +
 .../IgniteCacheExpiryPolicyAbstractTest.java    |   2 +-
 .../IgniteCacheExpiryPolicyTestSuite.java       |   2 +
 .../expiry/IgniteCacheTtlCleanupSelfTest.java   |  85 +++
 .../igfs/IgfsClientCacheSelfTest.java           | 132 +++++
 .../processors/igfs/IgfsOneClientNodeTest.java  | 133 +++++
 .../processors/igfs/IgfsStreamsSelfTest.java    |   2 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |   5 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   3 -
 .../IgniteCacheTxRecoverySelfTestSuite.java     |   4 +
 .../ignite/testsuites/IgniteIgfsTestSuite.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 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   4 +
 .../processors/query/h2/sql/GridSqlQuery.java   |  20 +
 .../query/h2/sql/GridSqlQueryParser.java        |  10 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |  11 +-
 .../processors/query/h2/sql/GridSqlSelect.java  |   2 +-
 .../processors/query/h2/sql/GridSqlUnion.java   |   2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   3 +
 .../h2/twostep/GridReduceQueryExecutor.java     | 119 ++++-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  21 +
 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 +-
 .../ignite/schema/generator/CodeGenerator.java  |  41 +-
 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 +-
 .../commands/cache/VisorCacheScanCommand.scala  |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 parent/pom.xml                                  |   2 +
 pom.xml                                         |  88 +--
 112 files changed, 3088 insertions(+), 1693 deletions(-)
----------------------------------------------------------------------


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

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


[03/19] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-4' into ignite-sprint-4

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


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

Branch: refs/heads/ignite-471-2
Commit: b218e78adb72d759e646a1042f44fa19b44cf09e
Parents: 5de74fe 54f9492
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue May 5 11:17:56 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue May 5 11:17:56 2015 -0700

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   2 +-
 assembly/release-base.xml                       |   2 +
 assembly/release-schema-import.xml              |  50 ++++
 .../streaming/wordcount/CacheConfig.java        |   5 -
 .../config/grid-client-config.properties        |  50 ++--
 .../ClientPropertiesConfigurationSelfTest.java  |  12 +-
 .../java/org/apache/ignite/IgniteCache.java     |   5 +
 .../org/apache/ignite/IgniteJdbcDriver.java     |  81 ++++---
 .../client/GridClientConfiguration.java         |   2 +-
 .../managers/communication/GridIoManager.java   |   8 +-
 .../processors/cache/GridCacheTtlManager.java   | 168 +++++++++-----
 .../processors/cache/GridCacheUtils.java        |   5 +-
 .../apache/ignite/lang/IgniteAsyncSupport.java  |   4 +-
 .../discovery/tcp/TcpClientDiscoverySpi.java    |   4 -
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   4 -
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   8 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java | 154 +++++++++++++
 ...GridCacheLoadingConcurrentGridStartTest.java | 154 -------------
 .../tcp/TcpClientDiscoverySelfTest.java         |   8 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 modules/schema-import/pom.xml                   |   6 +-
 pom.xml                                         | 227 ++++++++++++++++---
 23 files changed, 625 insertions(+), 340 deletions(-)
----------------------------------------------------------------------



[18/19] incubator-ignite git commit: ignite-471-2: GridQueryProcessor fix and other test suites fixes

Posted by sb...@apache.org.
ignite-471-2: GridQueryProcessor fix and other test suites fixes


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

Branch: refs/heads/ignite-471-2
Commit: 490a2526906e7d22b1e20da2436f664393dc65f5
Parents: 379c73f
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed May 20 12:38:23 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed May 20 12:38:23 2015 +0300

----------------------------------------------------------------------
 .../ignite/messaging/GridMessagingSelfTest.java | 559 +++++++++----------
 .../ignite/testframework/GridTestUtils.java     |  16 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |  26 +-
 .../IgniteMarshallerSelfTestSuite.java          |  28 +-
 4 files changed, 317 insertions(+), 312 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/490a2526/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
index c033750..36c2a9f 100644
--- a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -182,8 +181,6 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        ((OptimizedMarshaller)cfg.getMarshaller()).setRequireSerializable(false);
-
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
         discoSpi.setIpFinder(ipFinder);
@@ -199,34 +196,9 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
      * @throws Exception If error occurs.
      */
     public void testSendReceiveMessage() throws Exception {
-        final Collection<Object> rcvMsgs = new GridConcurrentHashSet<>();
-
-        final AtomicBoolean error = new AtomicBoolean(false); //to make it modifiable
-
-        final CountDownLatch rcvLatch = new CountDownLatch(3);
-
-        ignite1.message().localListen(null, new P2<UUID, Object>() {
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                try {
-                    log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
-
-                    if (!nodeId.equals(ignite2.cluster().localNode().id())) {
-                        log.error("Unexpected sender node: " + nodeId);
+        ReceiveRemoteMessageListener<UUID, Object> list = new ReceiveRemoteMessageListener<>(ignite2, 3);
 
-                        error.set(true);
-
-                        return false;
-                    }
-
-                    rcvMsgs.add(msg);
-
-                    return true;
-                }
-                finally {
-                    rcvLatch.countDown();
-                }
-            }
-        });
+        ignite1.message().localListen(null, list);
 
         ClusterGroup rNode1 = ignite2.cluster().forRemotes();
 
@@ -234,13 +206,13 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
         message(rNode1).send(null, MSG_2);
         message(rNode1).send(null, MSG_3);
 
-        assertTrue(rcvLatch.await(3, TimeUnit.SECONDS));
+        assertTrue(list.rcvLatch.await(3, TimeUnit.SECONDS));
 
-        assertFalse(error.get());
+        assertFalse(list.error.get());
 
-        assertTrue(rcvMsgs.contains(MSG_1));
-        assertTrue(rcvMsgs.contains(MSG_2));
-        assertTrue(rcvMsgs.contains(MSG_3));
+        assertTrue(list.rcvMsgs.contains(MSG_1));
+        assertTrue(list.rcvMsgs.contains(MSG_2));
+        assertTrue(list.rcvMsgs.contains(MSG_3));
     }
 
     /**
@@ -607,24 +579,9 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
      * @throws Exception If error occurs.
      */
     public void testRemoteListen() throws Exception {
-        final Collection<Object> rcvMsgs = new GridConcurrentHashSet<>();
-
-        final CountDownLatch rcvLatch = new CountDownLatch(4);
+        MessageReceiverListener list = new MessageReceiverListener();
 
-        ignite2.message().remoteListen(null, new P2<UUID, Object>() {
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                try {
-                    log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
-
-                    rcvMsgs.add(msg);
-
-                    return true;
-                }
-                finally {
-                    rcvLatch.countDown();
-                }
-            }
-        });
+        ignite2.message().remoteListen(null, list);
 
         ClusterGroup prj2 = ignite1.cluster().forRemotes(); // Includes node from grid2.
 
@@ -632,11 +589,11 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
         message(prj2).send(null, MSG_2);
         message(ignite2.cluster().forLocal()).send(null, MSG_3);
 
-        assertFalse(rcvLatch.await(3, TimeUnit.SECONDS)); // We should get only 3 message.
+        assertFalse(list.rcvLatch.await(3, TimeUnit.SECONDS)); // We should get only 3 message.
 
-        assertTrue(rcvMsgs.contains(MSG_1));
-        assertTrue(rcvMsgs.contains(MSG_2));
-        assertTrue(rcvMsgs.contains(MSG_3));
+        assertTrue(list.rcvMsgs.contains(MSG_1));
+        assertTrue(list.rcvMsgs.contains(MSG_2));
+        assertTrue(list.rcvMsgs.contains(MSG_3));
     }
 
     /**
@@ -644,45 +601,19 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings("TooBroadScope")
     public void testStopRemoteListen() throws Exception {
-        final AtomicInteger msgCnt1 = new AtomicInteger();
-
-        final AtomicInteger msgCnt2 = new AtomicInteger();
-
-        final AtomicInteger msgCnt3 = new AtomicInteger();
+        final IncrementTestListener list1 = new IncrementTestListener();
+        final IncrementTestListener list2 = new IncrementTestListener();
+        final IncrementTestListener list3 = new IncrementTestListener();
 
         final String topic1 = null;
         final String topic2 = "top2";
         final String topic3 = "top3";
 
-        UUID id1 = ignite2.message().remoteListen(topic1, new P2<UUID, Object>() {
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                System.out.println(Thread.currentThread().getName() + " Listener1 received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
-
-                msgCnt1.incrementAndGet();
-
-                return true;
-            }
-        });
-
-        UUID id2 = ignite2.message().remoteListen(topic2, new P2<UUID, Object>() {
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                System.out.println(Thread.currentThread().getName() + " Listener2 received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
-
-                msgCnt2.incrementAndGet();
-
-                return true;
-            }
-        });
-
-        UUID id3 = ignite2.message().remoteListen(topic3, new P2<UUID, Object>() {
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                System.out.println(Thread.currentThread().getName() + " Listener3 received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
+        UUID id1 = ignite2.message().remoteListen(topic1, list1);
 
-                msgCnt3.incrementAndGet();
+        UUID id2 = ignite2.message().remoteListen(topic2, list2);
 
-                return true;
-            }
-        });
+        UUID id3 = ignite2.message().remoteListen(topic3, list3);
 
         message(ignite1.cluster().forRemotes()).send(topic1, "msg1-1");
         message(ignite1.cluster().forRemotes()).send(topic2, "msg1-2");
@@ -690,13 +621,13 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.waitForCondition(new PA() {
             @Override public boolean apply() {
-                return msgCnt1.get() > 0 && msgCnt2.get() > 0 && msgCnt3.get() > 0;
+                return list1.msgCnt.get() > 0 && list2.msgCnt.get() > 0 && list3.msgCnt.get() > 0;
             }
         }, 5000);
 
-        assertEquals(1, msgCnt1.get());
-        assertEquals(1, msgCnt2.get());
-        assertEquals(1, msgCnt3.get());
+        assertEquals(1, list1.msgCnt.get());
+        assertEquals(1, list2.msgCnt.get());
+        assertEquals(1, list3.msgCnt.get());
 
         ignite2.message().stopRemoteListen(id2);
 
@@ -706,13 +637,13 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.waitForCondition(new PA() {
             @Override public boolean apply() {
-                return msgCnt1.get() > 1 && msgCnt3.get() > 1;
+                return list1.msgCnt.get() > 1 && list3.msgCnt.get() > 1;
             }
         }, 5000);
 
-        assertEquals(2, msgCnt1.get());
-        assertEquals(1, msgCnt2.get());
-        assertEquals(2, msgCnt3.get());
+        assertEquals(2, list1.msgCnt.get());
+        assertEquals(1, list2.msgCnt.get());
+        assertEquals(2, list3.msgCnt.get());
 
         ignite2.message().stopRemoteListen(id2); // Try remove one more time.
 
@@ -725,9 +656,9 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         U.sleep(1000);
 
-        assertEquals(2, msgCnt1.get());
-        assertEquals(1, msgCnt2.get());
-        assertEquals(2, msgCnt3.get());
+        assertEquals(2, list1.msgCnt.get());
+        assertEquals(1, list2.msgCnt.get());
+        assertEquals(2, list3.msgCnt.get());
     }
 
     /**
@@ -742,46 +673,21 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
             new TestMessage(MSG_2, 3000),
             new TestMessage(MSG_3));
 
-        final Collection<Object> rcvMsgs = new ConcurrentLinkedDeque<>();
+        ReceiveRemoteMessageListener receiver = new ReceiveRemoteMessageListener<>(ignite1, 3);
 
-        final AtomicBoolean error = new AtomicBoolean(false); //to make it modifiable
-
-        final CountDownLatch rcvLatch = new CountDownLatch(3);
-
-        ignite2.message().remoteListen(S_TOPIC_1, new P2<UUID, Object>() {
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                try {
-                    log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
-
-                    if (!nodeId.equals(ignite1.cluster().localNode().id())) {
-                        log.error("Unexpected sender node: " + nodeId);
-
-                        error.set(true);
-
-                        return false;
-                    }
-
-                    rcvMsgs.add(msg);
-
-                    return true;
-                }
-                finally {
-                    rcvLatch.countDown();
-                }
-            }
-        });
+        ignite2.message().remoteListen(S_TOPIC_1, receiver);
 
         ClusterGroup prj2 = ignite1.cluster().forRemotes(); // Includes node from grid2.
 
         for (TestMessage msg : msgs)
             message(prj2).sendOrdered(S_TOPIC_1, msg, 15000);
 
-        assertTrue(rcvLatch.await(6, TimeUnit.SECONDS));
+        assertTrue(receiver.rcvLatch.await(6, TimeUnit.SECONDS));
 
-        assertFalse(error.get());
+        assertFalse(receiver.error.get());
 
         //noinspection AssertEqualsBetweenInconvertibleTypes
-        assertEquals(msgs, Arrays.asList(rcvMsgs.toArray()));
+        assertEquals(msgs, Arrays.asList(receiver.rcvMsgs.toArray()));
     }
 
     /**
@@ -791,122 +697,17 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
      * @throws Exception If error occurs.
      */
     public void testRemoteListenWithIntTopic() throws Exception {
-        final Collection<Object> rcvMsgs = new GridConcurrentHashSet<>();
-
-        final AtomicBoolean error = new AtomicBoolean(false); //to make it modifiable
-
-        final CountDownLatch rcvLatch = new CountDownLatch(3);
-
-        ignite2.message().remoteListen(I_TOPIC_1, new P2<UUID, Object>() {
-            @IgniteInstanceResource
-            private transient Ignite g;
-
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                assertEquals(ignite2, g);
-
-                try {
-                    log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId +
-                        ", topic=" + I_TOPIC_1 + ']');
-
-                    if (!nodeId.equals(ignite1.cluster().localNode().id())) {
-                        log.error("Unexpected sender node: " + nodeId);
-
-                        error.set(true);
-
-                        return false;
-                    }
-
-                    if (!MSG_1.equals(msg)) {
-                        log.error("Unexpected message " + msg + " for topic: " + I_TOPIC_1);
-
-                        error.set(true);
-
-                        return false;
-                    }
-
-                    rcvMsgs.add(msg);
-
-                    return true;
-                }
-                finally {
-                    rcvLatch.countDown();
-                }
-            }
-        });
-
-        ignite2.message().remoteListen(I_TOPIC_2, new P2<UUID, Object>() {
-            @IgniteInstanceResource
-            private transient Ignite g;
-
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                assertEquals(ignite2, g);
-
-                try {
-                    log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId +
-                        ", topic=" + I_TOPIC_2 + ']');
-
-                    if (!nodeId.equals(ignite1.cluster().localNode().id())) {
-                        log.error("Unexpected sender node: " + nodeId);
-
-                        error.set(true);
-
-                        return false;
-                    }
-
-                    if (!MSG_2.equals(msg)) {
-                        log.error("Unexpected message " + msg + " for topic: " + I_TOPIC_2);
-
-                        error.set(true);
-
-                        return false;
-                    }
-
-                    rcvMsgs.add(msg);
-
-                    return true;
-                }
-                finally {
-                    rcvLatch.countDown();
-                }
-            }
-        });
-
-        ignite2.message().remoteListen(null, new P2<UUID, Object>() {
-            @IgniteInstanceResource
-            private transient Ignite g;
-
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                assertEquals(ignite2, g);
-
-                try {
-                    log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId +
-                        ", topic=default]");
+        ListenWithIntTopic topList1 = new ListenWithIntTopic(ignite1, ignite2, I_TOPIC_1, MSG_1);
 
-                    if (!nodeId.equals(ignite1.cluster().localNode().id())) {
-                        log.error("Unexpected sender node: " + nodeId);
-
-                        error.set(true);
-
-                        return false;
-                    }
-
-                    if (!MSG_3.equals(msg)) {
-                        log.error("Unexpected message " + msg + " for topic: default");
+        ListenWithIntTopic topList2 = new ListenWithIntTopic(ignite1, ignite2, I_TOPIC_2, MSG_2);
 
-                        error.set(true);
+        ListenWithIntTopic topList3 = new ListenWithIntTopic(ignite1, ignite2, null, MSG_3);
 
-                        return false;
-                    }
+        ignite2.message().remoteListen(I_TOPIC_1, topList1);
 
-                    rcvMsgs.add(msg);
+        ignite2.message().remoteListen(I_TOPIC_2, topList2);
 
-                    return true;
-                }
-                finally {
-                    rcvLatch.countDown();
-                }
-            }
-        });
+        ignite2.message().remoteListen(null, topList3);
 
         ClusterGroup prj2 = ignite1.cluster().forRemotes(); // Includes node from grid2.
 
@@ -914,13 +715,17 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
         message(prj2).send(I_TOPIC_2, MSG_2);
         message(prj2).send(null, MSG_3);
 
-        assertTrue(rcvLatch.await(3, TimeUnit.SECONDS));
+        assertTrue(topList1.rcvLatch.await(3, TimeUnit.SECONDS));
+        assertTrue(topList2.rcvLatch.await(3, TimeUnit.SECONDS));
+        assertTrue(topList3.rcvLatch.await(3, TimeUnit.SECONDS));
 
-        assertFalse(error.get());
+        assertFalse(topList1.error.get());
+        assertFalse(topList2.error.get());
+        assertFalse(topList3.error.get());
 
-        assertTrue(rcvMsgs.contains(MSG_1));
-        assertTrue(rcvMsgs.contains(MSG_2));
-        assertTrue(rcvMsgs.contains(MSG_3));
+        assertTrue(topList1.rcvMsgs.contains(MSG_1));
+        assertTrue(topList2.rcvMsgs.contains(MSG_2));
+        assertTrue(topList3.rcvMsgs.contains(MSG_3));
     }
 
     /**
@@ -936,36 +741,15 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         Class rcCls = extLdr.loadClass(EXT_RESOURCE_CLS_NAME);
 
-        final AtomicBoolean error = new AtomicBoolean(false); //to make it modifiable
-
-        final CountDownLatch rcvLatch = new CountDownLatch(1);
-
-        ignite2.message().remoteListen(S_TOPIC_1, new P2<UUID, Object>() {
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                try {
-                    log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
-
-                    if (!nodeId.equals(ignite1.cluster().localNode().id())) {
-                        log.error("Unexpected sender node: " + nodeId);
+        ReceiveRemoteMessageListener list = new ReceiveRemoteMessageListener(ignite1, 1);
 
-                        error.set(true);
-
-                        return false;
-                    }
-
-                    return true;
-                }
-                finally {
-                    rcvLatch.countDown();
-                }
-            }
-        });
+        ignite2.message().remoteListen(S_TOPIC_1, list);
 
         message(ignite1.cluster().forRemotes()).send(S_TOPIC_1, Collections.singleton(rcCls.newInstance()));
 
-        assertTrue(rcvLatch.await(3, TimeUnit.SECONDS));
+        assertTrue(list.rcvLatch.await(3, TimeUnit.SECONDS));
 
-        assertFalse(error.get());
+        assertFalse(list.error.get());
     }
 
     /**
@@ -1012,8 +796,6 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAsync() throws Exception {
-        final AtomicInteger msgCnt = new AtomicInteger();
-
         assertFalse(ignite2.message().isAsync());
 
         final IgniteMessaging msg = ignite2.message().withAsync();
@@ -1022,6 +804,8 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         assertFalse(ignite2.message().isAsync());
 
+        final IncrementTestListener list = new IncrementTestListener();
+
         GridTestUtils.assertThrows(log, new Callable<Void>() {
             @Override public Void call() throws Exception {
                 msg.future();
@@ -1032,16 +816,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         final String topic = "topic";
 
-        UUID id = msg.remoteListen(topic, new P2<UUID, Object>() {
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                System.out.println(Thread.currentThread().getName() +
-                    " Listener received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
-
-                msgCnt.incrementAndGet();
-
-                return true;
-            }
-        });
+        UUID id = msg.remoteListen(topic, list);
 
         Assert.assertNull(id);
 
@@ -1065,11 +840,11 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.waitForCondition(new PA() {
             @Override public boolean apply() {
-                return msgCnt.get() > 0;
+                return list.msgCnt.get() > 0;
             }
         }, 5000);
 
-        assertEquals(1, msgCnt.get());
+        assertEquals(1, list.msgCnt.get());
 
         msg.stopRemoteListen(id);
 
@@ -1091,7 +866,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         U.sleep(1000);
 
-        assertEquals(1, msgCnt.get());
+        assertEquals(1, list.msgCnt.get());
     }
 
     /**
@@ -1122,20 +897,218 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
         assertEquals(1, grp.nodes().size());
         assertEquals(expOldestIgnite.cluster().localNode().id(), grp.node().id());
 
-        ignite1.message(grp).remoteListen(null, new P2<UUID, Object>() {
-            @Override public boolean apply(UUID nodeId, Object msg) {
-                System.out.println("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
+        ignite1.message(grp).remoteListen(null, new ListenForOldestListener<UUID, Object>());
+
+        ignite1.message().send(null, MSG_1);
+
+        Thread.sleep(3000);
+
+        assertEquals(1, MSG_CNT.get());
+    }
+
+    /**
+     *
+     */
+    private static class IncrementTestListener<UUID, Object> implements P2<UUID, Object> {
+        /** */
+        final AtomicInteger msgCnt = new AtomicInteger();
+
+        /** */
+        @LoggerResource
+        private transient IgniteLogger log;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID nodeId, Object msg) {
+            log.info(Thread.currentThread().getName() +
+                         " Listener received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
 
-                MSG_CNT.incrementAndGet();
+            msgCnt.incrementAndGet();
+
+            return true;
+        }
+    }
+
+
+    /**
+     *
+     */
+    private static class ReceiveRemoteMessageListener<UUID, Object> implements P2<UUID, Object> {
+        /** */
+        final Collection<java.lang.Object> rcvMsgs = new ConcurrentLinkedDeque<>();
+
+        /** */
+        final AtomicBoolean error = new AtomicBoolean(false);
+
+        /** */
+        final CountDownLatch rcvLatch;
+
+        /** */
+        final Ignite sender;
+
+        /** */
+        @LoggerResource
+        private transient IgniteLogger  log;
+
+        /**
+         * @param sender
+         * @param latchCount
+         */
+        public ReceiveRemoteMessageListener(Ignite sender, int latchCount) {
+            this.sender = sender;
+            rcvLatch = new CountDownLatch(latchCount);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID nodeId, Object msg) {
+            try {
+                log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
+
+                if (!nodeId.equals(sender.cluster().localNode().id())) {
+                    log.info("Unexpected sender node: " + nodeId);
+
+                    error.set(true);
+
+                    return false;
+                }
+
+                rcvMsgs.add(msg);
 
                 return true;
             }
-        });
+            finally {
+                rcvLatch.countDown();
+            }
+        }
+    }
 
-        ignite1.message().send(null, MSG_1);
 
-        Thread.sleep(3000);
+    /**
+     *
+     */
+    private static class ListenForOldestListener<UUID, Object> implements P2<UUID, Object> {
+        /** */
+        @LoggerResource
+        private transient IgniteLogger log;
 
-        assertEquals(1, MSG_CNT.get());
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID nodeId, Object msg) {
+            log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
+
+            MSG_CNT.incrementAndGet();
+
+            return true;
+        }
+    }
+
+
+    /**
+     *
+     */
+    private static class ListenWithIntTopic implements P2<UUID, Object> {
+        /** */
+        final Collection<java.lang.Object> rcvMsgs = new ConcurrentLinkedDeque<>();
+
+        /** */
+        final AtomicBoolean error = new AtomicBoolean(false);
+
+        /** */
+        final CountDownLatch rcvLatch = new CountDownLatch(1);
+
+        /** */
+        private final Ignite sender;
+
+        /** */
+        private final Ignite receiver;
+
+        /** */
+        @IgniteInstanceResource
+        private transient Ignite g;
+
+        /** */
+        @LoggerResource
+        private transient IgniteLogger log;
+
+        /** */
+        final Integer topic;
+
+        /** */
+        final String message;
+
+        /**
+         * @param sender
+         * @param receiver
+         * @param topic
+         * @param message
+         */
+        public ListenWithIntTopic(Ignite sender, Ignite receiver, Integer topic, String message) {
+            this.sender = sender;
+            this.receiver = receiver;
+            this.topic = topic;
+            this.message = message;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID nodeId, Object msg) {
+            assertEquals(receiver, g);
+
+            try {
+                log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId +
+                             ", topic=" + topic + ']');
+
+                if (!nodeId.equals(sender.cluster().localNode().id())) {
+                    log.error("Unexpected sender node: " + nodeId);
+
+                    error.set(true);
+
+                    return false;
+                }
+
+                if (!message.equals(msg)) {
+                    log.error("Unexpected message " + msg + " for topic: " + topic);
+
+                    error.set(true);
+
+                    return false;
+                }
+
+                rcvMsgs.add(msg);
+
+                return true;
+            }
+            finally {
+                rcvLatch.countDown();
+            }
+        }
     }
+
+
+    /**
+     *
+     */
+    private static class MessageReceiverListener<UUID, Object> implements P2<UUID, Object> {
+        /** */
+        final Collection<java.lang.Object> rcvMsgs = new ConcurrentLinkedDeque<>();
+
+        /** */
+        final CountDownLatch rcvLatch = new CountDownLatch(4);
+
+        /** */
+        @LoggerResource
+        private transient IgniteLogger log;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID nodeId, Object msg) {
+            try {
+                log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');
+
+                rcvMsgs.add(msg);
+
+                return true;
+            }
+            finally {
+                rcvLatch.countDown();
+            }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/490a2526/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index e25aaee..ac3e939 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -1462,7 +1462,7 @@ public final class GridTestUtils {
         double dur = (System.currentTimeMillis() - startTime) / 1000d;
 
         System.out.printf("%s:\n operations:%d, duration=%fs, op/s=%d, latency=%fms\n", name, cnt, dur,
-            (long)(cnt / dur), dur / cnt);
+                          (long)(cnt / dur), dur / cnt);
     }
 
     /**
@@ -1495,4 +1495,18 @@ public final class GridTestUtils {
     public static String apacheIgniteTestPath() {
         return System.getProperty("IGNITE_TEST_PATH", U.getIgniteHome() + "/target/ignite");
     }
+
+    /**
+     * Adds test to the suite only if it's not in {@code ignoredTests} set.
+     *
+     * @param suite TestSuite where to place the test.
+     * @param test Test.
+     * @param ignoredTests Tests to ignore.
+     */
+    public static void addTestIfNeeded(TestSuite suite, Class test, Set<Class> ignoredTests) {
+        if (ignoredTests != null && ignoredTests.contains(test))
+            return;
+
+        suite.addTestSuite(test);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/490a2526/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 939346c..1a642d4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -32,6 +32,8 @@ import org.apache.ignite.messaging.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.testframework.config.*;
 
+import java.util.*;
+
 /**
  * Basic test suite.
  */
@@ -41,16 +43,20 @@ public class IgniteBasicTestSuite extends TestSuite {
      * @throws Exception Thrown in case of the failure.
      */
     public static TestSuite suite() throws Exception {
+        return suite(null);
+    }
+
+    /**
+     * @param ignoredTests
+     * @return Test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite(Set<Class> ignoredTests) throws Exception {
         TestSuite suite = new TestSuite("Ignite Basic Test Suite");
 
         suite.addTest(IgniteLangSelfTestSuite.suite());
         suite.addTest(IgniteUtilSelfTestSuite.suite());
-
-        Object marshClass = GridTestProperties.getProperty(GridTestProperties.MARSH_CLASS_NAME);
-
-        if (marshClass == null || marshClass.equals(OptimizedMarshaller.class.getName()) ||
-            marshClass.equals(JdkMarshaller.class.getName()))
-            suite.addTest(IgniteMarshallerSelfTestSuite.suite());
+        suite.addTest(IgniteMarshallerSelfTestSuite.suite(ignoredTests));
 
         suite.addTest(IgniteKernalSelfTestSuite.suite());
         suite.addTest(IgniteStartUpTestSuite.suite());
@@ -58,10 +64,10 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTest(IgniteP2PSelfTestSuite.suite());
         suite.addTest(IgniteCacheP2pUnmarshallingErrorTestSuit.suite());
 
-        suite.addTest(new TestSuite(GridSelfTest.class));
-        suite.addTest(new TestSuite(GridProjectionSelfTest.class));
-        suite.addTest(new TestSuite(GridMessagingSelfTest.class));
-        suite.addTest(new TestSuite(GridMessagingNoPeerClassLoadingSelfTest.class));
+        suite.addTestSuite(GridSelfTest.class);
+        suite.addTestSuite(GridProjectionSelfTest.class);
+        suite.addTestSuite(GridMessagingSelfTest.class);
+        suite.addTestSuite(GridMessagingNoPeerClassLoadingSelfTest.class);
 
         if (U.isLinux() || U.isMacOs())
             suite.addTest(IgniteIpcSharedMemorySelfTestSuite.suite());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/490a2526/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java
index 10afe10..40c32a2 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java
@@ -21,6 +21,9 @@ import junit.framework.*;
 import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.testframework.*;
+
+import java.util.*;
 
 /**
  * Test suite for all marshallers.
@@ -31,16 +34,25 @@ public class IgniteMarshallerSelfTestSuite extends TestSuite {
      * @throws Exception If failed.
      */
     public static TestSuite suite() throws Exception {
+        return suite(null);
+    }
+
+    /**
+     * @param ignoredTests
+     * @return Test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite(Set<Class> ignoredTests) throws Exception {
         TestSuite suite = new TestSuite("Ignite Marshaller Test Suite");
 
-        suite.addTest(new TestSuite(GridJdkMarshallerSelfTest.class));
-        suite.addTest(new TestSuite(OptimizedMarshallerEnumSelfTest.class));
-        suite.addTest(new TestSuite(OptimizedMarshallerSelfTest.class));
-        suite.addTest(new TestSuite(OptimizedMarshallerTest.class));
-        suite.addTest(new TestSuite(OptimizedObjectStreamSelfTest.class));
-        suite.addTest(new TestSuite(GridUnsafeDataOutputArraySizingSelfTest.class));
-        suite.addTest(new TestSuite(OptimizedMarshallerNodeFailoverTest.class));
-        suite.addTest(new TestSuite(OptimizedMarshallerSerialPersistentFieldsSelfTest.class));
+        GridTestUtils.addTestIfNeeded(suite, GridJdkMarshallerSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, OptimizedMarshallerEnumSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, OptimizedMarshallerSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, OptimizedMarshallerTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, OptimizedObjectStreamSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridUnsafeDataOutputArraySizingSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, OptimizedMarshallerNodeFailoverTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, OptimizedMarshallerSerialPersistentFieldsSelfTest.class, ignoredTests);
 
         return suite;
     }


[04/19] incubator-ignite git commit: Changed default timeout: SocketWrite - 200 AckTimeout - 50 HeartbeatFrequency -100 NetworkTimeout - 5000

Posted by sb...@apache.org.
Changed default timeout:
SocketWrite - 200
AckTimeout - 50
HeartbeatFrequency -100
NetworkTimeout - 5000


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

Branch: refs/heads/ignite-471-2
Commit: df25d35067beb8d5f40c97139eac0c06310b6666
Parents: c3dde57
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon May 18 11:41:41 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon May 18 11:41:41 2015 +0300

----------------------------------------------------------------------
 .../spi/discovery/tcp/TcpDiscoverySpiAdapter.java       | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/df25d350/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
index b7e3cd5..802da02 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
@@ -54,17 +54,17 @@ abstract class TcpDiscoverySpiAdapter extends IgniteSpiAdapter implements Discov
     /** Default socket operations timeout in milliseconds (value is <tt>200ms</tt>). */
     public static final long DFLT_SOCK_TIMEOUT = 200;
 
-    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>200ms</tt>). */
-    public static final long DFLT_ACK_TIMEOUT = 200;
+    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>50ms</tt>). */
+    public static final long DFLT_ACK_TIMEOUT = 50;
 
-    /** Default network timeout in milliseconds (value is <tt>200ms</tt>). */
-    public static final long DFLT_NETWORK_TIMEOUT = 200;
+    /** Default network timeout in milliseconds (value is <tt>5000ms</tt>). */
+    public static final long DFLT_NETWORK_TIMEOUT = 5000;
 
     /** Default value for thread priority (value is <tt>10</tt>). */
     public static final int DFLT_THREAD_PRI = 10;
 
-    /** Default heartbeat messages issuing frequency (value is <tt>300ms</tt>). */
-    public static final long DFLT_HEARTBEAT_FREQ = 300;
+    /** Default heartbeat messages issuing frequency (value is <tt>100ms</tt>). */
+    public static final long DFLT_HEARTBEAT_FREQ = 100;
 
     /** Default size of topology snapshots history. */
     public static final int DFLT_TOP_HISTORY_SIZE = 1000;


[17/19] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-471-2

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


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

Branch: refs/heads/ignite-471-2
Commit: 379c73f63ac5920978ed9032fb5efad939fc2b1e
Parents: 1f4dddd f9a4dd7
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed May 20 08:20:04 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed May 20 08:20:04 2015 +0300

----------------------------------------------------------------------
 LICENSE                                         | 238 +++++++++++++++++++
 LICENSE.txt                                     | 238 -------------------
 NOTICE                                          |  12 +
 NOTICE.txt                                      |  12 -
 assembly/release-base.xml                       |   4 +-
 .../internal/GridEventConsumeHandler.java       |  26 ++
 .../apache/ignite/internal/IgniteKernal.java    |  26 +-
 .../interop/InteropAwareEventFilter.java        |  37 +++
 .../interop/InteropLocalEventListener.java      |  28 +++
 .../managers/communication/GridIoManager.java   |   6 +-
 .../GridLifecycleAwareMessageFilter.java        |   5 +-
 .../eventstorage/GridEventStorageManager.java   |  24 +-
 .../processors/cache/GridCacheIoManager.java    |   6 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../distributed/GridDistributedTxMapping.java   |   5 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   6 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   1 +
 .../dht/atomic/GridDhtAtomicCache.java          |   4 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  42 +++-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  36 ++-
 .../colocated/GridDhtColocatedLockFuture.java   |   4 +-
 .../distributed/near/GridNearCacheEntry.java    |   2 +-
 .../cache/distributed/near/GridNearTxLocal.java |   5 +-
 .../near/GridNearTxPrepareFutureAdapter.java    |   9 +-
 .../near/GridNearTxPrepareResponse.java         |  28 ++-
 .../transactions/IgniteTxLocalAdapter.java      |   4 +-
 .../cache/transactions/IgniteTxManager.java     |  24 ++
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |  12 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java | 145 +++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../testsuites/IgniteCacheTestSuite2.java       |   1 +
 31 files changed, 695 insertions(+), 301 deletions(-)
----------------------------------------------------------------------



[02/19] incubator-ignite git commit: IGNITE-80 - Merge branch 'ignite-80-1' into ignite-sprint-4

Posted by sb...@apache.org.
IGNITE-80 - Merge branch 'ignite-80-1' into ignite-sprint-4


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

Branch: refs/heads/ignite-471-2
Commit: 5de74fe6d7f3837310cfd9f4f9e5d2636560e182
Parents: 81ce0e6 dcda61b
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Apr 29 22:49:14 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Apr 29 22:49:14 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    |  5 +--
 .../GridCachePartitionExchangeManager.java      |  4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  6 ++-
 .../dht/atomic/GridDhtAtomicCache.java          |  4 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 42 +++++++++++++++-----
 .../dht/atomic/GridNearAtomicUpdateRequest.java | 36 ++++++++++++++---
 .../colocated/GridDhtColocatedLockFuture.java   |  4 +-
 .../cache/transactions/IgniteTxManager.java     | 24 +++++++++++
 8 files changed, 101 insertions(+), 24 deletions(-)
----------------------------------------------------------------------



[12/19] incubator-ignite git commit: Filenames fix

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

Branch: refs/heads/ignite-471-2
Commit: 4423a4655c03936bb48fb70b713240aefa3f4219
Parents: 9c30fba
Author: avinogradov <av...@gridgain.com>
Authored: Tue May 19 16:42:08 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue May 19 16:42:08 2015 +0300

----------------------------------------------------------------------
 LICENSE                   | 238 +++++++++++++++++++++++++++++++++++++++++
 LICENSE.txt               | 238 -----------------------------------------
 NOTICE                    |  12 +++
 NOTICE.txt                |  12 ---
 assembly/release-base.xml |   4 +-
 5 files changed, 252 insertions(+), 252 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4423a465/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
new file mode 100644
index 0000000..7649b39
--- /dev/null
+++ b/LICENSE
@@ -0,0 +1,238 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+
+
+==============================================================================
+Apache Ignite (incubating) Subcomponents:
+
+The Apache Ignite project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses.
+
+
+==============================================================================
+For SnapTree:
+==============================================================================
+This product bundles SnapTree, which is available under a
+"3-clause BSD" license.  For details, see
+https://github.com/nbronson/snaptree/blob/master/LICENSE.
+
+==============================================================================
+For JSR 166 classes in "org.jsr166" package
+==============================================================================
+This product bundles JSR-166 classes which are donated to public domain.
+For details, see CC0 1.0 Universal (1.0), Public Domain Dedication,
+http://creativecommons.org/publicdomain/zero/1.0/
+
+==============================================================================
+For books used for tests in "org.apache.ignite.internal.processors.hadoop.books"
+==============================================================================
+This code bundles book text files used for testing purposes which contain
+the following header:
+
+This eBook is for the use of anyone anywhere at no cost and with
+almost no restrictions whatsoever.  You may copy it, give it away or
+re-use it under the terms of the Project Gutenberg License included
+with this eBook or online at www.gutenberg.org

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4423a465/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
deleted file mode 100644
index 7649b39..0000000
--- a/LICENSE.txt
+++ /dev/null
@@ -1,238 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.
-
-
-
-==============================================================================
-Apache Ignite (incubating) Subcomponents:
-
-The Apache Ignite project contains subcomponents with separate copyright
-notices and license terms. Your use of the source code for the these
-subcomponents is subject to the terms and conditions of the following
-licenses.
-
-
-==============================================================================
-For SnapTree:
-==============================================================================
-This product bundles SnapTree, which is available under a
-"3-clause BSD" license.  For details, see
-https://github.com/nbronson/snaptree/blob/master/LICENSE.
-
-==============================================================================
-For JSR 166 classes in "org.jsr166" package
-==============================================================================
-This product bundles JSR-166 classes which are donated to public domain.
-For details, see CC0 1.0 Universal (1.0), Public Domain Dedication,
-http://creativecommons.org/publicdomain/zero/1.0/
-
-==============================================================================
-For books used for tests in "org.apache.ignite.internal.processors.hadoop.books"
-==============================================================================
-This code bundles book text files used for testing purposes which contain
-the following header:
-
-This eBook is for the use of anyone anywhere at no cost and with
-almost no restrictions whatsoever.  You may copy it, give it away or
-re-use it under the terms of the Project Gutenberg License included
-with this eBook or online at www.gutenberg.org

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4423a465/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
new file mode 100644
index 0000000..298d05b
--- /dev/null
+++ b/NOTICE
@@ -0,0 +1,12 @@
+Apache Ignite (incubating)
+Copyright 2015 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+
+This software includes code from IntelliJ IDEA Community Edition
+Copyright (C) JetBrains s.r.o.
+https://www.jetbrains.com/idea/
+Licensed under Apache License, Version 2.0.
+http://search.maven.org/#artifactdetails%7Corg.jetbrains%7Cannotations%7C13.0%7Cjar

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4423a465/NOTICE.txt
----------------------------------------------------------------------
diff --git a/NOTICE.txt b/NOTICE.txt
deleted file mode 100644
index 298d05b..0000000
--- a/NOTICE.txt
+++ /dev/null
@@ -1,12 +0,0 @@
-Apache Ignite (incubating)
-Copyright 2015 The Apache Software Foundation
-
-This product includes software developed at
-The Apache Software Foundation (http://www.apache.org/).
-
-
-This software includes code from IntelliJ IDEA Community Edition
-Copyright (C) JetBrains s.r.o.
-https://www.jetbrains.com/idea/
-Licensed under Apache License, Version 2.0.
-http://search.maven.org/#artifactdetails%7Corg.jetbrains%7Cannotations%7C13.0%7Cjar

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4423a465/assembly/release-base.xml
----------------------------------------------------------------------
diff --git a/assembly/release-base.xml b/assembly/release-base.xml
index 88f1d10..21c4518 100644
--- a/assembly/release-base.xml
+++ b/assembly/release-base.xml
@@ -23,12 +23,12 @@
            http://maven.apache.org/xsd/component-1.1.2.xsd">
     <files>
         <file>
-            <source>LICENSE.txt</source>
+            <source>LICENSE</source>
             <outputDirectory>/</outputDirectory>
         </file>
 
         <file>
-            <source>NOTICE.txt</source>
+            <source>NOTICE</source>
             <outputDirectory>/</outputDirectory>
         </file>
 


[10/19] incubator-ignite git commit: IGNITE-920 - Fixed value sending in near cache.

Posted by sb...@apache.org.
IGNITE-920 - Fixed value sending in near cache.


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

Branch: refs/heads/ignite-471-2
Commit: a927eb29276796831ead8e9351e30947c4480bf8
Parents: d3c056e
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon May 18 11:38:49 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon May 18 11:38:49 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |  24 +++-
 .../distributed/GridDistributedTxMapping.java   |   5 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   1 +
 .../distributed/near/GridNearCacheEntry.java    |   2 +-
 .../cache/distributed/near/GridNearTxLocal.java |   5 +-
 .../near/GridNearTxPrepareFuture.java           |  10 +-
 .../near/GridNearTxPrepareResponse.java         |  28 +++-
 .../transactions/IgniteTxLocalAdapter.java      |   4 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java | 144 +++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 10 files changed, 212 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a927eb29/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index d98b023..2d9828a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2297,7 +2297,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), null, false).get();
+            if (ctx.cache().cache(cacheCfg.getName()) == null)
+                ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), null, false).get();
 
             return ctx.cache().publicJCache(cacheCfg.getName());
         }
@@ -2341,7 +2342,14 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false).get();
+            IgniteInternalCache<Object, Object> cache = ctx.cache().cache(cacheCfg.getName());
+
+            if (cache == null)
+                ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false).get();
+            else {
+                if (cache.configuration().getNearConfiguration() == null)
+                    ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false).get();
+            }
 
             return ctx.cache().publicJCache(cacheCfg.getName());
         }
@@ -2380,7 +2388,14 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false).get();
+            IgniteInternalCache<Object, Object> internalCache = ctx.cache().cache(cacheName);
+
+            if (internalCache == null)
+                ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false).get();
+            else {
+                if (internalCache.configuration().getNearConfiguration() == null)
+                    ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false).get();
+            }
 
             return ctx.cache().publicJCache(cacheName);
         }
@@ -2418,7 +2433,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            ctx.cache().getOrCreateFromTemplate(cacheName).get();
+            if (ctx.cache().cache(cacheName) == null)
+                ctx.cache().getOrCreateFromTemplate(cacheName).get();
 
             return ctx.cache().publicJCache(cacheName);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a927eb29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
index 58c7725..fded3c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
@@ -169,12 +169,13 @@ public class GridDistributedTxMapping implements Externalizable {
 
     /**
      * @param dhtVer DHT version.
+     * @param writeVer DHT writeVersion.
      */
-    public void dhtVersion(GridCacheVersion dhtVer) {
+    public void dhtVersion(GridCacheVersion dhtVer, GridCacheVersion writeVer) {
         this.dhtVer = dhtVer;
 
         for (IgniteTxEntry e : entries)
-            e.dhtVersion(dhtVer);
+            e.dhtVersion(writeVer);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a927eb29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 3a1a80a..8cb10cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -586,6 +586,7 @@ public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFutu
             tx.colocated() ? tx.xid() : tx.nearFutureId(),
             nearMiniId == null ? tx.xid() : nearMiniId,
             tx.xidVersion(),
+            tx.writeVersion(),
             tx.invalidPartitions(),
             ret,
             prepErr);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a927eb29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
index c7fa4ab..29a8e5e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
@@ -301,7 +301,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
         else {
             CacheObject val0 = valueBytesUnlocked();
 
-            return F.t(ver, val0);
+            return F.t(dhtVer, val0);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a927eb29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index c665354..1e9b502 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -388,15 +388,16 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
     /**
      * @param nodeId Node ID.
      * @param dhtVer DHT version.
+     * @param writeVer Write version.
      */
-    void addDhtVersion(UUID nodeId, GridCacheVersion dhtVer) {
+    void addDhtVersion(UUID nodeId, GridCacheVersion dhtVer, GridCacheVersion writeVer) {
         // This step is very important as near and DHT versions grow separately.
         cctx.versions().onReceived(nodeId, dhtVer);
 
         GridDistributedTxMapping m = mappings.get(nodeId);
 
         if (m != null)
-            m.dhtVersion(dhtVer);
+            m.dhtVersion(dhtVer, writeVer);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a927eb29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
index f573187..9284f49 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
@@ -1023,10 +1023,16 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
                     }
 
                     if (!m.empty()) {
+                        GridCacheVersion writeVer = res.writeVersion();
+
+                        // Backward compatibility.
+                        if (writeVer == null)
+                            writeVer = res.dhtVersion();
+
                         // Register DHT version.
-                        tx.addDhtVersion(m.node().id(), res.dhtVersion());
+                        tx.addDhtVersion(m.node().id(), res.dhtVersion(), writeVer);
 
-                        m.dhtVersion(res.dhtVersion());
+                        m.dhtVersion(res.dhtVersion(), writeVer);
 
                         if (m.near())
                             tx.readyNearLocks(m, res.pending(), res.committedVersions(), res.rolledbackVersions());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a927eb29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
index 2456674..f8c07f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
@@ -53,6 +53,9 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
     /** DHT version. */
     private GridCacheVersion dhtVer;
 
+    /** Write version. */
+    private GridCacheVersion writeVer;
+
     /** */
     @GridToStringInclude
     @GridDirectCollection(int.class)
@@ -101,6 +104,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
         IgniteUuid futId,
         IgniteUuid miniId,
         GridCacheVersion dhtVer,
+        GridCacheVersion writeVer,
         Collection<Integer> invalidParts,
         GridCacheReturn retVal,
         Throwable err
@@ -114,6 +118,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
         this.futId = futId;
         this.miniId = miniId;
         this.dhtVer = dhtVer;
+        this.writeVer = writeVer;
         this.invalidParts = invalidParts;
         this.retVal = retVal;
     }
@@ -158,6 +163,13 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
     }
 
     /**
+     * @return Write version.
+     */
+    public GridCacheVersion writeVersion() {
+        return writeVer;
+    }
+
+    /**
      * Adds owned value.
      *
      * @param key Key.
@@ -371,6 +383,12 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
 
                 writer.incrementState();
 
+            case 19:
+                if (!writer.writeMessage("writeVer", writeVer))
+                    return false;
+
+                writer.incrementState();
+
         }
 
         return true;
@@ -459,6 +477,14 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
 
                 reader.incrementState();
 
+            case 19:
+                writeVer = reader.readMessage("writeVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
         }
 
         return true;
@@ -471,7 +497,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 19;
+        return 20;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a927eb29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index fc3efba..5c5076e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -738,7 +738,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                     // in order to keep near entries on backup nodes until
                                     // backup remote transaction completes.
                                     if (cacheCtx.isNear()) {
-                                        ((GridNearCacheEntry)cached).recordDhtVersion(txEntry.dhtVersion());
+                                        if (txEntry.op() == CREATE || txEntry.op() == UPDATE ||
+                                            txEntry.op() == DELETE || txEntry.op() == TRANSFORM)
+                                            ((GridNearCacheEntry)cached).recordDhtVersion(txEntry.dhtVersion());
 
                                         if ((txEntry.op() == CREATE || txEntry.op() == UPDATE) &&
                                             txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a927eb29/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
new file mode 100644
index 0000000..fe60331
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
+/**
+ *
+ */
+public class IgniteCacheNearLockValueSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (getTestGridName(0).equals(gridName))
+            cfg.setClientMode(true);
+
+        cfg.setCommunicationSpi(new TestCommunicationSpi());
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDhtVersion() throws Exception {
+        CacheConfiguration<Object, Object> pCfg = new CacheConfiguration<>("partitioned");
+
+        pCfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+
+        try (IgniteCache<Object, Object> cache = ignite(0).getOrCreateCache(pCfg, new NearCacheConfiguration<>())) {
+            cache.put("key1", "val1");
+
+            for (int i = 0; i < 3; i++) {
+                ((TestCommunicationSpi)ignite(0).configuration().getCommunicationSpi()).clear();
+                ((TestCommunicationSpi)ignite(1).configuration().getCommunicationSpi()).clear();
+
+                try (Transaction tx = ignite(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                    cache.get("key1");
+
+                    tx.commit();
+                }
+
+                TestCommunicationSpi comm = (TestCommunicationSpi)ignite(0).configuration().getCommunicationSpi();
+
+                assertEquals(1, comm.requests().size());
+
+                GridCacheAdapter<Object, Object> primary = ((IgniteKernal)grid(1)).internalCache("partitioned");
+
+                GridCacheEntryEx dhtEntry = primary.peekEx(primary.context().toCacheKeyObject("key1"));
+
+                assertNotNull(dhtEntry);
+
+                GridNearLockRequest req = comm.requests().iterator().next();
+
+                assertEquals(dhtEntry.version(), req.dhtVersion(0));
+
+                // Check entry version in near cache after commit.
+                GridCacheAdapter<Object, Object> near = ((IgniteKernal)grid(0)).internalCache("partitioned");
+
+                GridNearCacheEntry nearEntry = (GridNearCacheEntry)near.peekEx(near.context().toCacheKeyObject("key1"));
+
+                assertNotNull(nearEntry);
+
+                assertEquals(dhtEntry.version(), nearEntry.dhtVersion());
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** */
+        private Collection<GridNearLockRequest> reqs = new ConcurrentLinkedDeque<>();
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
+            if (msg instanceof GridIoMessage) {
+                GridIoMessage ioMsg = (GridIoMessage)msg;
+
+                if (ioMsg.message() instanceof GridNearLockRequest)
+                    reqs.add((GridNearLockRequest)ioMsg.message());
+            }
+
+            super.sendMessage(node, msg);
+        }
+
+        /**
+         * @return Collected requests.
+         */
+        public Collection<GridNearLockRequest> requests() {
+            return reqs;
+        }
+
+        /**
+         *
+         */
+        public void clear() {
+            reqs.clear();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a927eb29/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 28b10d9..159a8d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -446,6 +446,8 @@ public class IgniteCacheTestSuite extends TestSuite {
 
         suite.addTestSuite(CacheNoValueClassOnServerNodeTest.class);
 
+        suite.addTestSuite(IgniteCacheNearLockValueSelfTest.class);
+
         return suite;
     }
 }


[11/19] incubator-ignite git commit: IGNITE-920 - Trigger TC.

Posted by sb...@apache.org.
IGNITE-920 - Trigger TC.


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

Branch: refs/heads/ignite-471-2
Commit: f1b5ecd16dd2315bab79944f192dbf9a1113b81b
Parents: a927eb2
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon May 18 14:22:54 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon May 18 14:22:54 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f1b5ecd1/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 2d9828a..ffd264d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -118,7 +118,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     @GridToStringExclude
     private GridKernalContextImpl ctx;
 
-    /** */
+    /** Configuration. */
     private IgniteConfiguration cfg;
 
     /** */


[07/19] incubator-ignite git commit: Merge branch 'ignite-timeout' into ignite-sprint-5

Posted by sb...@apache.org.
Merge branch 'ignite-timeout' 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/15d55b1a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/15d55b1a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/15d55b1a

Branch: refs/heads/ignite-471-2
Commit: 15d55b1a5fcaee7c91f450426bc537f3dffd77cf
Parents: 36805cc df25d35
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Mon May 18 14:49:20 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Mon May 18 14:49:20 2015 +0300

----------------------------------------------------------------------
 .../spi/discovery/tcp/TcpDiscoverySpiAdapter.java       | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[16/19] incubator-ignite git commit: Merge branch ignite-920 into ignite-sprint-5

Posted by sb...@apache.org.
Merge branch ignite-920 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/f9a4dd7f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/f9a4dd7f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/f9a4dd7f

Branch: refs/heads/ignite-471-2
Commit: f9a4dd7f580abc3d66630531bb66c3bf4d75d1d8
Parents: 02d0acd
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue May 19 11:16:27 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue May 19 11:16:27 2015 -0700

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheIoManager.java       | 1 +
 .../distributed/near/GridNearTxPrepareFutureAdapter.java    | 9 +++++++--
 .../processors/cache/IgniteCacheNearLockValueSelfTest.java  | 1 +
 3 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f9a4dd7f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index d5dd492..02f16c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -487,6 +487,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                     req.futureId(),
                     req.miniId(),
                     req.version(),
+                    req.version(),
                     null, null, null);
 
                 res.error(req.classError());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f9a4dd7f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java
index 60b918c..b7a2fee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java
@@ -214,10 +214,15 @@ public abstract class GridNearTxPrepareFutureAdapter extends GridCompoundIdentit
         }
 
         if (!m.empty()) {
+            GridCacheVersion writeVer = res.writeVersion();
+
+            if (writeVer == null)
+                writeVer = res.dhtVersion();
+
             // Register DHT version.
-            tx.addDhtVersion(m.node().id(), res.dhtVersion());
+            tx.addDhtVersion(m.node().id(), res.dhtVersion(), writeVer);
 
-            m.dhtVersion(res.dhtVersion());
+            m.dhtVersion(res.dhtVersion(), writeVer);
 
             if (m.near())
                 tx.readyNearLocks(m, res.pending(), res.committedVersions(), res.rolledbackVersions());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f9a4dd7f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
index fe60331..5cc9d04 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;


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

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


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

Branch: refs/heads/ignite-471-2
Commit: 36805cc1dd157b343301ff4661bbdf3db5b596ff
Parents: 489323b c3dde57
Author: avinogradov <av...@gridgain.com>
Authored: Mon May 18 12:55:06 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon May 18 12:55:06 2015 +0300

----------------------------------------------------------------------
 .../socket/WordsSocketStreamerServer.java       |   2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 136 ++++---------
 .../internal/interop/InteropBootstrap.java      |  34 ++++
 .../interop/InteropBootstrapFactory.java        |  39 ++++
 .../internal/interop/InteropIgnition.java       | 166 ++++++++++++++++
 .../internal/interop/InteropProcessor.java      |  36 ++++
 .../processors/cache/GridCacheAdapter.java      |   8 +-
 .../processors/cache/GridCacheMapEntry.java     |  35 +---
 .../distributed/GridDistributedLockRequest.java | 111 ++---------
 .../GridDistributedTxFinishRequest.java         |  70 ++-----
 .../GridDistributedTxPrepareRequest.java        | 112 +++--------
 .../GridDistributedTxRemoteAdapter.java         |  20 +-
 .../distributed/dht/GridDhtLockFuture.java      |   2 -
 .../distributed/dht/GridDhtLockRequest.java     |  45 ++---
 .../dht/GridDhtTransactionalCacheAdapter.java   |   6 -
 .../distributed/dht/GridDhtTxFinishFuture.java  |   3 -
 .../distributed/dht/GridDhtTxFinishRequest.java |  43 ++---
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 -
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  68 +------
 .../distributed/dht/GridDhtTxPrepareFuture.java |  18 +-
 .../dht/GridDhtTxPrepareRequest.java            |  60 +++---
 .../cache/distributed/dht/GridDhtTxRemote.java  |   8 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   6 -
 .../distributed/near/GridNearLockFuture.java    |   6 -
 .../distributed/near/GridNearLockRequest.java   |  61 +++---
 .../near/GridNearOptimisticTxPrepareFuture.java |  15 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   2 -
 .../near/GridNearTransactionalCache.java        |   4 -
 .../near/GridNearTxFinishRequest.java           |  28 +--
 .../cache/distributed/near/GridNearTxLocal.java |  20 +-
 .../near/GridNearTxPrepareRequest.java          |  52 +++--
 .../distributed/near/GridNearTxRemote.java      |  24 +--
 .../cache/transactions/IgniteInternalTx.java    |  10 -
 .../transactions/IgniteTransactionsImpl.java    |   4 +-
 .../cache/transactions/IgniteTxAdapter.java     |  72 +------
 .../cache/transactions/IgniteTxEntry.java       |  48 +----
 .../cache/transactions/IgniteTxHandler.java     |   6 -
 .../transactions/IgniteTxLocalAdapter.java      | 165 ++--------------
 .../cache/transactions/IgniteTxLocalEx.java     |  21 +-
 .../cache/transactions/IgniteTxManager.java     |  62 +-----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  26 ---
 .../near/IgniteCacheNearOnlyTxTest.java         | 190 +++++++++++++++++++
 .../processors/cache/jta/CacheJtaManager.java   |   4 +-
 43 files changed, 744 insertions(+), 1110 deletions(-)
----------------------------------------------------------------------



[08/19] incubator-ignite git commit: # Minor changes.

Posted by sb...@apache.org.
# Minor changes.


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

Branch: refs/heads/ignite-471-2
Commit: ba7fddb004a840fdad66bc9fd127eafda27fee2f
Parents: 15d55b1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon May 18 17:45:38 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon May 18 17:45:38 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/managers/communication/GridIoManager.java  | 6 +++---
 .../communication/GridLifecycleAwareMessageFilter.java         | 5 ++++-
 2 files changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ba7fddb0/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 16d582b..c877d57 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -1697,10 +1697,10 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             this.predLsnr = predLsnr;
 
             if (predLsnr != null) {
-                ctx.resource().injectGeneric(predLsnr);
-
                 if (predLsnr instanceof GridLifecycleAwareMessageFilter)
-                    ((GridLifecycleAwareMessageFilter)predLsnr).initialize();
+                    ((GridLifecycleAwareMessageFilter)predLsnr).initialize(ctx);
+                else
+                    ctx.resource().injectGeneric(predLsnr);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ba7fddb0/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridLifecycleAwareMessageFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridLifecycleAwareMessageFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridLifecycleAwareMessageFilter.java
index cb99d2e..f8cd78f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridLifecycleAwareMessageFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridLifecycleAwareMessageFilter.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.managers.communication;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 
 /**
@@ -25,8 +26,10 @@ import org.apache.ignite.lang.*;
 public interface GridLifecycleAwareMessageFilter<K, V> extends IgniteBiPredicate<K, V> {
     /**
      * Initializes the filter.
+     *
+     * @param ctx Kernal context.
      */
-    public void initialize();
+    public void initialize(GridKernalContext ctx);
 
     /**
      * Closes the filter.


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

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

Branch: refs/heads/ignite-471-2
Commit: 3f7a80a26ee06723f75bbb7247f4ce04d4586038
Parents: ba7fddb 4423a46
Author: avinogradov <av...@gridgain.com>
Authored: Tue May 19 16:42:46 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue May 19 16:42:46 2015 +0300

----------------------------------------------------------------------
 LICENSE                   | 238 +++++++++++++++++++++++++++++++++++++++++
 LICENSE.txt               | 238 -----------------------------------------
 NOTICE                    |  12 +++
 NOTICE.txt                |  12 ---
 assembly/release-base.xml |   4 +-
 5 files changed, 252 insertions(+), 252 deletions(-)
----------------------------------------------------------------------