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/08/21 10:13:07 UTC

[01/34] incubator-ignite git commit: IGNITE-1265 - Properly handle invalid partitions in DHT prepare response.

Repository: incubator-ignite
Updated Branches:
  refs/heads/master ebb5d4ad9 -> 0399ccd83


IGNITE-1265 - Properly handle invalid partitions in DHT prepare response.


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

Branch: refs/heads/master
Commit: 7a43dde77b47478e6b02bbab9d81ad70a2299c51
Parents: 5faffb9
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Aug 18 10:35:59 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Aug 18 10:35:59 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    |  1 -
 .../processors/cache/GridCacheUtils.java        | 20 +++++++++++++++++
 .../distributed/dht/GridDhtTxPrepareFuture.java | 23 +++++++++++++++++++-
 .../dht/GridDhtTxPrepareResponse.java           | 17 +++++++++++++++
 .../near/GridNearTxPrepareResponse.java         |  3 ---
 .../cache/transactions/IgniteInternalTx.java    |  2 +-
 .../cache/transactions/IgniteTxAdapter.java     | 19 +++++++++++-----
 .../cache/transactions/IgniteTxHandler.java     |  5 ++---
 .../ignite/internal/util/lang/GridFunc.java     | 14 ++++++++++++
 9 files changed, 90 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a43dde7/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 29e3551..c128aa6 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
@@ -493,7 +493,6 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                     req.version(),
                     null,
                     null,
-                    null,
                     null);
 
                 res.error(req.classError());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a43dde7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 41e3896..1e3cd67 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1681,6 +1681,26 @@ public class GridCacheUtils {
     }
 
     /**
+     * @param partsMap Cache ID to partition IDs collection map.
+     * @return Cache ID to partition ID array map.
+     */
+    public static Map<Integer, Integer[]> convertInvalidPartitions(Map<Integer, Set<Integer>> partsMap) {
+        Map<Integer, Integer[]> res = new HashMap<>(partsMap.size());
+
+        for (Map.Entry<Integer, Set<Integer>> entry : partsMap.entrySet()) {
+            Set<Integer> parts = entry.getValue();
+
+            Integer[] partsArray = new Integer[parts.size()];
+
+            partsArray = parts.toArray(partsArray);
+
+            res.put(entry.getKey(), partsArray);
+        }
+
+        return res;
+    }
+
+    /**
      * Stops store session listeners.
      *
      * @param ctx Kernal context.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a43dde7/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 af0fbdf..27de8cf 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
@@ -574,13 +574,14 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         // Send reply back to originating near node.
         Throwable prepErr = err.get();
 
+        assert F.isEmpty(tx.invalidPartitions());
+
         GridNearTxPrepareResponse res = new GridNearTxPrepareResponse(
             tx.nearXidVersion(),
             tx.colocated() ? tx.xid() : tx.nearFutureId(),
             nearMiniId == null ? tx.xid() : nearMiniId,
             tx.xidVersion(),
             tx.writeVersion(),
-            tx.invalidPartitions(),
             ret,
             prepErr,
             null);
@@ -1194,6 +1195,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 }
 
                 // Process invalid partitions (no need to remap).
+                // Keep this loop for backward compatibility.
                 if (!F.isEmpty(res.invalidPartitions())) {
                     for (Iterator<IgniteTxEntry> it = dhtMapping.entries().iterator(); it.hasNext();) {
                         IgniteTxEntry entry  = it.next();
@@ -1206,6 +1208,25 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                                     ", tx=" + tx + ", dhtMapping=" + dhtMapping + ']');
                         }
                     }
+                }
+
+                // Process invalid partitions (no need to remap).
+                if (!F.isEmpty(res.invalidPartitionsByCacheId())) {
+                    Map<Integer, Integer[]> invalidPartsMap = res.invalidPartitionsByCacheId();
+
+                    for (Iterator<IgniteTxEntry> it = dhtMapping.entries().iterator(); it.hasNext();) {
+                        IgniteTxEntry entry  = it.next();
+
+                        Integer[] invalidParts = invalidPartsMap.get(entry.cacheId());
+
+                        if (F.contains(invalidParts, entry.cached().partition())) {
+                            it.remove();
+
+                            if (log.isDebugEnabled())
+                                log.debug("Removed mapping for entry from dht mapping [key=" + entry.key() +
+                                    ", tx=" + tx + ", dhtMapping=" + dhtMapping + ']');
+                        }
+                    }
 
                     if (dhtMapping.empty()) {
                         dhtMap.remove(nodeId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a43dde7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
index 753c117..cc85628 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
@@ -55,6 +55,9 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
     @GridDirectCollection(int.class)
     private Collection<Integer> invalidParts;
 
+    /** Invalid partitions by cache ID. */
+    private Map<Integer, Integer[]> invalidPartsByCacheId;
+
     /** Preload entries. */
     @GridDirectCollection(GridCacheEntryInfo.class)
     private List<GridCacheEntryInfo> preloadEntries;
@@ -140,6 +143,20 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
     }
 
     /**
+     * @return Map from cacheId to an array of invalid partitions.
+     */
+    public Map<Integer, Integer[]> invalidPartitionsByCacheId() {
+        return invalidPartsByCacheId;
+    }
+
+    /**
+     * @param invalidPartsByCacheId Map from cache ID to an array of invalid partitions.
+     */
+    public void invalidPartitionsByCacheId(Map<Integer, Set<Integer>> invalidPartsByCacheId) {
+        this.invalidPartsByCacheId = CU.convertInvalidPartitions(invalidPartsByCacheId);
+    }
+
+    /**
      * Gets preload entries found on backup node.
      *
      * @return Collection of entry infos need to be preloaded.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a43dde7/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 b418500..b24c62d 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
@@ -101,7 +101,6 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
      * @param miniId Mini future ID.
      * @param dhtVer DHT version.
      * @param writeVer Write version.
-     * @param invalidParts Invalid partitions.
      * @param retVal Return value.
      * @param err Error.
      * @param clientRemapVer Not {@code null} if client node should remap transaction.
@@ -112,7 +111,6 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
         IgniteUuid miniId,
         GridCacheVersion dhtVer,
         GridCacheVersion writeVer,
-        Collection<Integer> invalidParts,
         GridCacheReturn retVal,
         Throwable err,
         AffinityTopologyVersion clientRemapVer
@@ -127,7 +125,6 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
         this.miniId = miniId;
         this.dhtVer = dhtVer;
         this.writeVer = writeVer;
-        this.invalidParts = invalidParts;
         this.retVal = retVal;
         this.clientRemapVer = clientRemapVer;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a43dde7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
index b16e950..f2f20dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
@@ -302,7 +302,7 @@ public interface IgniteInternalTx extends AutoCloseable, GridTimeoutObject {
     /**
      * @return Invalid partitions.
      */
-    public Set<Integer> invalidPartitions();
+    public Map<Integer, Set<Integer>> invalidPartitions();
 
     /**
      * Gets owned version for near remote transaction.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a43dde7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 709c208..4fc6f0c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -162,7 +162,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     private AtomicBoolean preparing = new AtomicBoolean();
 
     /** */
-    private Set<Integer> invalidParts = new GridLeanSet<>();
+    private Map<Integer, Set<Integer>> invalidParts = new HashMap<>(3);
 
     /**
      * Transaction state. Note that state is not protected, as we want to
@@ -671,16 +671,25 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     }
 
     /** {@inheritDoc} */
-    @Override public Set<Integer> invalidPartitions() {
+    @Override public Map<Integer, Set<Integer>> invalidPartitions() {
         return invalidParts;
     }
 
     /** {@inheritDoc} */
     @Override public void addInvalidPartition(GridCacheContext<?, ?> cacheCtx, int part) {
-        invalidParts.add(part);
+        Set<Integer> parts = invalidParts.get(cacheCtx.cacheId());
+
+        if (parts == null) {
+            parts = new GridLeanSet<>();
+
+            invalidParts.put(cacheCtx.cacheId(), parts);
+        }
+
+        parts.add(part);
 
         if (log.isDebugEnabled())
-            log.debug("Added invalid partition for transaction [part=" + part + ", tx=" + this + ']');
+            log.debug("Added invalid partition for transaction [cache=" + cacheCtx.name() + ", part=" + part +
+                ", tx=" + this + ']');
     }
 
     /** {@inheritDoc} */
@@ -1765,7 +1774,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         }
 
         /** {@inheritDoc} */
-        @Override public Set<Integer> invalidPartitions() {
+        @Override public Map<Integer, Set<Integer>> invalidPartitions() {
             throw new IllegalStateException("Deserialized transaction can only be used as read-only.");
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a43dde7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index e481e25..227cb34 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -290,7 +290,6 @@ public class IgniteTxHandler {
                         req.version(),
                         null,
                         null,
-                        null,
                         top.topologyVersion());
 
                     try {
@@ -803,7 +802,7 @@ public class IgniteTxHandler {
                 res.nearEvicted(nearTx.evicted());
 
             if (dhtTx != null && !F.isEmpty(dhtTx.invalidPartitions()))
-                res.invalidPartitions(dhtTx.invalidPartitions());
+                res.invalidPartitionsByCacheId(dhtTx.invalidPartitions());
 
             if (req.onePhaseCommit()) {
                 assert req.last();
@@ -1154,7 +1153,7 @@ public class IgniteTxHandler {
             if (req.last())
                 tx.state(PREPARED);
 
-            res.invalidPartitions(tx.invalidPartitions());
+            res.invalidPartitionsByCacheId(tx.invalidPartitions());
 
             if (tx.empty() && req.last()) {
                 tx.rollback();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a43dde7/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index 6f544e0..8a354ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -4083,6 +4083,20 @@ public class GridFunc {
      * @param val Value to find.
      * @return {@code True} if array contains given value.
      */
+    public static boolean contains(Integer[] arr, int val) {
+        for (Integer el : arr) {
+            if (el == val)
+                return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * @param arr Array.
+     * @param val Value to find.
+     * @return {@code True} if array contains given value.
+     */
     @SuppressWarnings("ForLoopReplaceableByForEach")
     public static boolean contains(long[] arr, long val) {
         for (int i = 0; i < arr.length; i++) {


[29/34] incubator-ignite git commit: Added TTL to multicast IP finder

Posted by sb...@apache.org.
Added TTL to multicast IP finder


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

Branch: refs/heads/master
Commit: 94e95ea77bc930a7f416386a2ede1295bddf73d0
Parents: abbd308
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Aug 20 17:36:02 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Aug 20 17:36:02 2015 -0700

----------------------------------------------------------------------
 .../TcpDiscoveryMulticastIpFinder.java          | 38 ++++++++++++++++++++
 1 file changed, 38 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/94e95ea7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
index bdc7865..e4a1626 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
@@ -99,6 +99,9 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
     /** Local address */
     private String locAddr;
 
+    /** Time to live. */
+    private Integer ttl;
+
     /** */
     @GridToStringExclude
     private Collection<AddressSender> addrSnds;
@@ -223,6 +226,32 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
         return locAddr;
     }
 
+    /**
+     * Set the default time-to-live for multicast packets sent out on this
+     * IP finder in order to control the scope of the multicast.
+     * <p>
+     * The TTL has to be in the range {@code  0 <= TTL <= 255}.
+     * <p>
+     * If TTL is {@code 0}, packets are not transmitted on the network,
+     * but may be delivered locally.
+     *
+     * @param ttl Time to live.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setTimeToLive(int ttl) {
+        this.ttl = ttl;
+    }
+
+    /**
+     * Set the default time-to-live for multicast packets sent out on this
+     * IP finder.
+     *
+     * @return Time to live.
+     */
+    public int getTimeToLive() {
+        return ttl;
+    }
+
     /** {@inheritDoc} */
     @Override public void initializeLocalAddresses(Collection<InetSocketAddress> addrs) throws IgniteSpiException {
         // If IGNITE_OVERRIDE_MCAST_GRP system property is set, use its value to override multicast group from
@@ -245,6 +274,9 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
             throw new IgniteSpiException("Invalid number of address request attempts, " +
                 "value greater than zero is expected: " + addrReqAttempts);
 
+        if (ttl != null && (ttl < 0 || ttl > 255))
+            throw new IgniteSpiException("Time-to-live value is out of 0 <= TTL <= 255 range: " + ttl);
+
         if (F.isEmpty(getRegisteredAddresses()))
             U.warn(log, "TcpDiscoveryMulticastIpFinder has no pre-configured addresses " +
                 "(it is recommended in production to specify at least one address in " +
@@ -452,6 +484,9 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
 
                     sock.setSoTimeout(resWaitTime);
 
+                    if (ttl != null)
+                        sock.setTimeToLive(ttl);
+
                     reqPckt.setData(MSG_ADDR_REQ_DATA);
 
                     try {
@@ -721,6 +756,9 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
 
             sock.joinGroup(mcastGrp);
 
+            if (ttl != null)
+                sock.setTimeToLive(ttl);
+
             return sock;
         }
 


[05/34] incubator-ignite git commit: IGNITE-1265 - Entry processor must always have the correct cache value.

Posted by sb...@apache.org.
IGNITE-1265 - Entry processor must always have the correct cache value.


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

Branch: refs/heads/master
Commit: 5065a1eccb3d71b2573d37bb6ff2c78a1bbc107c
Parents: ccaa2b2
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Aug 18 19:35:50 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Aug 18 19:35:50 2015 -0700

----------------------------------------------------------------------
 .../dht/GridClientPartitionTopology.java        |  20 +++
 .../dht/GridDhtPartitionTopology.java           |   7 +
 .../dht/GridDhtPartitionTopologyImpl.java       |  20 +++
 .../cache/distributed/dht/GridDhtTxLocal.java   |   4 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java | 136 +++++++++++++++++--
 .../cache/transactions/IgniteTxEntry.java       |  18 +++
 .../IgniteCacheEntryProcessorNodeJoinTest.java  |  54 ++++----
 .../cache/IgniteCacheInvokeReadThroughTest.java |   2 +-
 8 files changed, 223 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5065a1ec/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index c3f3e7f..531678e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -331,6 +331,26 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public GridDhtPartitionState partitionState(UUID nodeId, int part) {
+        lock.readLock().lock();
+
+        try {
+            GridDhtPartitionMap partMap = node2part.get(nodeId);
+
+            if (partMap != null) {
+                GridDhtPartitionState state = partMap.get(part);
+
+                return state == null ? EVICTED : state;
+            }
+
+            return EVICTED;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public Collection<ClusterNode> nodes(int p, AffinityTopologyVersion topVer) {
         lock.readLock().lock();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5065a1ec/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index c551fb3..7b08510 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -129,6 +129,13 @@ public interface GridDhtPartitionTopology {
     public GridDhtPartitionMap localPartitionMap();
 
     /**
+     * @param nodeId Node ID.
+     * @param part Partition.
+     * @return Partition state.
+     */
+    public GridDhtPartitionState partitionState(UUID nodeId, int part);
+
+    /**
      * @return Current update sequence.
      */
     public long updateSequence();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5065a1ec/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index de7f876..f356138 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -614,6 +614,26 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public GridDhtPartitionState partitionState(UUID nodeId, int part) {
+        lock.readLock().lock();
+
+        try {
+            GridDhtPartitionMap partMap = node2part.get(nodeId);
+
+            if (partMap != null) {
+                GridDhtPartitionState state = partMap.get(part);
+
+                return state == null ? EVICTED : state;
+            }
+
+            return EVICTED;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public Collection<ClusterNode> nodes(int p, AffinityTopologyVersion topVer) {
         Collection<ClusterNode> affNodes = cctx.affinity().nodes(p, topVer);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5065a1ec/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
index 6a72c89..7da6e07 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
@@ -363,8 +363,8 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
      * @return Future that will be completed when locks are acquired.
      */
     public IgniteInternalFuture<GridNearTxPrepareResponse> prepareAsync(
-        @Nullable Iterable<IgniteTxEntry> reads,
-        @Nullable Iterable<IgniteTxEntry> writes,
+        @Nullable Collection<IgniteTxEntry> reads,
+        @Nullable Collection<IgniteTxEntry> writes,
         Map<IgniteTxKey, GridCacheVersion> verMap,
         long msgId,
         IgniteUuid nearMiniId,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5065a1ec/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 2b7e1bc..ad1023f 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
@@ -135,6 +135,9 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
     /** Keys that should be locked. */
     private GridConcurrentHashSet<IgniteTxKey> lockKeys = new GridConcurrentHashSet<>();
 
+    /** Force keys future for correct transforms. */
+    private IgniteInternalFuture<?> forceKeysFut;
+
     /** Locks ready flag. */
     private volatile boolean locksReady;
 
@@ -291,7 +294,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
                 boolean hasFilters = !F.isEmptyOrNulls(txEntry.filters()) && !F.isAlwaysTrue(txEntry.filters());
 
-                if (hasFilters || retVal || txEntry.op() == GridCacheOperation.DELETE) {
+                if (hasFilters || retVal || txEntry.op() == DELETE || txEntry.op() == TRANSFORM) {
                     cached.unswap(retVal);
 
                     boolean readThrough = (retVal || hasFilters) &&
@@ -312,7 +315,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         null,
                         null);
 
-                    if (retVal) {
+                    if (retVal || txEntry.op() == TRANSFORM) {
                         if (!F.isEmpty(txEntry.entryProcessors())) {
                             invoke = true;
 
@@ -339,6 +342,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                                 }
                             }
 
+                            txEntry.entryProcessorCalculatedValue(val);
+
                             if (err != null || procRes != null)
                                 ret.addEntryProcessResult(txEntry.context(), key, null, procRes, err);
                             else
@@ -362,7 +367,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         ret.success(false);
                     }
                     else
-                        ret.success(txEntry.op() != GridCacheOperation.DELETE || cached.hasValue());
+                        ret.success(txEntry.op() != DELETE || cached.hasValue());
                 }
             }
             catch (IgniteCheckedException e) {
@@ -466,7 +471,25 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
      */
     private boolean mapIfLocked() {
         if (checkLocks()) {
-            prepare0();
+            if (!mapped.compareAndSet(false, true))
+                return false;
+
+            if (forceKeysFut == null || (forceKeysFut.isDone() && forceKeysFut.error() == null))
+                prepare0();
+            else {
+                forceKeysFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> f) {
+                        try {
+                            f.get();
+
+                            prepare0();
+                        }
+                        catch (IgniteCheckedException e) {
+                            onError(e);
+                        }
+                    }
+                });
+            }
 
             return true;
         }
@@ -709,7 +732,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
      * @param writes Write entries.
      * @param txNodes Transaction nodes mapping.
      */
-    public void prepare(Iterable<IgniteTxEntry> reads, Iterable<IgniteTxEntry> writes,
+    @SuppressWarnings("TypeMayBeWeakened")
+    public void prepare(Collection<IgniteTxEntry> reads, Collection<IgniteTxEntry> writes,
         Map<UUID, Collection<UUID>> txNodes) {
         if (tx.empty()) {
             tx.setRollbackOnly();
@@ -721,6 +745,15 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         this.writes = writes;
         this.txNodes = txNodes;
 
+        if (!F.isEmpty(writes)) {
+            Map<Integer, Collection<KeyCacheObject>> forceKeys = null;
+
+            for (IgniteTxEntry entry : writes)
+                forceKeys = checkNeedRebalanceKeys(entry, forceKeys);
+
+            forceKeysFut = forceRebalanceKeys(forceKeys);
+        }
+
         readyLocks();
 
         mapIfLocked();
@@ -735,12 +768,75 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
     }
 
     /**
+     * Checks if this transaction needs previous value for the given tx entry. Will use passed in map to store
+     * required key or will create new map if passed in map is {@code null}.
      *
+     * @param e TX entry.
+     * @param map Map with needed preload keys.
+     * @return Map if it was created.
      */
-    private void prepare0() {
-        if (!mapped.compareAndSet(false, true))
-            return;
+    private Map<Integer, Collection<KeyCacheObject>> checkNeedRebalanceKeys(
+        IgniteTxEntry e,
+        Map<Integer, Collection<KeyCacheObject>> map
+    ) {
+        if (retVal || !F.isEmpty(e.entryProcessors())) {
+            if (map == null)
+                map = new HashMap<>();
+
+            Collection<KeyCacheObject> keys = map.get(e.cacheId());
+
+            if (keys == null) {
+                keys = new ArrayList<>();
+
+                map.put(e.cacheId(), keys);
+            }
+
+            keys.add(e.key());
+        }
+
+        return map;
+    }
+
+    private IgniteInternalFuture<Object> forceRebalanceKeys(Map<Integer, Collection<KeyCacheObject>> keysMap) {
+        if (F.isEmpty(keysMap))
+            return null;
+
+        GridCompoundFuture<Object, Object> compFut = null;
+        IgniteInternalFuture<Object> lastForceFut = null;
 
+        for (Map.Entry<Integer, Collection<KeyCacheObject>> entry : keysMap.entrySet()) {
+            if (lastForceFut != null && compFut == null) {
+                compFut = new GridCompoundFuture();
+
+                compFut.add(lastForceFut);
+            }
+
+            int cacheId = entry.getKey();
+
+            Collection<KeyCacheObject> keys = entry.getValue();
+
+            lastForceFut = cctx.cacheContext(cacheId).preloader().request(keys, tx.topologyVersion());
+
+            if (compFut != null)
+                compFut.add(lastForceFut);
+        }
+
+        if (compFut != null) {
+            compFut.markInitialized();
+
+            return compFut;
+        }
+        else {
+            assert lastForceFut != null;
+
+            return lastForceFut;
+        }
+    }
+
+    /**
+     *
+     */
+    private void prepare0() {
         try {
             // We are holding transaction-level locks for entries here, so we can get next write version.
             onEntriesLocked();
@@ -957,7 +1053,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
     private boolean map(
         IgniteTxEntry entry,
         Map<UUID, GridDistributedTxMapping> futDhtMap,
-        Map<UUID, GridDistributedTxMapping> futNearMap) {
+        Map<UUID, GridDistributedTxMapping> futNearMap
+    ) {
         if (entry.cached().isLocal())
             return false;
 
@@ -1024,14 +1121,31 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
      * @param locMap Exclude map.
      * @return {@code True} if mapped.
      */
-    private boolean map(IgniteTxEntry entry, Iterable<ClusterNode> nodes,
-        Map<UUID, GridDistributedTxMapping> globalMap, Map<UUID, GridDistributedTxMapping> locMap) {
+    private boolean map(
+        IgniteTxEntry entry,
+        Iterable<ClusterNode> nodes,
+        Map<UUID, GridDistributedTxMapping> globalMap,
+        Map<UUID, GridDistributedTxMapping> locMap
+    ) {
         boolean ret = false;
 
         if (nodes != null) {
             for (ClusterNode n : nodes) {
                 GridDistributedTxMapping global = globalMap.get(n.id());
 
+                if (!F.isEmpty(entry.entryProcessors())) {
+                    GridDhtPartitionState state = entry.context().topology().partitionState(n.id(),
+                        entry.cached().partition());
+
+                    if (state != GridDhtPartitionState.OWNING && state != GridDhtPartitionState.EVICTED) {
+                        CacheObject procVal = entry.entryProcessorCalculatedValue();
+
+                        entry.op(procVal == null ? DELETE : UPDATE);
+                        entry.value(procVal, true, false);
+                        entry.entryProcessors(null);
+                    }
+                }
+
                 if (global == null)
                     globalMap.put(n.id(), global = new GridDistributedTxMapping(n));
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5065a1ec/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
index 247d350..7890831 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
@@ -79,6 +79,10 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     @GridDirectTransient
     private Collection<T2<EntryProcessor<Object, Object, Object>, Object[]>> entryProcessorsCol;
 
+    /** Transient field for calculated entry processor value. */
+    @GridDirectTransient
+    private CacheObject entryProcessorCalcVal;
+
     /** Transform closure bytes. */
     @GridToStringExclude
     private byte[] transformClosBytes;
@@ -775,6 +779,20 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
         return expiryPlc;
     }
 
+    /**
+     * @return Entry processor calculated value.
+     */
+    public CacheObject entryProcessorCalculatedValue() {
+        return entryProcessorCalcVal;
+    }
+
+    /**
+     * @param entryProcessorCalcVal Entry processor calculated value.
+     */
+    public void entryProcessorCalculatedValue(CacheObject entryProcessorCalcVal) {
+        this.entryProcessorCalcVal = entryProcessorCalcVal;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5065a1ec/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
index 9c17ebd..94bfd8f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
@@ -122,38 +122,44 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
         final AtomicReference<Throwable> error = new AtomicReference<>();
         final int started = 6;
 
-        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    for (int i = 0; i < started; i++) {
-                        U.sleep(1_000);
-
-                        startGrid(GRID_CNT + i);
+        try {
+            IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+                @Override public void run() {
+                    try {
+                        for (int i = 0; i < started; i++) {
+                            U.sleep(1_000);
+
+                            startGrid(GRID_CNT + i);
+                        }
+                    }
+                    catch (Exception e) {
+                        error.compareAndSet(null, e);
                     }
                 }
-                catch (Exception e) {
-                    error.compareAndSet(null, e);
-                }
-            }
-        }, 1, "starter");
+            }, 1, "starter");
 
-        try {
-            checkIncrement(invokeAll);
-        }
-        finally {
-            stop.set(true);
+            try {
+                checkIncrement(invokeAll);
+            }
+            finally {
+                stop.set(true);
 
-            fut.get(getTestTimeout());
-        }
+                fut.get(getTestTimeout());
+            }
 
-        for (int i = 0; i < NUM_SETS; i++) {
-            for (int g = 0; g < GRID_CNT + started; g++) {
-                Set<String> vals = ignite(g).<String, Set<String>>cache(null).get("set-" + i);
+            for (int i = 0; i < NUM_SETS; i++) {
+                for (int g = 0; g < GRID_CNT + started; g++) {
+                    Set<String> vals = ignite(g).<String, Set<String>>cache(null).get("set-" + i);
 
-                assertNotNull(vals);
-                assertEquals(100, vals.size());
+                    assertNotNull(vals);
+                    assertEquals(100, vals.size());
+                }
             }
         }
+        finally {
+            for (int i = 0; i < started; i++)
+                stopGrid(GRID_CNT + i);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5065a1ec/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
index 10ab1ab..b72540d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
@@ -34,7 +34,7 @@ import static org.apache.ignite.cache.CacheMode.*;
 public class IgniteCacheInvokeReadThroughTest extends IgniteCacheAbstractTest {
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-114");
+//        fail("https://issues.apache.org/jira/browse/IGNITE-114");
     }
 
     /** */


[19/34] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-1.3.3-p3' into ignite-1.3.3-p3

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


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

Branch: refs/heads/master
Commit: 2673ecab7052c2ebf5a67ac3277140e214f3b784
Parents: 939afff ef14950
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Aug 20 16:53:55 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 20 16:53:55 2015 +0700

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       |  3 +-
 .../OptimizedMarshallerNodeFailoverTest.java    | 97 +++++++++++++++++++-
 2 files changed, 94 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[20/34] incubator-ignite git commit: # ignite-1.3.3-p3 added test for cross cache transaction operations

Posted by sb...@apache.org.
# ignite-1.3.3-p3 added test for cross cache transaction operations


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

Branch: refs/heads/master
Commit: 05fda0cceda69ef569b554facca7c2397dbaaad4
Parents: ef14950
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 20 13:08:32 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 20 13:08:32 2015 +0300

----------------------------------------------------------------------
 .../cache/CrossCacheTxRandomOperationsTest.java | 490 +++++++++++++++++++
 ...gniteCachePutRetryTransactionalSelfTest.java |  15 +-
 2 files changed, 500 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/05fda0cc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
new file mode 100644
index 0000000..f3159a3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
@@ -0,0 +1,490 @@
+/*
+ * 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.cache.affinity.fair.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.processor.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
+/**
+ *
+ */
+public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String CACHE1 = "cache1";
+
+    /** */
+    private static final String CACHE2 = "cache2";
+
+    /** */
+    private static final int GRID_CNT = 5;
+
+    /** */
+    private static final int KEY_RANGE = 1000;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        if (gridName.equals(getTestGridName(GRID_CNT - 1)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(GRID_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheTxOperations() throws Exception {
+        txOperations(PARTITIONED, FULL_SYNC, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheTxOperationsPrimarySync() throws Exception {
+        txOperations(PARTITIONED, PRIMARY_SYNC, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheTxOperationsFairAffinity() throws Exception {
+        txOperations(PARTITIONED, FULL_SYNC, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheTxOperationsReplicated() throws Exception {
+        txOperations(REPLICATED, FULL_SYNC, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheTxOperationsReplicatedPrimarySync() throws Exception {
+        txOperations(REPLICATED, PRIMARY_SYNC, false);
+    }
+
+    /**
+     * @param name Cache name.
+     * @param cacheMode Cache mode.
+     * @param writeSync Write synchronization mode.
+     * @param fairAff If {@code true} uses {@link FairAffinityFunction}, otherwise {@link RendezvousAffinityFunction}.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration(String name,
+        CacheMode cacheMode,
+        CacheWriteSynchronizationMode writeSync,
+        boolean fairAff) {
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setName(name);
+        ccfg.setCacheMode(cacheMode);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(writeSync);
+
+        if (cacheMode == PARTITIONED)
+            ccfg.setBackups(1);
+
+        ccfg.setAffinity(fairAff ? new FairAffinityFunction() : new RendezvousAffinityFunction());
+
+        return ccfg;
+    }
+
+    /**
+     * @param cacheMode Cache mode.
+     * @param writeSync Write synchronization mode.
+     * @param fairAff If {@code true} uses {@link FairAffinityFunction}, otherwise {@link RendezvousAffinityFunction}.
+     * @throws Exception If failed.
+     */
+    private void txOperations(CacheMode cacheMode,
+        CacheWriteSynchronizationMode writeSync,
+        boolean fairAff) throws Exception {
+        Ignite ignite = ignite(0);
+
+        try {
+            ignite.createCache(cacheConfiguration(CACHE1, cacheMode, writeSync, fairAff));
+            ignite.createCache(cacheConfiguration(CACHE2, cacheMode, writeSync, fairAff));
+
+            txOperations(PESSIMISTIC, REPEATABLE_READ, false);
+            txOperations(PESSIMISTIC, REPEATABLE_READ, true);
+
+            txOperations(OPTIMISTIC, REPEATABLE_READ, false);
+            txOperations(OPTIMISTIC, REPEATABLE_READ, true);
+        }
+        finally {
+            ignite.destroyCache(CACHE1);
+            ignite.destroyCache(CACHE2);
+        }
+    }
+
+    /**
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @param client If {@code true} uses client node.
+     */
+    private void txOperations(TransactionConcurrency concurrency,
+        TransactionIsolation isolation,
+        boolean client) {
+        Map<TestKey, TestValue> expData1 = new HashMap<>();
+        Map<TestKey, TestValue> expData2 = new HashMap<>();
+
+        Ignite ignite = client ? ignite(GRID_CNT - 1) : ignite(0);
+
+        assertEquals(client, (boolean)ignite.configuration().isClientMode());
+
+        IgniteCache<TestKey, TestValue> cache1 = ignite.cache(CACHE1);
+        IgniteCache<TestKey, TestValue> cache2 = ignite.cache(CACHE2);
+
+        assertNotNull(cache1);
+        assertNotNull(cache2);
+        assertNotSame(cache1, cache2);
+
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        long seed = System.currentTimeMillis();
+
+        log.info("Test tx operations [concurrency=" + concurrency +
+            ", isolation=" + isolation +
+            ", client=" + client +
+            ", seed=" + seed + ']');
+
+        IgniteTransactions txs = ignite.transactions();
+
+        List<TestKey> keys = new ArrayList<>();
+
+        for (int i = 0; i < KEY_RANGE; i++)
+            keys.add(new TestKey(i));
+
+        for (int i = 0; i < 10_000; i++) {
+            if (i % 100 == 0)
+                log.info("Iteration: " + i);
+
+            boolean rollback = i % 10 == 0;
+
+            try (Transaction tx = txs.txStart(concurrency, isolation)) {
+                cacheOperation(expData1, rnd, cache1, rollback);
+                cacheOperation(expData2, rnd, cache2, rollback);
+
+                if (rollback)
+                    tx.rollback();
+                else
+                    tx.commit();
+            }
+        }
+
+        List<IgniteCache<TestKey, TestValue>> caches1 = new ArrayList<>();
+        List<IgniteCache<TestKey, TestValue>> caches2 = new ArrayList<>();
+
+        for (int i = 0; i < GRID_CNT; i++) {
+            caches1.add(ignite(i).<TestKey, TestValue>cache(CACHE1));
+            caches2.add(ignite(i).<TestKey, TestValue>cache(CACHE2));
+        }
+
+        checkData(caches1, keys, expData1);
+        checkData(caches2, keys, expData2);
+
+        cache1.removeAll();
+        cache2.removeAll();
+    }
+
+    /**
+     * @param caches Caches.
+     * @param keys Keys.
+     * @param expData Expected data.
+     */
+    private void checkData(List<IgniteCache<TestKey, TestValue>> caches,
+        List<TestKey> keys, Map<TestKey, TestValue> expData) {
+        for (IgniteCache<TestKey, TestValue> cache : caches) {
+            for (TestKey key : keys) {
+                TestValue val = cache.get(key);
+                TestValue expVal = expData.get(key);
+
+                assertEquals(expVal, val);
+            }
+        }
+    }
+
+    /**
+     * @param expData Expected cache data.
+     * @param rnd Random.
+     * @param cache Cache.
+     * @param willRollback {@code True} if will rollback transaction.
+     */
+    private void cacheOperation(
+        Map<TestKey, TestValue> expData,
+        ThreadLocalRandom rnd,
+        IgniteCache<TestKey, TestValue> cache,
+        boolean willRollback) {
+        TestKey key = key(rnd);
+        TestValue val = new TestValue(rnd.nextLong());
+
+        switch (rnd.nextInt(8)) {
+            case 0: {
+                cache.put(key, val);
+
+                if (!willRollback)
+                    expData.put(key, val);
+
+                break;
+            }
+
+            case 1: {
+                TestValue oldVal = cache.getAndPut(key, val);
+
+                TestValue expOld = expData.get(key);
+
+                assertEquals(expOld, oldVal);
+
+                if (!willRollback)
+                    expData.put(key, val);
+
+                break;
+            }
+
+            case 2: {
+                boolean rmv = cache.remove(key);
+
+                assertEquals(expData.containsKey(key), rmv);
+
+                if (!willRollback)
+                    expData.remove(key);
+
+                break;
+            }
+
+            case 3: {
+                TestValue oldVal = cache.getAndRemove(key);
+
+                TestValue expOld = expData.get(key);
+
+                assertEquals(expOld, oldVal);
+
+                if (!willRollback)
+                    expData.remove(key);
+
+                break;
+            }
+
+            case 4: {
+                boolean put = cache.putIfAbsent(key, val);
+
+                boolean expPut = !expData.containsKey(key);
+
+                assertEquals(expPut, put);
+
+                if (expPut && !willRollback)
+                    expData.put(key, val);
+
+                break;
+            }
+
+            case 5: {
+                TestValue oldVal = cache.invoke(key, new TestEntryProcessor(val.value()));
+                TestValue expOld = expData.get(key);
+
+                assertEquals(expOld, oldVal);
+
+                if (!willRollback)
+                    expData.put(key, val);
+
+                break;
+            }
+
+            case 6: {
+                TestValue oldVal = cache.invoke(key, new TestEntryProcessor(null));
+                TestValue expOld = expData.get(key);
+
+                assertEquals(expOld, oldVal);
+
+                break;
+            }
+
+            case 7: {
+                TestValue oldVal = cache.get(key);
+                TestValue expOld = expData.get(key);
+
+                assertEquals(expOld, oldVal);
+
+                break;
+            }
+
+            default:
+                assert false;
+        }
+    }
+
+    /**
+     * @param rnd Random.
+     * @return Key.
+     */
+    private TestKey key(ThreadLocalRandom rnd) {
+        return new TestKey(rnd.nextInt(KEY_RANGE));
+    }
+
+    /**
+     *
+     */
+    private static class TestKey implements Serializable {
+        /** */
+        private long key;
+
+        /**
+         * @param key Key.
+         */
+        public TestKey(long key) {
+            this.key = key;
+        }
+
+        /**
+         * @return Key.
+         */
+        public long key() {
+            return key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestKey other = (TestKey)o;
+
+            return key == other.key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return (int)(key ^ (key >>> 32));
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestKey.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestValue implements Serializable {
+        /** */
+        private long val;
+
+        /**
+         * @param val Value.
+         */
+        public TestValue(long val) {
+            this.val = val;
+        }
+
+        /**
+         * @return Value.
+         */
+        public long value() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestValue other = (TestValue)o;
+
+            return val == other.val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestValue.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestEntryProcessor implements CacheEntryProcessor<TestKey, TestValue, TestValue> {
+        /** */
+        private Long val;
+
+        /**
+         * @param val Value.
+         */
+        public TestEntryProcessor(@Nullable Long val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public TestValue process(MutableEntry<TestKey, TestValue> e, Object... args) {
+            TestValue old = e.getValue();
+
+            if (val != null)
+                e.setValue(new TestValue(val));
+
+            return old;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/05fda0cc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index 9a6bb31..9c4446d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@ -61,8 +61,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         IgniteAtomicLong atomic = ignite(0).atomicLong("TestAtomic", 0, true);
 
         IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
-            @Override
-            public Object call() throws Exception {
+            @Override public Object call() throws Exception {
                 while (!finished.get()) {
                     stopGrid(3);
 
@@ -157,7 +156,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
      * @param ignite Ignite instance.
      * @param clo Closure.
      * @return Result of closure execution.
-     * @throws Exception
+     * @throws Exception If failed.
      */
     private <T> T doInTransaction(Ignite ignite, Callable<T> clo) throws Exception {
         while (true) {
@@ -213,10 +212,16 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         }
 
         /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
         @Override public Void call() throws Exception {
-            ((IgniteCache<String, String>)cache).put("key-" + base + "-" + i, "value-" + i);
+            String key1 = "key-" + base + "-" + i;
+            String key2 = "key-" + base;
+
+            assert key1.compareTo(key2) > 0;
+
+            ((IgniteCache<String, String>)cache).put(key1, "value-" + i);
 
-            ((IgniteCache<String, Set<String>>)cache).invoke("key-" + base, new AddEntryProcessor("value-" + i));
+            ((IgniteCache<String, Set<String>>)cache).invoke(key2, new AddEntryProcessor("value-" + i));
 
             return null;
         }


[04/34] incubator-ignite git commit: IGNITE-1265 - EntryProcessorTest when nodes joining topology.

Posted by sb...@apache.org.
IGNITE-1265 - EntryProcessorTest when nodes joining topology.


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

Branch: refs/heads/master
Commit: ccaa2b20dab5438603471796b7155f309261a41f
Parents: 013d707
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Aug 18 18:38:36 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Aug 18 18:38:36 2015 -0700

----------------------------------------------------------------------
 .../IgniteCacheEntryProcessorNodeJoinTest.java  | 216 +++++++++++++++++++
 .../IgniteCacheEntryProcessorRestartTest.java   | 185 ----------------
 2 files changed, 216 insertions(+), 185 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccaa2b20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
new file mode 100644
index 0000000..9c17ebd
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
@@ -0,0 +1,216 @@
+/*
+ * 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.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.processor.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * Tests cache in-place modification logic with iterative value increment.
+ */
+public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Number of nodes to test on. */
+    private static final int GRID_CNT = 2;
+
+    /** Number of increment iterations. */
+    private static final int NUM_SETS = 50;
+
+    /** Helper for excluding stopped node from iteration logic. */
+    private AtomicReferenceArray<Ignite> grids;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cache = new CacheConfiguration();
+
+        cache.setCacheMode(PARTITIONED);
+        cache.setAtomicityMode(TRANSACTIONAL);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setBackups(1);
+        cache.setRebalanceMode(SYNC);
+
+        cfg.setCacheConfiguration(cache);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGrids(GRID_CNT);
+
+        grids = new AtomicReferenceArray<>(GRID_CNT);
+
+        for (int i = 0; i < GRID_CNT; i++)
+            grids.set(i, grid(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        grids = null;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSingleEntryProcessorNodeJoin() throws Exception {
+        checkEntryProcessorNodeJoin(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAllEntryProcessorNodeJoin() throws Exception {
+        checkEntryProcessorNodeJoin(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkEntryProcessorNodeJoin(boolean invokeAll) throws Exception {
+        final AtomicBoolean stop = new AtomicBoolean();
+        final AtomicReference<Throwable> error = new AtomicReference<>();
+        final int started = 6;
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                try {
+                    for (int i = 0; i < started; i++) {
+                        U.sleep(1_000);
+
+                        startGrid(GRID_CNT + i);
+                    }
+                }
+                catch (Exception e) {
+                    error.compareAndSet(null, e);
+                }
+            }
+        }, 1, "starter");
+
+        try {
+            checkIncrement(invokeAll);
+        }
+        finally {
+            stop.set(true);
+
+            fut.get(getTestTimeout());
+        }
+
+        for (int i = 0; i < NUM_SETS; i++) {
+            for (int g = 0; g < GRID_CNT + started; g++) {
+                Set<String> vals = ignite(g).<String, Set<String>>cache(null).get("set-" + i);
+
+                assertNotNull(vals);
+                assertEquals(100, vals.size());
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkIncrement(boolean invokeAll) throws Exception {
+        for (int k = 0; k < 100; k++) {
+            if (invokeAll) {
+                IgniteCache<String, Set<String>> cache = ignite(0).cache(null);
+
+                Map<String, Processor> procs = new LinkedHashMap<>();
+
+                for (int i = 0; i < NUM_SETS; i++) {
+                    String key = "set-" + i;
+
+                    String val = "value-" + k;
+
+                    cache.invoke(key, new Processor(val));
+                }
+
+                cache.invokeAll(procs);
+            }
+            else {
+                for (int i = 0; i < NUM_SETS; i++) {
+                    String key = "set-" + i;
+
+                    String val = "value-" + k;
+
+                    IgniteCache<String, Set<String>> cache = ignite(0).cache(null);
+
+                    cache.invoke(key, new Processor(val));
+                }
+            }
+        }
+    }
+
+    /** */
+    private static class Processor implements EntryProcessor<String, Set<String>, Void>, Serializable {
+        /** */
+        private String val;
+
+        private Processor(String val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void process(MutableEntry<String, Set<String>> e, Object... args) {
+            Set<String> vals = e.getValue();
+
+            if (vals == null)
+                vals = new HashSet<>();
+
+            vals.add(val);
+
+            e.setValue(vals);
+
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccaa2b20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorRestartTest.java
deleted file mode 100644
index c027ee4..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorRestartTest.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.apache.ignite.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.spi.communication.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import javax.cache.processor.*;
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CacheRebalanceMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-
-/**
- * Tests cache in-place modification logic with iterative value increment.
- */
-public class IgniteCacheEntryProcessorRestartTest extends GridCommonAbstractTest {
-    /** IP finder. */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** Number of nodes to test on. */
-    private static final int GRID_CNT = 2;
-
-    /** Number of increment iterations. */
-    private static final int NUM_SETS = 50;
-
-    /** Helper for excluding stopped node from iteration logic. */
-    private AtomicReferenceArray<Ignite> grids;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration cache = new CacheConfiguration();
-
-        cache.setCacheMode(PARTITIONED);
-        cache.setAtomicityMode(TRANSACTIONAL);
-        cache.setWriteSynchronizationMode(FULL_SYNC);
-        cache.setBackups(1);
-        cache.setRebalanceMode(SYNC);
-
-        cfg.setCacheConfiguration(cache);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
-
-        commSpi.setSharedMemoryPort(-1);
-
-        cfg.setCommunicationSpi(commSpi);
-
-        cfg.setDiscoverySpi(disco);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        startGrids(GRID_CNT);
-
-        grids = new AtomicReferenceArray<>(GRID_CNT);
-
-        for (int i = 0; i < GRID_CNT; i++)
-            grids.set(i, grid(i));
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids();
-
-        grids = null;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testEntryProcessorRestart() throws Exception {
-        final AtomicBoolean stop = new AtomicBoolean();
-        final AtomicReference<Throwable> error = new AtomicReference<>();
-        final int started = 6;
-
-        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    for (int i = 0; i < started; i++) {
-                        U.sleep(1_000);
-
-                        startGrid(GRID_CNT + i);
-                    }
-                }
-                catch (Exception e) {
-                    error.compareAndSet(null, e);
-                }
-            }
-        }, 1, "starter");
-
-        try {
-            checkIncrement();
-        }
-        finally {
-            stop.set(true);
-
-            fut.get(getTestTimeout());
-        }
-
-        for (int i = 0; i < NUM_SETS; i++) {
-            for (int g = 0; g < GRID_CNT + started; g++) {
-                Set<String> vals = ignite(g).<String, Set<String>>cache(null).get("set-" + i);
-
-                assertNotNull(vals);
-                assertEquals(100, vals.size());
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void checkIncrement() throws Exception {
-        for (int k = 0; k < 100; k++) {
-            for (int i = 0; i < NUM_SETS; i++) {
-                String key = "set-" + i;
-
-                String val = "value-" + k;
-
-                IgniteCache<String, Set<String>> cache = ignite(0).cache(null);
-
-                cache.invoke(key, new Processor(val));
-            }
-        }
-    }
-
-    /** */
-    private static class Processor implements EntryProcessor<String, Set<String>, Void>, Serializable {
-        /** */
-        private String val;
-
-        private Processor(String val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<String, Set<String>> e, Object... args) {
-            Set<String> vals = e.getValue();
-
-            if (vals == null)
-                vals = new HashSet<>();
-
-            vals.add(val);
-
-            e.setValue(vals);
-
-            return null;
-        }
-    }
-}


[32/34] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p3'

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p3'


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

Branch: refs/heads/master
Commit: d859587948862067c2389cb5f2461e351a35ddfa
Parents: dd204de 9603b25
Author: sboikov <sb...@gridgain.com>
Authored: Fri Aug 21 11:00:22 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Aug 21 11:00:22 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../store/jdbc/CacheAbstractJdbcStore.java      |  45 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  32 +-
 .../store/jdbc/dialect/BasicJdbcDialect.java    |   3 +
 .../cache/store/jdbc/dialect/DB2Dialect.java    |   3 +
 .../cache/store/jdbc/dialect/H2Dialect.java     |   3 +
 .../cache/store/jdbc/dialect/JdbcDialect.java   |   3 +-
 .../cache/store/jdbc/dialect/MySQLDialect.java  |   3 +
 .../cache/store/jdbc/dialect/OracleDialect.java |   3 +
 .../store/jdbc/dialect/SQLServerDialect.java    |   3 +
 .../cluster/ClusterTopologyException.java       |  18 +
 .../ignite/internal/MarshallerContextImpl.java  |  24 +-
 .../ClusterTopologyCheckedException.java        |  18 +
 .../CachePartialUpdateCheckedException.java     |  11 +-
 .../processors/cache/GridCacheAdapter.java      |  81 ++-
 .../processors/cache/GridCacheIoManager.java    |   1 -
 .../cache/GridCacheSharedContext.java           |  17 +
 .../processors/cache/GridCacheUtils.java        |  23 +
 .../distributed/GridDistributedCacheEntry.java  |   7 +
 .../dht/GridClientPartitionTopology.java        |  20 +
 .../distributed/dht/GridDhtCacheAdapter.java    |  12 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |  12 +-
 .../dht/GridDhtPartitionTopology.java           |   7 +
 .../dht/GridDhtPartitionTopologyImpl.java       |  20 +
 .../cache/distributed/dht/GridDhtTxLocal.java   |   4 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java | 182 ++++++-
 .../dht/GridDhtTxPrepareResponse.java           |  42 +-
 .../dht/GridPartitionedGetFuture.java           | 104 ++--
 .../dht/atomic/GridDhtAtomicCache.java          |  16 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   3 +
 .../dht/colocated/GridDhtColocatedCache.java    |  19 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   8 +-
 .../GridDhtPartitionsExchangeFuture.java        |  40 +-
 .../distributed/near/GridNearAtomicCache.java   |   6 +-
 .../distributed/near/GridNearCacheAdapter.java  |  15 +-
 .../distributed/near/GridNearCacheEntry.java    |  10 +-
 .../distributed/near/GridNearGetFuture.java     | 121 +++--
 .../distributed/near/GridNearLockFuture.java    |  12 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |  13 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   9 +-
 .../near/GridNearTransactionalCache.java        |   9 +-
 .../cache/distributed/near/GridNearTxLocal.java |   7 +-
 .../near/GridNearTxPrepareResponse.java         |   3 -
 .../local/atomic/GridLocalAtomicCache.java      |  17 +-
 .../cache/transactions/IgniteInternalTx.java    |   2 +-
 .../cache/transactions/IgniteTxAdapter.java     |  19 +-
 .../cache/transactions/IgniteTxEntry.java       |  18 +
 .../cache/transactions/IgniteTxHandler.java     |   5 +-
 .../ignite/internal/util/IgniteUtils.java       |  10 +-
 .../ignite/internal/util/lang/GridFunc.java     |  14 +
 .../config/store/jdbc/ignite-type-metadata.xml  |   8 +
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |  33 +-
 ...eJdbcStoreAbstractMultithreadedSelfTest.java |  16 +-
 .../ignite/cache/store/jdbc/model/Person.java   |  26 +-
 .../cache/CrossCacheTxRandomOperationsTest.java | 490 +++++++++++++++++++
 ...teAtomicCacheEntryProcessorNodeJoinTest.java |  32 ++
 .../IgniteCacheEntryProcessorNodeJoinTest.java  | 225 +++++++++
 .../IgniteCacheTopologySafeGetSelfTest.java     | 215 ++++++++
 .../GridCacheTransformEventSelfTest.java        |   2 +
 .../IgniteCacheCrossCacheTxFailoverTest.java    | 433 ++++++++++++++++
 .../IgniteCachePutRetryAbstractSelfTest.java    |   1 +
 ...gniteCachePutRetryTransactionalSelfTest.java | 187 +++++++
 .../near/GridCacheNearOnlyTopologySelfTest.java |   4 +-
 .../near/GridCacheNearTxForceKeyTest.java       |  76 +++
 ...idCachePartitionedHitsAndMissesSelfTest.java |  20 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |  97 +++-
 .../IgniteCacheFailoverTestSuite.java           |   2 +
 .../testsuites/IgniteCacheTestSuite2.java       |   4 +
 .../ignite/schema/model/PojoDescriptor.java     |   6 +-
 .../ignite/schema/model/SchemaDescriptor.java   |  61 +++
 .../schema/parser/DatabaseMetadataParser.java   |  54 +-
 .../parser/dialect/DB2MetadataDialect.java      |   3 +-
 .../parser/dialect/DatabaseMetadataDialect.java |  13 +-
 .../parser/dialect/JdbcMetadataDialect.java     | 129 +++--
 .../parser/dialect/MySQLMetadataDialect.java    |  57 +++
 .../parser/dialect/OracleMetadataDialect.java   | 111 +++--
 .../org/apache/ignite/schema/ui/Controls.java   |  25 +-
 .../ignite/schema/ui/SchemaImportApp.java       | 157 +++++-
 .../schema/test/AbstractSchemaImportTest.java   |  10 +-
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |  14 +-
 80 files changed, 3186 insertions(+), 405 deletions(-)
----------------------------------------------------------------------


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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d8595879/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/d8595879/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d8595879/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/d8595879/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index bb3673d,a85962f..6e39672
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@@ -625,30 -653,31 +653,38 @@@ public class GridPartitionedGetFuture<K
              if (log.isDebugEnabled())
                  log.debug("Remote node left grid while sending or waiting for reply (will retry): " + this);
  
-             final AffinityTopologyVersion updTopVer =
-                 new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
+             // Try getting from existing nodes.
+             if (!canRemap) {
+                 map(keys.keySet(), F.t(node, keys), topVer);
  
-             final GridFutureRemapTimeoutObject timeout = new GridFutureRemapTimeoutObject(this,
-                 cctx.kernalContext().config().getNetworkTimeout(),
-                 updTopVer,
-                 e);
- 
-             cctx.affinity().affinityReadyFuture(updTopVer).listen(
-                 new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
-                         if (timeout.finish()) {
-                             cctx.kernalContext().timeout().removeTimeoutObject(timeout);
- 
-                             try {
-                                 fut.get();
- 
-                                 // Remap.
-                                 map(keys.keySet(), F.t(node, keys), updTopVer);
- 
-                                 onDone(Collections.<K, V>emptyMap());
-                             }
-                             catch (IgniteCheckedException e) {
-                                 GridPartitionedGetFuture.this.onDone(e);
+                 onDone(Collections.<K, V>emptyMap());
+             }
+             else {
+                 final AffinityTopologyVersion updTopVer =
+                     new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
+ 
+                 final GridFutureRemapTimeoutObject timeout = new GridFutureRemapTimeoutObject(this,
+                     cctx.kernalContext().config().getNetworkTimeout(),
+                     updTopVer,
+                     e);
+ 
+                 cctx.affinity().affinityReadyFuture(updTopVer).listen(
+                     new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
+                             if (timeout.finish()) {
+                                 cctx.kernalContext().timeout().removeTimeoutObject(timeout);
+ 
 -                                // Remap.
 -                                map(keys.keySet(), F.t(node, keys), updTopVer);
++                                try {
++                                    fut.get();
+ 
 -                                onDone(Collections.<K, V>emptyMap());
++                                    // Remap.
++                                    map(keys.keySet(), F.t(node, keys), updTopVer);
++
++                                    onDone(Collections.<K, V>emptyMap());
++                                }
++                                catch (IgniteCheckedException e) {
++                                    GridPartitionedGetFuture.this.onDone(e);
++                                }
                              }
                          }
                      }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d8595879/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/d8595879/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/d8595879/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index cbf6b40,4971ca6..1a90de9
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@@ -738,22 -761,20 +750,26 @@@ public class GridDhtPartitionsExchangeF
                          break;
                      }
                      catch (IgniteFutureTimeoutCheckedException ignored) {
-                         U.warn(log, "Failed to wait for locks release future. " +
-                             "Dumping pending objects that might be the cause: " + cctx.localNodeId());
+                         if (dumpedObjects < DUMP_PENDING_OBJECTS_THRESHOLD) {
+                             U.warn(log, "Failed to wait for locks release future. " +
+                                 "Dumping pending objects that might be the cause: " + cctx.localNodeId());
  
-                         U.warn(log, "Locked keys:");
 -                            U.warn(log, "Locked entries:");
++                            U.warn(log, "Locked keys:");
 +
-                         for (IgniteTxKey key : cctx.mvcc().lockedKeys())
-                             U.warn(log, "Locked key: " + key);
++                            for (IgniteTxKey key : cctx.mvcc().lockedKeys())
++                                U.warn(log, "Locked key: " + key);
 +
-                         for (IgniteTxKey key : cctx.mvcc().nearLockedKeys())
-                             U.warn(log, "Locked near key: " + key);
++                            for (IgniteTxKey key : cctx.mvcc().nearLockedKeys())
++                                U.warn(log, "Locked near key: " + key);
  
-                         Map<IgniteTxKey, Collection<GridCacheMvccCandidate>> locks =
-                             cctx.mvcc().unfinishedLocks(exchId.topologyVersion());
+                             Map<IgniteTxKey, Collection<GridCacheMvccCandidate>> locks =
+                                 cctx.mvcc().unfinishedLocks(exchId.topologyVersion());
  
-                         for (Map.Entry<IgniteTxKey, Collection<GridCacheMvccCandidate>> e : locks.entrySet())
-                             U.warn(log, "Awaited locked entry [key=" + e.getKey() + ", mvcc=" + e.getValue() + ']');
+                             for (Map.Entry<IgniteTxKey, Collection<GridCacheMvccCandidate>> e : locks.entrySet())
 -                                U.warn(log, "Locked entry [key=" + e.getKey() + ", mvcc=" + e.getValue() + ']');
 -
++                                U.warn(log, "Awaited locked entry [key=" + e.getKey() + ", mvcc=" + e.getValue() + ']');
++                            
+                             dumpedObjects++;
+                         }
                      }
                  }
  

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d8595879/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 2017654,ca460c5..c4a5d70
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@@ -752,30 -781,31 +781,38 @@@ public final class GridNearGetFuture<K
              if (log.isDebugEnabled())
                  log.debug("Remote node left grid while sending or waiting for reply (will retry): " + this);
  
-             final AffinityTopologyVersion updTopVer =
-                 new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
- 
-             final GridFutureRemapTimeoutObject timeout = new GridFutureRemapTimeoutObject(this,
-                 cctx.kernalContext().config().getNetworkTimeout(),
-                 updTopVer,
-                 e);
- 
-             cctx.affinity().affinityReadyFuture(updTopVer).listen(
-                 new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
-                         if (timeout.finish()) {
-                             cctx.kernalContext().timeout().removeTimeoutObject(timeout);
- 
-                             try {
-                                 fut.get();
- 
-                                 // Remap.
-                                 map(keys.keySet(), F.t(node, keys), updTopVer);
- 
-                                 onDone(Collections.<K, V>emptyMap());
-                             }
-                             catch (IgniteCheckedException e) {
-                                 GridNearGetFuture.this.onDone(e);
+             // Try getting value from alive nodes.
+             if (!canRemap) {
+                 // Remap
+                 map(keys.keySet(), F.t(node, keys), topVer);
+ 
+                 onDone(Collections.<K, V>emptyMap());
+             } else {
+                 final AffinityTopologyVersion updTopVer =
+                     new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
+ 
+                 final GridFutureRemapTimeoutObject timeout = new GridFutureRemapTimeoutObject(this,
+                     cctx.kernalContext().config().getNetworkTimeout(),
+                     updTopVer,
+                     e);
+ 
+                 cctx.affinity().affinityReadyFuture(updTopVer).listen(
+                     new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
+                             if (timeout.finish()) {
+                                 cctx.kernalContext().timeout().removeTimeoutObject(timeout);
+ 
 -                                // Remap.
 -                                map(keys.keySet(), F.t(node, keys), updTopVer);
++                                try {
++                                    fut.get();
+ 
 -                                onDone(Collections.<K, V>emptyMap());
++                                    // Remap.
++                                    map(keys.keySet(), F.t(node, keys), updTopVer);
++
++                                    onDone(Collections.<K, V>emptyMap());
++                                }
++                                catch (IgniteCheckedException e) {
++                                    GridNearGetFuture.this.onDone(e);
++                                }
                              }
                          }
                      }

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d8595879/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d8595879/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d8595879/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index 1d14dec,c2fc46c..0ab5729
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@@ -64,16 -76,185 +76,191 @@@ public class IgniteCachePutRetryTransac
  
          int keysCnt = keysCount();
  
 -        for (int i = 0; i < keysCnt; i++)
 -            atomic.incrementAndGet();
 +        try {
 +            for (int i = 0; i < keysCnt; i++)
 +                atomic.incrementAndGet();
  
 -        finished.set(true);
 -        fut.get();
 +            finished.set(true);
 +
 +            fut.get();
 +        }
 +        finally {
 +            finished.set(true);
 +        }
      }
+ 
+     /** {@inheritDoc} */
+     public void testExplicitTransactionRetries() throws Exception {
+         final AtomicInteger idx = new AtomicInteger();
+         int threads = 8;
+ 
+         final AtomicReferenceArray<Exception> err = new AtomicReferenceArray<>(threads);
+ 
+         IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+             @Override
+             public Object call() throws Exception {
+                 int th = idx.getAndIncrement();
+                 int base = th * FACTOR;
+ 
+                 Ignite ignite = ignite(0);
+                 final IgniteCache<Object, Object> cache = ignite.cache(null);
+ 
+                 try {
+                     for (int i = 0; i < FACTOR; i++) {
+                         doInTransaction(ignite, new ProcessCallable(cache, base, i));
+ 
+                         if (i > 0 && i % 500 == 0)
+                             info("Done: " + i);
+                     }
+                 }
+                 catch (Exception e) {
+                     err.set(th, e);
+                 }
+ 
+                 return null;
+             }
+         }, threads, "tx-runner");
+ 
+         while (!fut.isDone()) {
+             int stopIdx = ThreadLocalRandom.current().nextInt(2, 4); // Random in [2, 3].
+ 
+             stopGrid(stopIdx);
+ 
+             U.sleep(500);
+ 
+             startGrid(stopIdx);
+         }
+ 
+         for (int i = 0; i < threads; i++) {
+             Exception error = err.get(i);
+ 
+             if (error != null)
+                 throw error;
+         }
+ 
+         // Verify contents of the cache.
+         for (int g = 0; g < gridCount(); g++) {
+             IgniteCache<Object, Object> cache = ignite(g).cache(null);
+ 
+             for (int th = 0; th < threads; th++) {
+                 int base = th * FACTOR;
+ 
+                 String key = "key-" + base;
+ 
+                 Set<String> set = (Set<String>)cache.get(key);
+ 
+                 assertNotNull("Missing set for key: " + key, set);
+                 assertEquals(FACTOR, set.size());
+ 
+                 for (int i = 0; i < FACTOR; i++) {
+                     assertEquals("value-" + i, cache.get("key-" + base + "-" + i));
+                     assertTrue(set.contains("value-" + i));
+                 }
+             }
+         }
+     }
+ 
+     /**
+      * @param ignite Ignite instance.
+      * @param clo Closure.
+      * @return Result of closure execution.
+      * @throws Exception If failed.
+      */
+     private <T> T doInTransaction(Ignite ignite, Callable<T> clo) throws Exception {
+         while (true) {
+             try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                 T res = clo.call();
+ 
+                 tx.commit();
+ 
+                 return res;
+             }
+             catch (CacheException e) {
+                 if (e.getCause() instanceof ClusterTopologyException) {
+                     ClusterTopologyException topEx = (ClusterTopologyException)e.getCause();
+ 
+                     topEx.retryReadyFuture().get();
+                 }
+                 else
+                     throw e;
+             }
+             catch (ClusterTopologyException e) {
+                 IgniteFuture<?> fut = e.retryReadyFuture();
+ 
+                 fut.get();
+             }
+             catch (TransactionRollbackException ignore) {
+                 // Safe to retry right away.
+             }
+         }
+     }
+ 
+     /**
+      * Callable to process inside transaction.
+      */
+     private static class ProcessCallable implements Callable<Void> {
+         /** */
+         private IgniteCache cache;
+ 
+         /** */
+         private int base;
+ 
+         /** */
+         private int i;
+ 
+         /**
+          * @param cache Cache.
+          * @param base Base index.
+          * @param i Iteration index.
+          */
+         private ProcessCallable(IgniteCache<Object, Object> cache, int base, int i) {
+             this.cache = cache;
+             this.base = base;
+             this.i = i;
+         }
+ 
+         /** {@inheritDoc} */
+         @SuppressWarnings("unchecked")
+         @Override public Void call() throws Exception {
+             String key1 = "key-" + base + "-" + i;
+             String key2 = "key-" + base;
+ 
+             assert key1.compareTo(key2) > 0;
+ 
+             ((IgniteCache<String, String>)cache).put(key1, "value-" + i);
+ 
+             ((IgniteCache<String, Set<String>>)cache).invoke(key2, new AddEntryProcessor("value-" + i));
+ 
+             return null;
+         }
+     }
+ 
+     /**
+      *
+      */
+     private static class AddEntryProcessor implements CacheEntryProcessor<String, Set<String>, Void> {
+         /** */
+         private String addVal;
+ 
+         /**
+          * @param addVal Value to add.
+          */
+         private AddEntryProcessor(String addVal) {
+             this.addVal = addVal;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public Void process(MutableEntry<String, Set<String>> entry, Object... arguments) throws EntryProcessorException {
+             Set<String> set = entry.getValue();
+ 
+             if (set == null)
+                 set = new HashSet<>();
+ 
+             set.add(addVal);
+ 
+             entry.setValue(set);
+ 
+             return null;
+         }
+     }
  }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d8595879/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 2efdb82,495719f..cb17501
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@@ -138,8 -138,12 +138,12 @@@ public class IgniteCacheTestSuite2 exte
          suite.addTest(new TestSuite(IgniteCachePartitionMapUpdateTest.class));
          suite.addTest(new TestSuite(IgniteCacheClientNodePartitionsExchangeTest.class));
          suite.addTest(new TestSuite(IgniteCacheClientNodeChangingTopologyTest.class));
 -        suite.addTest(new TestSuite(IgniteCacheClientNodeConcurrentStart.class));
 +        suite.addTest(new TestSuite(IgniteCacheServerNodeConcurrentStart.class));
  
+         suite.addTest(new TestSuite(IgniteCacheEntryProcessorNodeJoinTest.class));
+         suite.addTest(new TestSuite(IgniteAtomicCacheEntryProcessorNodeJoinTest.class));
+         suite.addTest(new TestSuite(GridCacheNearTxForceKeyTest.class));
+ 
          return suite;
      }
  }


[30/34] incubator-ignite git commit: IGNITE-1275 - Use topology-safe method in marshaller context.

Posted by sb...@apache.org.
IGNITE-1275 - Use topology-safe method in marshaller context.


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

Branch: refs/heads/master
Commit: d31a8df01fbdf2c3e32663ba57cb35d2ef7797d9
Parents: 94e95ea 6b93ee7
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Aug 20 18:31:48 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Aug 20 18:31:48 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/MarshallerContextImpl.java  |   2 +-
 .../processors/cache/GridCacheAdapter.java      |  82 +++++--
 .../distributed/dht/GridDhtCacheAdapter.java    |  12 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |  12 +-
 .../dht/GridPartitionedGetFuture.java           |  86 +++++---
 .../dht/atomic/GridDhtAtomicCache.java          |  16 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  19 +-
 .../distributed/near/GridNearAtomicCache.java   |   6 +-
 .../distributed/near/GridNearCacheAdapter.java  |  15 +-
 .../distributed/near/GridNearCacheEntry.java    |   4 +-
 .../distributed/near/GridNearGetFuture.java     | 101 ++++++---
 .../near/GridNearTransactionalCache.java        |   9 +-
 .../cache/distributed/near/GridNearTxLocal.java |   7 +-
 .../local/atomic/GridLocalAtomicCache.java      |  17 +-
 .../IgniteCacheTopologySafeGetSelfTest.java     | 215 +++++++++++++++++++
 ...gniteCachePutRetryTransactionalSelfTest.java |   2 +
 .../IgniteCacheFailoverTestSuite.java           |   2 +
 17 files changed, 494 insertions(+), 113 deletions(-)
----------------------------------------------------------------------



[02/34] incubator-ignite git commit: IGNITE-1265 - Rebuilt messages.

Posted by sb...@apache.org.
IGNITE-1265 - Rebuilt messages.


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

Branch: refs/heads/master
Commit: 574c6793adb96caa614b9c0c540f4812cfc52ee5
Parents: 7a43dde
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Aug 18 11:11:43 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Aug 18 11:11:43 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheUtils.java        | 11 +++++---
 .../distributed/dht/GridDhtTxPrepareFuture.java |  6 ++--
 .../dht/GridDhtTxPrepareResponse.java           | 29 +++++++++++++++-----
 3 files changed, 32 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/574c6793/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 1e3cd67..80e0d69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1684,15 +1684,18 @@ public class GridCacheUtils {
      * @param partsMap Cache ID to partition IDs collection map.
      * @return Cache ID to partition ID array map.
      */
-    public static Map<Integer, Integer[]> convertInvalidPartitions(Map<Integer, Set<Integer>> partsMap) {
-        Map<Integer, Integer[]> res = new HashMap<>(partsMap.size());
+    public static Map<Integer, int[]> convertInvalidPartitions(Map<Integer, Set<Integer>> partsMap) {
+        Map<Integer, int[]> res = new HashMap<>(partsMap.size());
 
         for (Map.Entry<Integer, Set<Integer>> entry : partsMap.entrySet()) {
             Set<Integer> parts = entry.getValue();
 
-            Integer[] partsArray = new Integer[parts.size()];
+            int[] partsArray = new int[parts.size()];
 
-            partsArray = parts.toArray(partsArray);
+            int idx = 0;
+
+            for (Integer part : parts)
+                partsArray[idx++] = part;
 
             res.put(entry.getKey(), partsArray);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/574c6793/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 27de8cf..2b7e1bc 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
@@ -1212,14 +1212,14 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
                 // Process invalid partitions (no need to remap).
                 if (!F.isEmpty(res.invalidPartitionsByCacheId())) {
-                    Map<Integer, Integer[]> invalidPartsMap = res.invalidPartitionsByCacheId();
+                    Map<Integer, int[]> invalidPartsMap = res.invalidPartitionsByCacheId();
 
                     for (Iterator<IgniteTxEntry> it = dhtMapping.entries().iterator(); it.hasNext();) {
                         IgniteTxEntry entry  = it.next();
 
-                        Integer[] invalidParts = invalidPartsMap.get(entry.cacheId());
+                        int[] invalidParts = invalidPartsMap.get(entry.cacheId());
 
-                        if (F.contains(invalidParts, entry.cached().partition())) {
+                        if (invalidParts != null && F.contains(invalidParts, entry.cached().partition())) {
                             it.remove();
 
                             if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/574c6793/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
index cc85628..bcf7f8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java
@@ -56,7 +56,8 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
     private Collection<Integer> invalidParts;
 
     /** Invalid partitions by cache ID. */
-    private Map<Integer, Integer[]> invalidPartsByCacheId;
+    @GridDirectMap(keyType = Integer.class, valueType = int[].class)
+    private Map<Integer, int[]> invalidPartsByCacheId;
 
     /** Preload entries. */
     @GridDirectCollection(GridCacheEntryInfo.class)
@@ -145,7 +146,7 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
     /**
      * @return Map from cacheId to an array of invalid partitions.
      */
-    public Map<Integer, Integer[]> invalidPartitionsByCacheId() {
+    public Map<Integer, int[]> invalidPartitionsByCacheId() {
         return invalidPartsByCacheId;
     }
 
@@ -255,18 +256,24 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
                 writer.incrementState();
 
             case 10:
-                if (!writer.writeIgniteUuid("miniId", miniId))
+                if (!writer.writeMap("invalidPartsByCacheId", invalidPartsByCacheId, MessageCollectionItemType.INT, MessageCollectionItemType.INT_ARR))
                     return false;
 
                 writer.incrementState();
 
             case 11:
-                if (!writer.writeCollection("nearEvicted", nearEvicted, MessageCollectionItemType.MSG))
+                if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
             case 12:
+                if (!writer.writeCollection("nearEvicted", nearEvicted, MessageCollectionItemType.MSG))
+                    return false;
+
+                writer.incrementState();
+
+            case 13:
                 if (!writer.writeCollection("preloadEntries", preloadEntries, MessageCollectionItemType.MSG))
                     return false;
 
@@ -305,7 +312,7 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
                 reader.incrementState();
 
             case 10:
-                miniId = reader.readIgniteUuid("miniId");
+                invalidPartsByCacheId = reader.readMap("invalidPartsByCacheId", MessageCollectionItemType.INT, MessageCollectionItemType.INT_ARR, false);
 
                 if (!reader.isLastRead())
                     return false;
@@ -313,7 +320,7 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
                 reader.incrementState();
 
             case 11:
-                nearEvicted = reader.readCollection("nearEvicted", MessageCollectionItemType.MSG);
+                miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -321,6 +328,14 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
                 reader.incrementState();
 
             case 12:
+                nearEvicted = reader.readCollection("nearEvicted", MessageCollectionItemType.MSG);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 13:
                 preloadEntries = reader.readCollection("preloadEntries", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -340,6 +355,6 @@ public class GridDhtTxPrepareResponse extends GridDistributedTxPrepareResponse {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 13;
+        return 14;
     }
 }


[22/34] incubator-ignite git commit: IGNITE-1261: Implemented schema selection for schema import utility.

Posted by sb...@apache.org.
IGNITE-1261: Implemented schema selection for schema import utility.


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

Branch: refs/heads/master
Commit: e0f445eead0a1d14b2fe5a498cb0de57ecd18f1d
Parents: 2673eca
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Aug 20 17:43:40 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 20 17:43:40 2015 +0700

----------------------------------------------------------------------
 .../ignite/schema/model/PojoDescriptor.java     |   6 +-
 .../ignite/schema/model/SchemaDescriptor.java   |  61 +++++++
 .../schema/parser/DatabaseMetadataParser.java   |  54 +++++--
 .../parser/dialect/DB2MetadataDialect.java      |   3 +-
 .../parser/dialect/DatabaseMetadataDialect.java |  13 +-
 .../parser/dialect/JdbcMetadataDialect.java     | 129 ++++++++++-----
 .../parser/dialect/MySQLMetadataDialect.java    |  57 +++++++
 .../parser/dialect/OracleMetadataDialect.java   | 111 +++++++++----
 .../org/apache/ignite/schema/ui/Controls.java   |  25 ++-
 .../ignite/schema/ui/SchemaImportApp.java       | 160 ++++++++++++++++---
 .../schema/test/AbstractSchemaImportTest.java   |  10 +-
 11 files changed, 508 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
index 5a4a1fa..d35c934 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
@@ -493,6 +493,8 @@ public class PojoDescriptor {
             case NCHAR:
             case NVARCHAR:
             case LONGNVARCHAR:
+            case CLOB:
+            case NCLOB:
                 return String.class;
 
             case DATE:
@@ -504,10 +506,6 @@ public class PojoDescriptor {
             case TIMESTAMP:
                 return java.sql.Timestamp.class;
 
-            case CLOB:
-            case NCLOB:
-                return String.class;
-
             // BINARY, VARBINARY, LONGVARBINARY, ARRAY, BLOB, NULL, DATALINK
             // OTHER, JAVA_OBJECT, DISTINCT, STRUCT, REF, ROWID, SQLXML
             default:

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/model/SchemaDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/SchemaDescriptor.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/SchemaDescriptor.java
new file mode 100644
index 0000000..0ab3c57
--- /dev/null
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/SchemaDescriptor.java
@@ -0,0 +1,61 @@
+/*
+ * 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.schema.model;
+
+import javafx.beans.property.*;
+
+/**
+ * Descriptor for schema.
+ */
+public class SchemaDescriptor {
+    /** Schema name */
+    private final String schema;
+
+    /** State of schema selection. */
+    private final BooleanProperty selected;
+
+    /**
+     * Constructor of schema descriptor.
+     *
+     * @param schema Schema.
+     * @param selected Selection state.
+     */
+    public SchemaDescriptor(String schema, boolean selected) {
+        this.schema = schema;
+        this.selected = new SimpleBooleanProperty(selected);
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schema() {
+        return schema;
+    }
+
+    /**
+     * @return Boolean property support for {@code selected} property.
+     */
+    public BooleanProperty selected() {
+        return selected;
+    }
+
+    @Override
+    public String toString() {
+        return schema;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java
index 696ca62..9f4450f 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java
@@ -33,37 +33,67 @@ public class DatabaseMetadataParser {
     private static final Logger log = Logger.getLogger(DatabaseMetadataParser.class.getName());
 
     /**
-     * Parse database metadata.
+     * Get specified dialect object for selected database.
      *
      * @param conn Connection to database.
-     * @param tblsOnly If {@code true} then process tables only else process tables and views.
-     * @return Collection of POJO descriptors.
-     * @throws SQLException If parsing failed.
+     * @return Specific dialect object.
      */
-    public static ObservableList<PojoDescriptor> parse(Connection conn, boolean tblsOnly) throws SQLException {
-        DatabaseMetadataDialect dialect;
-
+    private static DatabaseMetadataDialect dialect(Connection conn) {
         try {
             String dbProductName = conn.getMetaData().getDatabaseProductName();
 
             if ("Oracle".equals(dbProductName))
-                dialect = new OracleMetadataDialect();
+                return new OracleMetadataDialect();
             else if (dbProductName.startsWith("DB2/"))
-                dialect = new DB2MetadataDialect();
+                return new DB2MetadataDialect();
+            else if (dbProductName.equals("MySQL"))
+                return new MySQLMetadataDialect();
             else
-                dialect = new JdbcMetadataDialect();
+                return new JdbcMetadataDialect();
         }
         catch (SQLException e) {
             log.log(Level.SEVERE, "Failed to resolve dialect (JdbcMetaDataDialect will be used.", e);
 
-            dialect = new JdbcMetadataDialect();
+            return new JdbcMetadataDialect();
         }
+    }
+
+    /**
+     * Get list of schemas from database.
+     *
+     * @param conn Connection to database.
+     * @return List of schema descriptors.
+     * @throws SQLException If shemas loading failed.
+     */
+    public static ObservableList<SchemaDescriptor> schemas(Connection conn) throws SQLException  {
+        List<String> dbSchemas = dialect(conn).schemas(conn);
+
+        List<SchemaDescriptor> uiSchemas = new ArrayList<>(dbSchemas.size());
+
+        for (String schema : dbSchemas)
+            uiSchemas.add(new SchemaDescriptor(schema, false));
+
+        return FXCollections.observableList(uiSchemas);
+    }
+
+    /**
+     * Parse database metadata.
+     *
+     * @param conn Connection to database.
+     * @param schemas Collention of schema names to load.
+     * @param tblsOnly If {@code true} then process tables only else process tables and views.
+     * @return Collection of POJO descriptors.
+     * @throws SQLException If parsing failed.
+     */
+    public static ObservableList<PojoDescriptor> parse(Connection conn, List<String> schemas, boolean tblsOnly)
+        throws SQLException {
+        DatabaseMetadataDialect dialect = dialect(conn);
 
         Map<String, PojoDescriptor> parents = new HashMap<>();
 
         Map<String, Collection<PojoDescriptor>> childrens = new HashMap<>();
 
-        for (DbTable tbl : dialect.tables(conn, tblsOnly)) {
+        for (DbTable tbl : dialect.tables(conn, schemas, tblsOnly)) {
             String schema = tbl.schema();
 
             PojoDescriptor parent = parents.get(schema);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
index 17eb8b2..15063e2 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
@@ -25,6 +25,7 @@ import java.util.*;
 public class DB2MetadataDialect extends JdbcMetadataDialect {
     /** {@inheritDoc} */
     @Override public Set<String> systemSchemas() {
-        return new HashSet<>(Arrays.asList("SYSIBM", "SYSCAT", "SYSSTAT", "SYSTOOLS"));
+        return new HashSet<>(Arrays.asList("SYSIBM", "SYSCAT", "SYSSTAT", "SYSTOOLS", "SYSFUN", "SYSIBMADM",
+            "SYSIBMINTERNAL", "SYSIBMTS", "SYSPROC", "SYSPUBLIC"));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
index 0d17567..9c059b8 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
@@ -27,14 +27,25 @@ import java.util.*;
  */
 public abstract class DatabaseMetadataDialect {
     /**
+     * Gets schemas from database.
+     *
+     * @param conn Database connection.
+     * @return Collection of schema descriptors.
+     * @throws SQLException If failed to get schemas.
+     */
+    public abstract List<String> schemas(Connection conn) throws SQLException;
+
+    /**
      * Gets tables from database.
      *
      * @param conn Database connection.
+     * @param schemas Collention of schema names to load.
      * @param tblsOnly If {@code true} then gets only tables otherwise gets tables and views.
      * @return Collection of table descriptors.
      * @throws SQLException If failed to get tables.
      */
-    public abstract Collection<DbTable> tables(Connection conn, boolean tblsOnly) throws SQLException;
+    public abstract Collection<DbTable> tables(Connection conn, List<String> schemas, boolean tblsOnly)
+        throws SQLException;
 
     /**
      * @return Collection of database system schemas.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
index ab65e7a..1bb6840 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
@@ -63,76 +63,117 @@ public class JdbcMetadataDialect extends DatabaseMetadataDialect {
     private static final int IDX_ASC_OR_DESC_IDX = 10;
 
     /** {@inheritDoc} */
-    @Override public Collection<DbTable> tables(Connection conn, boolean tblsOnly) throws SQLException {
+    @Override public List<String> schemas(Connection conn) throws SQLException {
+        List<String> schemas = new ArrayList<>();
+
+        ResultSet rs = conn.getMetaData().getSchemas();
+
+        Set<String> sys = systemSchemas();
+
+        while(rs.next()) {
+            String schema = rs.getString(1);
+
+            // Skip system schemas.
+            if (sys.contains(schema))
+                continue;
+
+            schemas.add(schema);
+        }
+
+        return schemas;
+    }
+
+    /**
+     * @return If {@code true} use catalogs for table division.
+     */
+    protected boolean useCatalog() {
+        return false;
+    }
+
+    /**
+     * @return If {@code true} use schemas for table division.
+     */
+    protected boolean useSchema() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<DbTable> tables(Connection conn, List<String> schemas, boolean tblsOnly)
+        throws SQLException {
         DatabaseMetaData dbMeta = conn.getMetaData();
 
         Set<String> sys = systemSchemas();
 
         Collection<DbTable> tbls = new ArrayList<>();
 
-        try (ResultSet tblsRs = dbMeta.getTables(null, null, "%",
-            tblsOnly ? TABLES_ONLY : TABLES_AND_VIEWS)) {
-            while (tblsRs.next()) {
-                String tblCatalog = tblsRs.getString(TBL_CATALOG_IDX);
-                String tblSchema = tblsRs.getString(TBL_SCHEMA_IDX);
-                String tblName = tblsRs.getString(TBL_NAME_IDX);
+        if (schemas.size() == 0)
+            schemas.add(null);
 
-                // In case of MySql we should use catalog.
-                String schema = tblSchema != null ? tblSchema : tblCatalog;
+        for (String toSchema: schemas) {
+            try (ResultSet tblsRs = dbMeta.getTables(useCatalog() ? toSchema : null, useSchema() ? toSchema : null, "%",
+                    tblsOnly ? TABLES_ONLY : TABLES_AND_VIEWS)) {
+                while (tblsRs.next()) {
+                    String tblCatalog = tblsRs.getString(TBL_CATALOG_IDX);
+                    String tblSchema = tblsRs.getString(TBL_SCHEMA_IDX);
+                    String tblName = tblsRs.getString(TBL_NAME_IDX);
 
-                // Skip system schemas.
-                if (sys.contains(schema))
-                    continue;
+                    // In case of MySql we should use catalog.
+                    String schema = tblSchema != null ? tblSchema : tblCatalog;
 
-                Set<String> pkCols = new HashSet<>();
+                    // Skip system schemas.
+                    if (sys.contains(schema))
+                        continue;
 
-                try (ResultSet pkRs = dbMeta.getPrimaryKeys(tblCatalog, tblSchema, tblName)) {
-                    while (pkRs.next())
-                        pkCols.add(pkRs.getString(PK_COL_NAME_IDX));
-                }
+                    Set<String> pkCols = new HashSet<>();
+
+                    try (ResultSet pkRs = dbMeta.getPrimaryKeys(tblCatalog, tblSchema, tblName)) {
+                        while (pkRs.next())
+                            pkCols.add(pkRs.getString(PK_COL_NAME_IDX));
+                    }
 
-                List<DbColumn> cols = new ArrayList<>();
+                    List<DbColumn> cols = new ArrayList<>();
 
-                try (ResultSet colsRs = dbMeta.getColumns(tblCatalog, tblSchema, tblName, null)) {
-                    while (colsRs.next()) {
-                        String colName = colsRs.getString(COL_NAME_IDX);
+                    try (ResultSet colsRs = dbMeta.getColumns(tblCatalog, tblSchema, tblName, null)) {
+                        while (colsRs.next()) {
+                            String colName = colsRs.getString(COL_NAME_IDX);
 
-                        cols.add(new DbColumn(
-                            colName,
-                            colsRs.getInt(COL_DATA_TYPE_IDX),
-                            pkCols.contains(colName),
-                            colsRs.getInt(COL_NULLABLE_IDX) == DatabaseMetaData.columnNullable));
+                            cols.add(new DbColumn(
+                                    colName,
+                                    colsRs.getInt(COL_DATA_TYPE_IDX),
+                                    pkCols.contains(colName),
+                                    colsRs.getInt(COL_NULLABLE_IDX) == DatabaseMetaData.columnNullable));
+                        }
                     }
-                }
 
-                Map<String, Map<String, Boolean>> idxs = new LinkedHashMap<>();
+                    Map<String, Map<String, Boolean>> idxs = new LinkedHashMap<>();
 
-                try (ResultSet idxRs = dbMeta.getIndexInfo(tblCatalog, tblSchema, tblName, false, true)) {
-                    while (idxRs.next()) {
-                        String idxName = idxRs.getString(IDX_NAME_IDX);
+                    try (ResultSet idxRs = dbMeta.getIndexInfo(tblCatalog, tblSchema, tblName, false, true)) {
+                        while (idxRs.next()) {
+                            String idxName = idxRs.getString(IDX_NAME_IDX);
 
-                        String colName = idxRs.getString(IDX_COL_NAME_IDX);
+                            String colName = idxRs.getString(IDX_COL_NAME_IDX);
 
-                        if (idxName == null || colName == null)
-                            continue;
+                            if (idxName == null || colName == null)
+                                continue;
 
-                        Map<String, Boolean> idx = idxs.get(idxName);
+                            Map<String, Boolean> idx = idxs.get(idxName);
 
-                        if (idx == null) {
-                            idx = new LinkedHashMap<>();
+                            if (idx == null) {
+                                idx = new LinkedHashMap<>();
 
-                            idxs.put(idxName, idx);
-                        }
+                                idxs.put(idxName, idx);
+                            }
 
-                        String askOrDesc = idxRs.getString(IDX_ASC_OR_DESC_IDX);
+                            String askOrDesc = idxRs.getString(IDX_ASC_OR_DESC_IDX);
 
-                        Boolean desc = askOrDesc != null ? "D".equals(askOrDesc) : null;
+                            Boolean desc = askOrDesc != null ? "D".equals(askOrDesc) : null;
 
-                        idx.put(colName, desc);
+                            idx.put(colName, desc);
+                        }
                     }
-                }
 
-                tbls.add(table(schema, tblName, cols, idxs));
+                    tbls.add(table(schema, tblName, cols, idxs));
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java
new file mode 100644
index 0000000..b592321
--- /dev/null
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java
@@ -0,0 +1,57 @@
+/*
+ * 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.schema.parser.dialect;
+
+import java.sql.*;
+import java.util.*;
+
+/**
+ * MySQL specific metadata dialect.
+ */
+public class MySQLMetadataDialect extends JdbcMetadataDialect {
+    /** {@inheritDoc} */
+    @Override public List<String> schemas(Connection conn) throws SQLException {
+        List<String> schemas = new ArrayList<>();
+
+        ResultSet rs = conn.getMetaData().getCatalogs();
+
+        Set<String> sys = systemSchemas();
+
+        while(rs.next()) {
+            String schema = rs.getString(1);
+
+            // Skip system schemas.
+            if (sys.contains(schema))
+                continue;
+
+            schemas.add(schema);
+        }
+
+        return schemas;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean useCatalog() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean useSchema() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
index 30dda5d..c569a29 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
@@ -31,8 +31,8 @@ public class OracleMetadataDialect extends DatabaseMetadataDialect {
     /** SQL to get columns metadata. */
     private static final String SQL_COLUMNS = "SELECT a.owner, a.table_name, a.column_name, a.nullable," +
         " a.data_type, a.data_precision, a.data_scale " +
-        "FROM all_tab_columns a %s" +
-        " WHERE a.owner = '%s'" +
+        "FROM all_tab_columns a %s " +
+        " %s " +
         " ORDER BY a.owner, a.table_name, a.column_id";
 
     /** SQL to get list of PRIMARY KEYS columns. */
@@ -81,6 +81,36 @@ public class OracleMetadataDialect extends DatabaseMetadataDialect {
     /** Index column sort order index. */
     private static final int IDX_COL_DESCEND_IDX = 4;
 
+    /** {@inheritDoc} */
+    @Override public Set<String> systemSchemas() {
+        return new HashSet<>(Arrays.asList("ANONYMOUS", "CTXSYS", "DBSNMP", "EXFSYS", "LBACSYS", "MDSYS", "MGMT_VIEW",
+            "OLAPSYS", "OWBSYS", "ORDPLUGINS", "ORDSYS", "OUTLN", "SI_INFORMTN_SCHEMA", "SYS", "SYSMAN", "SYSTEM",
+            "TSMSYS", "WK_TEST", "WKSYS", "WKPROXY", "WMSYS", "XDB",
+
+            "APEX_040000", "APEX_PUBLIC_USER", "DIP", "FLOWS_30000", "FLOWS_FILES", "MDDATA", "ORACLE_OCM",
+            "SPATIAL_CSW_ADMIN_USR", "SPATIAL_WFS_ADMIN_USR", "XS$NULL",
+
+            "BI", "HR", "OE", "PM", "IX", "SH"));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<String> schemas(Connection conn) throws SQLException {
+        List<String> schemas = new ArrayList<>();
+
+        ResultSet rs = conn.getMetaData().getSchemas();
+
+        Set<String> sysSchemas = systemSchemas();
+
+        while(rs.next()) {
+            String schema = rs.getString(1);
+
+            if (!sysSchemas.contains(schema) && !schema.startsWith("FLOWS_"))
+                schemas.add(schema);
+        }
+
+        return schemas;
+    }
+
     /**
      * @param rs Result set with column type metadata from Oracle database.
      * @return JDBC type.
@@ -225,57 +255,70 @@ public class OracleMetadataDialect extends DatabaseMetadataDialect {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<DbTable> tables(Connection conn, boolean tblsOnly) throws SQLException {
+    @Override public Collection<DbTable> tables(Connection conn, List<String> schemas, boolean tblsOnly)
+        throws SQLException {
         Collection<DbTable> tbls = new ArrayList<>();
 
         PreparedStatement pkStmt = conn.prepareStatement(SQL_PRIMARY_KEYS);
 
         PreparedStatement idxStmt = conn.prepareStatement(SQL_INDEXES);
 
+        if (schemas.size() == 0)
+            schemas.add(null);
+
+        Set<String> sysSchemas = systemSchemas();
+
         try (Statement colsStmt = conn.createStatement()) {
-            Collection<DbColumn> cols = new ArrayList<>();
+            for (String schema: schemas) {
+                if (systemSchemas().contains(schema) || (schema != null && schema.startsWith("FLOWS_")))
+                    continue;
 
-            Set<String> pkCols = Collections.emptySet();
-            Map<String, Map<String, Boolean>> idxs = Collections.emptyMap();
+                Collection<DbColumn> cols = new ArrayList<>();
 
-            String user = conn.getMetaData().getUserName().toUpperCase();
+                Set<String> pkCols = Collections.emptySet();
+                Map<String, Map<String, Boolean>> idxs = Collections.emptyMap();
 
-            String sql = String.format(SQL_COLUMNS,
-                tblsOnly ? "INNER JOIN all_tables b on a.table_name = b.table_name and a.owner = b.owner" : "", user);
+                String sql = String.format(SQL_COLUMNS,
+                        tblsOnly ? "INNER JOIN all_tables b on a.table_name = b.table_name and a.owner = b.owner" : "",
+                        schema != null ? String.format(" WHERE a.owner = '%s' ", schema) : "");
 
-            try (ResultSet colsRs = colsStmt.executeQuery(sql)) {
-                String prevSchema = "";
-                String prevTbl = "";
+                try (ResultSet colsRs = colsStmt.executeQuery(sql)) {
+                    String prevSchema = "";
+                    String prevTbl = "";
 
-                boolean first = true;
+                    boolean first = true;
 
-                while (colsRs.next()) {
-                    String owner = colsRs.getString(OWNER_IDX);
-                    String tbl = colsRs.getString(TBL_NAME_IDX);
+                    while (colsRs.next()) {
+                        String owner = colsRs.getString(OWNER_IDX);
+                        String tbl = colsRs.getString(TBL_NAME_IDX);
 
-                    boolean changed = !owner.equals(prevSchema) || !tbl.equals(prevTbl);
+                        if (sysSchemas.contains(owner) || (schema != null && schema.startsWith("FLOWS_")))
+                            continue;
 
-                    if (changed) {
-                        if (first)
-                            first = false;
-                        else
-                            tbls.add(table(prevSchema, prevTbl, cols, idxs));
+                        boolean changed = !owner.equals(prevSchema) || !tbl.equals(prevTbl);
 
-                        prevSchema = owner;
-                        prevTbl = tbl;
-                        cols = new ArrayList<>();
-                        pkCols = primaryKeys(pkStmt, owner, tbl);
-                        idxs = indexes(idxStmt, owner, tbl);
-                    }
+                        if (changed) {
+                            if (first)
+                                first = false;
+                            else
+                                tbls.add(table(prevSchema, prevTbl, cols, idxs));
 
-                    String colName = colsRs.getString(COL_NAME_IDX);
+                            prevSchema = owner;
+                            prevTbl = tbl;
+                            cols = new ArrayList<>();
+                            pkCols = primaryKeys(pkStmt, owner, tbl);
+                            idxs = indexes(idxStmt, owner, tbl);
+                        }
 
-                    cols.add(new DbColumn(colName, decodeType(colsRs), pkCols.contains(colName),
-                        !"N".equals(colsRs.getString(NULLABLE_IDX))));
-                }
+                        String colName = colsRs.getString(COL_NAME_IDX);
 
-                if (!cols.isEmpty())
-                    tbls.add(table(prevSchema, prevTbl, cols, idxs));
+                        cols.add(new DbColumn(colName, decodeType(colsRs), pkCols.contains(colName),
+                                !"N".equals(colsRs.getString(NULLABLE_IDX))));
+                    }
+
+                    if (!cols.isEmpty())
+                        tbls.add(table(prevSchema, prevTbl, cols, idxs));
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/Controls.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/Controls.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/Controls.java
index 6c4d6bd..794fe6f 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/Controls.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/Controls.java
@@ -350,17 +350,40 @@ public class Controls {
      *
      * @param title Title.
      * @param node Node.
+     * @param collapsible Collapsible flag.
      * @return New {@code TitledPane} instance.
      */
-    public static TitledPane titledPane(String title, Node node) {
+    public static TitledPane titledPane(String title, Node node, boolean collapsible) {
         TitledPane tp = new TitledPane(title, node);
 
+        tp.setCollapsible(collapsible);
         tp.setExpanded(false);
 
         return tp;
     }
 
     /**
+     * Create list view.
+     *
+     * @param tip Tooltip text.
+     * @param cb Callback function for list view cell data binding.
+     * @param <T> Type of showed by viewer element.
+     * @return New {@code ListView} instance.
+     */
+    public static <T> ListView<T> list(String tip, Callback<T, ObservableValue<Boolean>> cb) {
+        ListView lst = new ListView<>();
+
+        lst.setCellFactory(CheckBoxListCell.forListView(cb));
+
+        lst.setMinHeight(70);
+        lst.getSelectionModel().setSelectionMode(SelectionMode.MULTIPLE);
+
+        tooltip(lst, tip);
+
+        return lst;
+    }
+
+    /**
      * Create table column.
      *
      * @param colName Column name to display.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
index 7b9c220..aab36b8 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
@@ -201,6 +201,9 @@ public class SchemaImportApp extends Application {
     private ComboBox<String> parseCb;
 
     /** */
+    private ListView<SchemaDescriptor> schemaLst;
+
+    /** */
     private GridPaneEx connPnl;
 
     /** */
@@ -245,6 +248,8 @@ public class SchemaImportApp extends Application {
     /** */
     private ProgressIndicator pi;
 
+    private ObservableList<SchemaDescriptor> schemas = FXCollections.emptyObservableList();
+
     /** List with POJOs descriptors. */
     private ObservableList<PojoDescriptor> pojos = FXCollections.emptyObservableList();
 
@@ -318,11 +323,12 @@ public class SchemaImportApp extends Application {
     }
 
     /**
-     * Fill tree with database metadata.
+     * Open connection to database.
+     *
+     * @return Connection to database.
+     * @throws SQLException If connection failed.
      */
-    private void fill() {
-        lockUI(connLayerPnl, connPnl, nextBtn);
-
+    private Connection connect() throws SQLException {
         final String jdbcDrvJarPath = jdbcDrvJarTf.getText().trim();
 
         final String jdbcDrvCls = jdbcDrvClsTf.getText();
@@ -341,6 +347,27 @@ public class SchemaImportApp extends Application {
         if (!pwd.isEmpty())
             jdbcInfo.put("password", pwd);
 
+        return connect(jdbcDrvJarPath, jdbcDrvCls, jdbcUrl, jdbcInfo);
+    }
+
+    /**
+     * Fill tree with database metadata.
+     */
+    private void fill() {
+        final List<String> selSchemas = new ArrayList<>();
+
+        for (SchemaDescriptor schema: schemas)
+            if (schema.selected().getValue())
+                selSchemas.add(schema.schema());
+
+        if (selSchemas.size() == 0)
+            if (!MessageBox.confirmDialog(owner, "No schemas selected.\nExtract tables for all available schemas?"))
+                return;
+
+        lockUI(connLayerPnl, connPnl, nextBtn);
+
+        final String jdbcUrl = jdbcUrlTf.getText();
+
         final boolean tblsOnly = parseCb.getSelectionModel().getSelectedIndex() == 0;
 
         Runnable task = new Task<Void>() {
@@ -348,8 +375,8 @@ public class SchemaImportApp extends Application {
             @Override protected Void call() throws Exception {
                 long started = System.currentTimeMillis();
 
-                try (Connection conn = connect(jdbcDrvJarPath, jdbcDrvCls, jdbcUrl, jdbcInfo)) {
-                    pojos = DatabaseMetadataParser.parse(conn, tblsOnly);
+                try (Connection conn = connect()) {
+                    pojos = DatabaseMetadataParser.parse(conn, selSchemas, tblsOnly);
                 }
 
                 perceptualDelay(started);
@@ -377,7 +404,6 @@ public class SchemaImportApp extends Application {
 
                     pojosTbl.requestFocus();
 
-
                     hdrPane.setLeft(genIcon);
 
                     titleLb.setText("Generate XML And POJOs");
@@ -415,6 +441,69 @@ public class SchemaImportApp extends Application {
     }
 
     /**
+     * Load schemas list from database.
+     */
+    private void loadSchemas() {
+        lockUI(connLayerPnl, connPnl, nextBtn);
+
+        final String jdbcUrl = jdbcUrlTf.getText();
+
+        Runnable task = new Task<Void>() {
+            /** {@inheritDoc} */
+            @Override protected Void call() throws Exception {
+                long started = System.currentTimeMillis();
+
+                try (Connection conn = connect()) {
+                    schemas = DatabaseMetadataParser.schemas(conn);
+                }
+
+                perceptualDelay(started);
+
+                return null;
+            }
+
+            /** {@inheritDoc} */
+            @Override protected void succeeded() {
+                try {
+                    super.succeeded();
+
+                    schemaLst.setItems(schemas);
+
+                    if (schemas.isEmpty()) {
+                        MessageBox.warningDialog(owner, "No schemas found in database. Recheck JDBC URL.\n" +
+                            "JDBC URL: " +  jdbcUrl);
+
+                        return;
+                    }
+
+                    nextBtn.setDisable(false);
+                }
+                finally {
+                    unlockUI(connLayerPnl, connPnl, nextBtn);
+                }
+            }
+
+            /** {@inheritDoc} */
+            @Override protected void cancelled() {
+                super.cancelled();
+
+                unlockUI(connLayerPnl, connPnl, nextBtn);
+            }
+
+            /** {@inheritDoc} */
+            @Override protected void failed() {
+                super.succeeded();
+
+                unlockUI(connLayerPnl, connPnl, nextBtn);
+
+                MessageBox.errorDialog(owner, "Failed to get schemas list from database.", getException());
+            }
+        };
+
+        exec.submit(task);
+    }
+
+    /**
      * Generate XML and POJOs.
      */
     private void generate() {
@@ -426,7 +515,7 @@ public class SchemaImportApp extends Application {
             return;
         }
 
-        if (checkInput(outFolderTf, true, "Output folder should not be empty!"))
+        if (!checkInput(outFolderTf, true, "Output folder should not be empty!"))
             return;
 
         lockUI(genLayerPnl, genPnl, prevBtn, nextBtn);
@@ -629,25 +718,22 @@ public class SchemaImportApp extends Application {
 
             MessageBox.warningDialog(owner, msg);
 
-            return true;
+            return false;
         }
 
-        return false;
+        return true;
     }
 
     /**
      * Go to &quot;Generate XML And POJOs&quot; panel or generate XML and POJOs.
      */
     private void next() {
-        if (rootPane.getCenter() == connLayerPnl) {
-            if (checkInput(jdbcDrvJarTf, true, "Path to JDBC driver is not specified!") ||
-                checkInput(jdbcDrvClsTf, true, "JDBC driver class name is not specified!") ||
-                checkInput(jdbcUrlTf, true, "JDBC URL connection string is not specified!") ||
+        if (rootPane.getCenter() == connLayerPnl)
+            if (checkInput(jdbcDrvJarTf, true, "Path to JDBC driver is not specified!") &&
+                checkInput(jdbcDrvClsTf, true, "JDBC driver class name is not specified!") &&
+                checkInput(jdbcUrlTf, true, "JDBC URL connection string is not specified!") &&
                 checkInput(userTf, true, "User name is not specified!"))
-                return;
-
-            fill();
-        }
+                fill();
         else
             generate();
     }
@@ -709,6 +795,9 @@ public class SchemaImportApp extends Application {
         connPnl.addColumn(100, 100, Double.MAX_VALUE, Priority.ALWAYS);
         connPnl.addColumn(35, 35, 35, Priority.NEVER);
 
+        connPnl.addRows(9);
+        connPnl.addRow(100, 100, Double.MAX_VALUE, Priority.ALWAYS);
+
         connPnl.add(text("This utility is designed to automatically generate configuration XML files and" +
             " POJO classes from database schema information.", 550), 3);
 
@@ -780,6 +869,27 @@ public class SchemaImportApp extends Application {
 
         parseCb = connPnl.addLabeled("Parse:", comboBox("Type of tables to parse", "Tables only", "Tables and Views"), 2);
 
+        GridPaneEx schemaPnl = paneEx(5, 5, 5, 5);
+        schemaPnl.addColumn(100, 100, Double.MAX_VALUE, Priority.ALWAYS);
+        schemaPnl.addColumn();
+
+        schemaLst = schemaPnl.add(list("Select schemas to load", new SchemaCell()));
+
+        schemaPnl.wrap();
+
+        schemaPnl.add(button("Load schemas", "Load schemas for specified database", new EventHandler<ActionEvent>() {
+            @Override
+            public void handle(ActionEvent evt) {
+                loadSchemas();
+            }
+        }));
+
+        TitledPane titledPnl = connPnl.add(titledPane("Schemas", schemaPnl, false), 3);
+
+        titledPnl.setExpanded(true);
+
+        GridPaneEx.setValignment(titledPnl, VPos.TOP);
+
         connLayerPnl = stackPane(connPnl);
 
         return connLayerPnl;
@@ -987,7 +1097,7 @@ public class SchemaImportApp extends Application {
                     " with the given replacement",
                 new EventHandler<ActionEvent>() {
                     @Override public void handle(ActionEvent evt) {
-                        if (checkInput(regexTf, false, "Regular expression should not be empty!"))
+                        if (!checkInput(regexTf, false, "Regular expression should not be empty!"))
                             return;
 
                         String sel = replaceCb.getSelectionModel().getSelectedItem();
@@ -1137,7 +1247,7 @@ public class SchemaImportApp extends Application {
         });
 
         genPnl.add(titledPane("Rename \"Key class name\", \"Value class name\" or  \"Java name\" for selected tables",
-            regexPnl), 3);
+            regexPnl, true), 3);
 
         genLayerPnl = stackPane(genPnl);
     }
@@ -1648,6 +1758,16 @@ public class SchemaImportApp extends Application {
     }
 
     /**
+     * Special list view cell to select loaded schemas.
+     */
+    private static class SchemaCell implements Callback<SchemaDescriptor, ObservableValue<Boolean>> {
+        @Override
+        public ObservableValue<Boolean> call(SchemaDescriptor item) {
+            return item.selected();
+        }
+    }
+
+    /**
      * Special table cell to select schema or table.
      */
     private static class PojoDescriptorCell extends TableCell<PojoDescriptor, Boolean> {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java
index 0c3ecb1..b9b9c76 100644
--- a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java
+++ b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java
@@ -19,13 +19,13 @@ package org.apache.ignite.schema.test;
 
 import junit.framework.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.schema.model.PojoDescriptor;
-import org.apache.ignite.schema.parser.DatabaseMetadataParser;
+import org.apache.ignite.schema.model.*;
+import org.apache.ignite.schema.parser.*;
 import org.apache.ignite.schema.ui.*;
 
 import java.io.*;
 import java.sql.*;
-import java.util.List;
+import java.util.*;
 
 import static org.apache.ignite.schema.ui.MessageBox.Result.*;
 
@@ -93,7 +93,9 @@ public abstract class AbstractSchemaImportTest extends TestCase {
 
         U.closeQuiet(stmt);
 
-        pojos = DatabaseMetadataParser.parse(conn, false);
+        List<String> schemas = new ArrayList<>();
+
+        pojos = DatabaseMetadataParser.parse(conn, schemas, false);
 
         U.closeQuiet(conn);
     }



[27/34] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-1.3.3-p3' into ignite-1.3.3-p3

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


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

Branch: refs/heads/master
Commit: abbd308da5ce8ef5260b511e41e6b394e1877d8b
Parents: d78e8bc afadd36
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 20 14:40:05 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 20 14:40:05 2015 +0300

----------------------------------------------------------------------
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  20 +--
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |  24 ++-
 .../ignite/schema/model/PojoDescriptor.java     |   6 +-
 .../ignite/schema/model/SchemaDescriptor.java   |  61 +++++++
 .../schema/parser/DatabaseMetadataParser.java   |  54 +++++--
 .../parser/dialect/DB2MetadataDialect.java      |   3 +-
 .../parser/dialect/DatabaseMetadataDialect.java |  13 +-
 .../parser/dialect/JdbcMetadataDialect.java     | 129 +++++++++------
 .../parser/dialect/MySQLMetadataDialect.java    |  57 +++++++
 .../parser/dialect/OracleMetadataDialect.java   | 111 +++++++++----
 .../org/apache/ignite/schema/ui/Controls.java   |  25 ++-
 .../ignite/schema/ui/SchemaImportApp.java       | 157 ++++++++++++++++---
 .../schema/test/AbstractSchemaImportTest.java   |  10 +-
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |   3 -
 14 files changed, 535 insertions(+), 138 deletions(-)
----------------------------------------------------------------------



[33/34] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p3'

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p3'


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

Branch: refs/heads/master
Commit: 1a033482cd656546af4a34f703efbbb10f6fe290
Parents: d859587
Author: sboikov <sb...@gridgain.com>
Authored: Fri Aug 21 11:11:36 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Aug 21 11:11:36 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheAdapter.java       | 10 ++++++++--
 .../cache/distributed/near/GridNearGetFuture.java         |  3 ++-
 .../cache/IgniteCacheTopologySafeGetSelfTest.java         |  7 +++++--
 3 files changed, 15 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a033482/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index f27f356..7adea2b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -1240,13 +1240,19 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             false,
             /*can remap*/true
         ).chain(new CX1<IgniteInternalFuture<Map<K, V>>, V>() {
-            @Override
-            public V applyx(IgniteInternalFuture<Map<K, V>> e) throws IgniteCheckedException {
+            @Override public V applyx(IgniteInternalFuture<Map<K, V>> e) throws IgniteCheckedException {
                 return e.get().get(key);
             }
         });
     }
 
+    /**
+     * Gets value without waiting for toplogy changes.
+     *
+     * @param key Key.
+     * @return Value.
+     * @throws IgniteCheckedException If failed.
+     */
     public V getTopologySafe(K key) throws IgniteCheckedException {
         String taskName = ctx.kernalContext().job().currentTaskName();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a033482/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index c4a5d70..951fddf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -787,7 +787,8 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                 map(keys.keySet(), F.t(node, keys), topVer);
 
                 onDone(Collections.<K, V>emptyMap());
-            } else {
+            }
+            else {
                 final AffinityTopologyVersion updTopVer =
                     new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a033482/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java
index ef031f6..7b69674 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java
@@ -160,10 +160,13 @@ public class IgniteCacheTopologySafeGetSelfTest extends GridCommonAbstractTest {
         }
     }
 
+    /**
+     * @return Future.
+     * @throws Exception If failed.
+     */
     private IgniteInternalFuture<?> startNodeAsync() throws Exception {
         IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
-            @Override
-            public Object call() throws Exception {
+            @Override public Object call() throws Exception {
                 startGrid(GRID_CNT);
 
                 return null;


[07/34] incubator-ignite git commit: # ignite-1265 set topology version for mvcc candidate

Posted by sb...@apache.org.
# ignite-1265 set topology version for mvcc candidate


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

Branch: refs/heads/master
Commit: b55365d316febf834cde553fbb55b33d42194069
Parents: 8b3fed8
Author: sboikov <sb...@gridgain.com>
Authored: Wed Aug 19 13:30:06 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Aug 19 13:53:41 2015 +0300

----------------------------------------------------------------------
 .../distributed/GridDistributedCacheEntry.java  |  7 ++
 .../distributed/dht/GridDhtTxPrepareFuture.java |  7 +-
 .../distributed/near/GridNearCacheEntry.java    |  6 ++
 .../distributed/near/GridNearLockFuture.java    |  4 +-
 .../cache/IgniteCacheInvokeReadThroughTest.java |  2 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |  4 +-
 .../near/GridCacheNearTxForceKeyTest.java       | 76 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |  1 +
 8 files changed, 100 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
index bd72764..e007190 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.distributed;
 
+import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -68,6 +69,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      *
      * @param threadId Owning thread ID.
      * @param ver Lock version.
+     * @param topVer Topology version.
      * @param timeout Timeout to acquire lock.
      * @param reenter Reentry flag.
      * @param tx Transaction flag.
@@ -78,6 +80,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
     @Nullable public GridCacheMvccCandidate addLocal(
         long threadId,
         GridCacheVersion ver,
+        AffinityTopologyVersion topVer,
         long timeout,
         boolean reenter,
         boolean tx,
@@ -105,6 +108,9 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
 
             cand = mvcc.addLocal(this, threadId, ver, timeout, reenter, tx, implicitSingle);
 
+            if (cand != null)
+                cand.topologyVersion(topVer);
+
             owner = mvcc.anyOwner();
 
             boolean emptyAfter = mvcc.isEmpty();
@@ -732,6 +738,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
             return addLocal(
                 tx.threadId(),
                 tx.xidVersion(),
+                tx.topologyVersion(),
                 timeout,
                 false,
                 true,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/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 ad1023f..1539a2a 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
@@ -797,6 +797,10 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         return map;
     }
 
+    /**
+     * @param keysMap Keys to request.
+     * @return Keys request future.
+     */
     private IgniteInternalFuture<Object> forceRebalanceKeys(Map<Integer, Collection<KeyCacheObject>> keysMap) {
         if (F.isEmpty(keysMap))
             return null;
@@ -978,7 +982,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         fut.onResult(e);
                     }
                     catch (IgniteCheckedException e) {
-                        fut.onResult(e);
+                        if (!cctx.kernalContext().isStopping())
+                            fut.onResult(e);
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/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 9e8d76b..194c68a 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
@@ -433,6 +433,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
     @Override public GridCacheMvccCandidate addLocal(
         long threadId,
         GridCacheVersion ver,
+        AffinityTopologyVersion topVer,
         long timeout,
         boolean reenter,
         boolean tx,
@@ -441,6 +442,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
             null,
             threadId,
             ver,
+            topVer,
             timeout,
             reenter,
             tx,
@@ -454,6 +456,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
      * @param dhtNodeId DHT node ID.
      * @param threadId Owning thread ID.
      * @param ver Lock version.
+     * @param topVer Topology version.
      * @param timeout Timeout to acquire lock.
      * @param reenter Reentry flag.
      * @param tx Transaction flag.
@@ -465,6 +468,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
         @Nullable UUID dhtNodeId,
         long threadId,
         GridCacheVersion ver,
+        AffinityTopologyVersion topVer,
         long timeout,
         boolean reenter,
         boolean tx,
@@ -513,6 +517,8 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
                 tx,
                 implicitSingle);
 
+            cand.topologyVersion(topVer);
+
             owner = mvcc.anyOwner();
 
             boolean emptyAfter = mvcc.isEmpty();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index 3d28018..b7e0d73 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -307,6 +307,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             dhtNodeId,
             threadId,
             lockVer,
+            topVer,
             timeout,
             !inTx(),
             inTx(),
@@ -319,9 +320,6 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             txEntry.cached(entry);
         }
 
-        if (c != null)
-            c.topologyVersion(topVer);
-
         synchronized (mux) {
             entries.add(entry);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
index b72540d..10ab1ab 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
@@ -34,7 +34,7 @@ import static org.apache.ignite.cache.CacheMode.*;
 public class IgniteCacheInvokeReadThroughTest extends IgniteCacheAbstractTest {
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-//        fail("https://issues.apache.org/jira/browse/IGNITE-114");
+        fail("https://issues.apache.org/jira/browse/IGNITE-114");
     }
 
     /** */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
index b6bc56e..d1d7c02 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
@@ -198,7 +198,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest {
             }
 
             // Test optimistic transaction.
-            GridTestUtils.assertThrows(log, new Callable<Object>() {
+            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     try (Transaction tx = igniteNearOnly.transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
                         nearOnly.put("key", "val");
@@ -208,7 +208,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest {
 
                     return null;
                 }
-            }, ClusterTopologyException.class, null);
+            }, ClusterTopologyCheckedException.class);
 
             // Test pessimistic transaction.
             GridTestUtils.assertThrowsWithCause(new Callable<Object>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java
new file mode 100644
index 0000000..44ef20d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ *
+ */
+public class GridCacheNearTxForceKeyTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(ASYNC);
+        ccfg.setRebalanceDelay(5000);
+        ccfg.setBackups(0);
+        ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * Test provokes scenario when primary node sends force key request to node started transaction.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNearTx() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        IgniteCache<Integer, Integer> cache = ignite0.cache(null);
+
+        Ignite ignite1 = startGrid(1);
+
+        final Integer key = 2;
+
+        assertNull(cache.getAndPut(key, key));
+
+        assertTrue(ignite0.affinity(null).isPrimary(ignite1.cluster().localNode(), key));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/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 ec50399..495719f 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
@@ -142,6 +142,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
 
         suite.addTest(new TestSuite(IgniteCacheEntryProcessorNodeJoinTest.class));
         suite.addTest(new TestSuite(IgniteAtomicCacheEntryProcessorNodeJoinTest.class));
+        suite.addTest(new TestSuite(GridCacheNearTxForceKeyTest.class));
 
         return suite;
     }


[23/34] incubator-ignite git commit: Review notes.

Posted by sb...@apache.org.
Review notes.


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

Branch: refs/heads/master
Commit: d93e1dbde5b22688530dd4deacbb06467f7e2b13
Parents: e0f445e
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Aug 20 17:44:10 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 20 17:44:10 2015 +0700

----------------------------------------------------------------------
 .../cache/store/jdbc/CacheJdbcPojoStore.java    | 20 ++++++++--------
 .../store/jdbc/CacheJdbcPojoStoreTest.java      | 24 +++++++++++++++-----
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |  3 ---
 3 files changed, 28 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d93e1dbd/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
index 428485f..1ff170e 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
@@ -99,8 +99,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
                         getters.put(field.getJavaName(), cls.getMethod("is" + prop));
                     }
                     catch (NoSuchMethodException e) {
-                        throw new CacheException("Failed to find getter in POJO class [class name=" + clsName +
-                            ", property=" + field.getJavaName() + "]", e);
+                        throw new CacheException("Failed to find getter in POJO class [clsName=" + clsName +
+                            ", prop=" + field.getJavaName() + "]", e);
                     }
                 }
 
@@ -108,8 +108,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
                     setters.put(field.getJavaName(), cls.getMethod("set" + prop, field.getJavaType()));
                 }
                 catch (NoSuchMethodException e) {
-                    throw new CacheException("Failed to find setter in POJO class [class name=" + clsName +
-                        ", property=" + field.getJavaName() + "]", e);
+                    throw new CacheException("Failed to find setter in POJO class [clsName=" + clsName +
+                        ", prop=" + field.getJavaName() + "]", e);
                 }
             }
         }
@@ -172,8 +172,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
                 Method setter = mc.setters.get(fldJavaName);
 
                 if (setter == null)
-                    throw new IllegalStateException("Failed to find setter in POJO class [class name=" + typeName +
-                        ", property=" + fldJavaName + "]");
+                    throw new IllegalStateException("Failed to find setter in POJO class [clsName=" + typeName +
+                        ", prop=" + fldJavaName + "]");
 
                 String fldDbName = field.getDatabaseName();
 
@@ -183,8 +183,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
                     setter.invoke(obj, getColumnValue(rs, colIdx, field.getJavaType()));
                 }
                 catch (Exception e) {
-                    throw new IllegalStateException("Failed to set property in POJO class [class name=" + typeName +
-                        ", property=" + fldJavaName + ", column=" + colIdx + ", db name=" + fldDbName + "]", e);
+                    throw new IllegalStateException("Failed to set property in POJO class [clsName=" + typeName +
+                        ", prop=" + fldJavaName + ", col=" + colIdx + ", dbName=" + fldDbName + "]", e);
                 }
             }
 
@@ -214,8 +214,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
             Method getter = mc.getters.get(fieldName);
 
             if (getter == null)
-                throw new CacheLoaderException("Failed to find getter in POJO class [class name=" + typeName +
-                    ", property=" + fieldName + "]");
+                throw new CacheLoaderException("Failed to find getter in POJO class [clsName=" + typeName +
+                    ", prop=" + fieldName + "]");
 
             return getter.invoke(obj);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d93e1dbd/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
index b1efb0d..68a77dc 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
@@ -183,12 +183,24 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
             // No-op.
         }
 
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS String_Entries (key varchar(100) not null, val varchar(100), PRIMARY KEY(key))");
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS UUID_Entries (key binary(16) not null, val binary(16), PRIMARY KEY(key))");
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Timestamp_Entries (key timestamp not null, val integer, PRIMARY KEY(key))");
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Organization (id integer not null, name varchar(50), city varchar(50), PRIMARY KEY(id))");
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Person (id integer not null, org_id integer, name varchar(50), PRIMARY KEY(id))");
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Person_Complex (id integer not null, org_id integer not null, city_id integer not null, name varchar(50), salary integer, PRIMARY KEY(id))");
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "String_Entries (key varchar(100) not null, val varchar(100), PRIMARY KEY(key))");
+
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "UUID_Entries (key binary(16) not null, val binary(16), PRIMARY KEY(key))");
+
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "Timestamp_Entries (key timestamp not null, val integer, PRIMARY KEY(key))");
+
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "Organization (id integer not null, name varchar(50), city varchar(50), PRIMARY KEY(id))");
+
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "Person (id integer not null, org_id integer, name varchar(50), PRIMARY KEY(id))");
+
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "Person_Complex (id integer not null, org_id integer not null, city_id integer not null, " +
+            "name varchar(50), salary integer, PRIMARY KEY(id))");
 
         conn.commit();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d93e1dbd/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
index faaf28f..8b4f3f9 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
@@ -62,9 +62,6 @@ public class CacheJdbcPojoStoreFactorySelfTest extends GridCommonAbstractTest {
                 checkStore(cache, JdbcDataSource.class);
             }
         }
-        catch (Exception e) {
-            fail("Failed to validate cache configuration. Cache store factory is not serializable.");
-        }
     }
 
     /**


[11/34] incubator-ignite git commit: # ignite-1265 fix prepare future

Posted by sb...@apache.org.
# ignite-1265 fix prepare future


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

Branch: refs/heads/master
Commit: 00417376c829c8a01cfc6ccbe5d328f01e23bf45
Parents: 7f928dc
Author: sboikov <sb...@gridgain.com>
Authored: Wed Aug 19 17:48:47 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Aug 19 17:48:47 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtTxPrepareFuture.java | 41 ++++++++++----------
 1 file changed, 20 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00417376/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 08d02f2..cd51b7b 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
@@ -297,34 +297,33 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 if (hasFilters || retVal || txEntry.op() == DELETE || txEntry.op() == TRANSFORM) {
                     CacheObject val;
 
-                    if (!txEntry.hasValue()) {
-                        cached.unswap(retVal);
+                    cached.unswap(retVal);
 
-                        boolean readThrough = (retVal || hasFilters) &&
-                            cacheCtx.config().isLoadPreviousValue() &&
-                            !txEntry.skipStore();
+                    boolean readThrough = (retVal || hasFilters) &&
+                        cacheCtx.config().isLoadPreviousValue() &&
+                        !txEntry.skipStore();
 
-                        val = cached.innerGet(
-                            tx,
-                            /*swap*/true,
-                            readThrough,
-                            /*fail fast*/false,
-                            /*unmarshal*/true,
-                            /*metrics*/retVal,
-                            /*event*/retVal,
-                            /*tmp*/false,
-                            null,
-                            null,
-                            null,
-                            null);
-                    }
-                    else
-                        val = txEntry.value();
+                    val = cached.innerGet(
+                        tx,
+                        /*swap*/true,
+                        readThrough,
+                        /*fail fast*/false,
+                        /*unmarshal*/true,
+                        /*metrics*/retVal,
+                        /*event*/retVal,
+                        /*tmp*/false,
+                        null,
+                        null,
+                        null,
+                        null);
 
                     if (retVal || txEntry.op() == TRANSFORM) {
                         if (!F.isEmpty(txEntry.entryProcessors())) {
                             invoke = true;
 
+                            if (txEntry.hasValue())
+                                val = txEntry.value();
+
                             KeyCacheObject key = txEntry.key();
 
                             Object procRes = null;


[16/34] incubator-ignite git commit: # ignite-1.3.3-p3 improved test

Posted by sb...@apache.org.
# ignite-1.3.3-p3 improved test


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

Branch: refs/heads/master
Commit: 5e81456c99a352c5dfe98aee00c6f7284af2fe13
Parents: c04fcea
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 20 10:34:55 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 20 10:34:55 2015 +0300

----------------------------------------------------------------------
 .../OptimizedMarshallerNodeFailoverTest.java    | 97 +++++++++++++++++++-
 1 file changed, 92 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5e81456c/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
index 35abf7e..65d9f36 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
@@ -33,6 +33,7 @@ import java.util.*;
 import java.util.concurrent.*;
 
 import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 
 /**
  *
@@ -65,8 +66,8 @@ public class OptimizedMarshallerNodeFailoverTest extends GridCommonAbstractTest
             CacheConfiguration ccfg = new CacheConfiguration();
 
             ccfg.setCacheMode(PARTITIONED);
-
             ccfg.setBackups(1);
+            ccfg.setWriteSynchronizationMode(FULL_SYNC);
 
             cfg.setCacheConfiguration(ccfg);
         }
@@ -79,16 +80,31 @@ public class OptimizedMarshallerNodeFailoverTest extends GridCommonAbstractTest
     /**
      * @throws Exception If failed.
      */
-    public void testClassCacheUpdateFailover() throws Exception {
+    public void testClassCacheUpdateFailover1() throws Exception {
+        classCacheUpdateFailover(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassCacheUpdateFailover2() throws Exception {
+        classCacheUpdateFailover(true);
+    }
+
+    /**
+     * @param stopSrv If {@code true} restarts server node, otherwise client node.
+     * @throws Exception If failed.
+     */
+    private void classCacheUpdateFailover(boolean stopSrv) throws Exception {
         cache = true;
 
         startGridsMultiThreaded(2);
 
-        cache = false;
+        cache = stopSrv;
 
         IgniteCache<Integer, Object> cache0 = ignite(0).cache(null);
 
-        for (int i = 0; i < 10; i++) {
+        for (int i = 0; i < 20; i++) {
             log.info("Iteration: " + i);
 
             Map<Integer, Object> map = new HashMap<>();
@@ -106,7 +122,7 @@ public class OptimizedMarshallerNodeFailoverTest extends GridCommonAbstractTest
                 }
             });
 
-            cache0.putAll(map); // Do not stop cache node, so put should not fail.
+            cache0.putAll(map);
 
             fut.get();
         }
@@ -210,6 +226,26 @@ public class OptimizedMarshallerNodeFailoverTest extends GridCommonAbstractTest
             case 9: return new TestClass9();
 
             case 10: return new TestClass10();
+
+            case 11: return new TestClass11();
+
+            case 12: return new TestClass12();
+
+            case 13: return new TestClass13();
+
+            case 14: return new TestClass14();
+
+            case 15: return new TestClass15();
+
+            case 16: return new TestClass16();
+
+            case 17: return new TestClass17();
+
+            case 18: return new TestClass18();
+
+            case 19: return new TestClass19();
+
+            case 20: return new TestClass20();
         }
 
         fail();
@@ -221,6 +257,7 @@ public class OptimizedMarshallerNodeFailoverTest extends GridCommonAbstractTest
      *
      */
     static class TestClass1 implements Serializable {
+        /** */
         int val;
     }
 
@@ -268,4 +305,54 @@ public class OptimizedMarshallerNodeFailoverTest extends GridCommonAbstractTest
      *
      */
     static class TestClass10 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass11 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass12 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass13 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass14 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass15 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass16 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass17 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass18 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass19 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass20 implements Serializable {}
 }


[15/34] incubator-ignite git commit: # ignite-1.3.3-p3 init missed retryReadyFuture, do not retry tryPutIfAbsent, backported fixes from master

Posted by sb...@apache.org.
# ignite-1.3.3-p3 init missed retryReadyFuture, do not retry tryPutIfAbsent, backported fixes from master


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

Branch: refs/heads/master
Commit: c04fceac8f2db6b024b285b537dd26171b95610a
Parents: 8f1c1c0
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 20 09:18:44 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 20 09:55:40 2015 +0300

----------------------------------------------------------------------
 .../dht/atomic/GridNearAtomicUpdateFuture.java          | 12 +++++++++++-
 .../near/GridNearOptimisticTxPrepareFuture.java         |  6 +++++-
 .../near/GridNearPessimisticTxPrepareFuture.java        |  2 ++
 .../processors/cache/transactions/IgniteTxManager.java  |  5 ++---
 4 files changed, 20 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04fceac/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 5dc5494..54857e3 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
@@ -231,6 +231,9 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
 
         nearEnabled = CU.isNearEnabled(cctx);
 
+        if (!waitTopFut)
+            remapCnt = 1;
+
         this.remapCnt = new AtomicInteger(remapCnt);
     }
 
@@ -340,6 +343,8 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Void> completeFuture(AffinityTopologyVersion topVer) {
         if (waitForPartitionExchange() && topologyVersion().compareTo(topVer) < 0) {
+            GridFutureAdapter<Void> fut = null;
+
             synchronized (this) {
                 if (this.topVer == AffinityTopologyVersion.ZERO)
                     return null;
@@ -348,9 +353,14 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                     if (topCompleteFut == null)
                         topCompleteFut = new GridFutureAdapter<>();
 
-                    return topCompleteFut;
+                    fut = topCompleteFut;
                 }
             }
+
+            if (fut != null && isDone())
+                fut.onDone();
+
+            return fut;
         }
 
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04fceac/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 4bb4c67..305840b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -567,8 +567,12 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
             try {
                 cctx.io().send(n, req, tx.ioPolicy());
             }
+            catch (ClusterTopologyCheckedException e) {
+                e.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion()));
+
+                fut.onResult(e);
+            }
             catch (IgniteCheckedException e) {
-                // Fail the whole thing.
                 fut.onResult(e);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04fceac/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index 3d43797..a522b62 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -229,6 +229,8 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
                     cctx.io().send(node, req, tx.ioPolicy());
                 }
                 catch (ClusterTopologyCheckedException e) {
+                    e.retryReadyFuture(cctx.nextAffinityReadyFuture(topVer));
+
                     fut.onNodeLeft(e);
                 }
                 catch (IgniteCheckedException e) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04fceac/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 b6c77f6..868d1f9 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
@@ -527,10 +527,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
                 TransactionState state = tx.state();
                 AffinityTopologyVersion txTopVer = tx.topologyVersion();
 
-                if ((state == PREPARING || state == PREPARED || state == COMMITTING)
-                    && txTopVer.compareTo(AffinityTopologyVersion.ZERO) > 0 && txTopVer.compareTo(topVer) < 0) {
+                if ((state != ACTIVE && state != COMMITTED && state != ROLLED_BACK && state != UNKNOWN)
+                    && txTopVer.compareTo(AffinityTopologyVersion.ZERO) > 0 && txTopVer.compareTo(topVer) < 0)
                     res.add(tx.finishFuture());
-                }
             }
         }
 


[10/34] incubator-ignite git commit: # ignite-1265 fix test

Posted by sb...@apache.org.
# ignite-1265 fix test


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

Branch: refs/heads/master
Commit: 7f928dc048a9536e5c020b9baebe5637b0fd4ba6
Parents: 7d1a550
Author: sboikov <sb...@gridgain.com>
Authored: Wed Aug 19 15:47:18 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Aug 19 17:30:06 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtTxPrepareFuture.java |  52 ++-
 .../GridCacheTransformEventSelfTest.java        |   2 +
 .../IgniteCacheCrossCacheTxFailoverTest.java    | 433 +++++++++++++++++++
 ...idCachePartitionedHitsAndMissesSelfTest.java |  20 +-
 4 files changed, 475 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f928dc0/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 1539a2a..08d02f2 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
@@ -295,25 +295,31 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 boolean hasFilters = !F.isEmptyOrNulls(txEntry.filters()) && !F.isAlwaysTrue(txEntry.filters());
 
                 if (hasFilters || retVal || txEntry.op() == DELETE || txEntry.op() == TRANSFORM) {
-                    cached.unswap(retVal);
+                    CacheObject val;
 
-                    boolean readThrough = (retVal || hasFilters) &&
-                        cacheCtx.config().isLoadPreviousValue() &&
-                        !txEntry.skipStore();
+                    if (!txEntry.hasValue()) {
+                        cached.unswap(retVal);
 
-                    CacheObject val = cached.innerGet(
-                        tx,
-                        /*swap*/true,
-                        readThrough,
-                        /*fail fast*/false,
-                        /*unmarshal*/true,
-                        /*metrics*/retVal,
-                        /*event*/retVal,
-                        /*tmp*/false,
-                        null,
-                        null,
-                        null,
-                        null);
+                        boolean readThrough = (retVal || hasFilters) &&
+                            cacheCtx.config().isLoadPreviousValue() &&
+                            !txEntry.skipStore();
+
+                        val = cached.innerGet(
+                            tx,
+                            /*swap*/true,
+                            readThrough,
+                            /*fail fast*/false,
+                            /*unmarshal*/true,
+                            /*metrics*/retVal,
+                            /*event*/retVal,
+                            /*tmp*/false,
+                            null,
+                            null,
+                            null,
+                            null);
+                    }
+                    else
+                        val = txEntry.value();
 
                     if (retVal || txEntry.op() == TRANSFORM) {
                         if (!F.isEmpty(txEntry.entryProcessors())) {
@@ -344,12 +350,14 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
                             txEntry.entryProcessorCalculatedValue(val);
 
-                            if (err != null || procRes != null)
-                                ret.addEntryProcessResult(txEntry.context(), key, null, procRes, err);
-                            else
-                                ret.invokeResult(true);
+                            if (retVal) {
+                                if (err != null || procRes != null)
+                                    ret.addEntryProcessResult(txEntry.context(), key, null, procRes, err);
+                                else
+                                    ret.invokeResult(true);
+                            }
                         }
-                        else
+                        else if (retVal)
                             ret.value(cacheCtx, val);
                     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f928dc0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
index 459e015..d05764c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
@@ -165,6 +165,8 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
 
         startGrids(GRID_CNT);
 
+        awaitPartitionMapExchange();
+
         ignites = new Ignite[GRID_CNT];
         ids = new UUID[GRID_CNT];
         caches = new IgniteCache[GRID_CNT];

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f928dc0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
new file mode 100644
index 0000000..5432e76
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
@@ -0,0 +1,433 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.*;
+import javax.cache.processor.*;
+import java.io.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
+/**
+ *
+ */
+public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String CACHE1 = "cache1";
+
+    /** */
+    private static final String CACHE2 = "cache2";
+
+    /** */
+    private static final int GRID_CNT = 4;
+
+    /** */
+    private static final int KEY_RANGE = 1000;
+
+    /** */
+    private static final long TEST_TIME = 3 * 60_000;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        if (gridName.equals(getTestGridName(GRID_CNT - 1)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(4);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @param name Cache name.
+     * @param cacheMode Cache mode.
+     * @param parts Number of partitions.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration(String name, CacheMode cacheMode, int parts) {
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setName(name);
+        ccfg.setCacheMode(cacheMode);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        if (cacheMode == PARTITIONED)
+            ccfg.setBackups(1);
+
+        ccfg.setAffinity(new RendezvousAffinityFunction(false, parts));
+
+        return ccfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return TEST_TIME + 60_000;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCachePessimisticTxFailover() throws Exception {
+        crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCachePessimisticTxFailoverDifferentAffinity() throws Exception {
+        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheOptimisticTxFailover() throws Exception {
+        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheOptimisticTxFailoverDifferentAffinity() throws Exception {
+        crossCacheTxFailover(PARTITIONED, false, OPTIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCachePessimisticTxFailoverReplicated() throws Exception {
+        crossCacheTxFailover(REPLICATED, true, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheOptimisticTxFailoverReplicated() throws Exception {
+        crossCacheTxFailover(REPLICATED, true, OPTIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCachePessimisticTxFailoverDifferentAffinityReplicated() throws Exception {
+        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @param cacheMode Cache mode.
+     * @param sameAff If {@code false} uses different number of partitions for caches.
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @throws Exception If failed.
+     */
+    private void crossCacheTxFailover(CacheMode cacheMode,
+        boolean sameAff,
+        final TransactionConcurrency concurrency,
+        final TransactionIsolation isolation) throws Exception {
+        IgniteKernal ignite0 = (IgniteKernal)ignite(0);
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        try {
+            ignite0.createCache(cacheConfiguration(CACHE1, cacheMode, 256));
+            ignite0.createCache(cacheConfiguration(CACHE2, cacheMode, sameAff ? 256 : 128));
+
+            final AtomicInteger threadIdx = new AtomicInteger();
+
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    int idx = threadIdx.getAndIncrement();
+
+                    Ignite ignite = ignite(idx % GRID_CNT);
+
+                    log.info("Started update thread [node=" + ignite.name() +
+                        ", client=" + ignite.configuration().isClientMode() + ']');
+
+                    IgniteCache<TestKey, TestValue> cache1 = ignite.cache(CACHE1);
+                    IgniteCache<TestKey, TestValue> cache2 = ignite.cache(CACHE2);
+
+                    assertNotSame(cache1, cache2);
+
+                    IgniteTransactions txs = ignite.transactions();
+
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    long iter = 0;
+
+                    while (!stop.get()) {
+                        boolean sameKey = rnd.nextBoolean();
+
+                        try {
+                            try (Transaction tx = txs.txStart(concurrency, isolation)) {
+                                if (sameKey) {
+                                    TestKey key = new TestKey(rnd.nextLong(KEY_RANGE));
+
+                                    cacheOperation(rnd, cache1, key);
+                                    cacheOperation(rnd, cache2, key);
+                                }
+                                else {
+                                    TestKey key1 = new TestKey(rnd.nextLong(KEY_RANGE));
+                                    TestKey key2 = new TestKey(key1.key() + 1);
+
+                                    cacheOperation(rnd, cache1, key1);
+                                    cacheOperation(rnd, cache2, key2);
+                                }
+
+                                tx.commit();
+                            }
+                        }
+                        catch (CacheException | IgniteException e) {
+                            log.info("Update error: " + e);
+                        }
+
+                        if (iter++ % 500 == 0)
+                            log.info("Iteration: " + iter);
+                    }
+
+                    return null;
+                }
+
+                /**
+                 * @param rnd Random.
+                 * @param cache Cache.
+                 * @param key Key.
+                 */
+                private void cacheOperation(ThreadLocalRandom rnd, IgniteCache<TestKey, TestValue> cache, TestKey key) {
+                    switch (rnd.nextInt(4)) {
+                        case 0:
+                            cache.put(key, new TestValue(rnd.nextLong()));
+
+                            break;
+
+                        case 1:
+                            cache.remove(key);
+
+                            break;
+
+                        case 2:
+                            cache.invoke(key, new TestEntryProcessor(rnd.nextBoolean() ? 1L : null));
+
+                            break;
+
+                        case 3:
+                            cache.get(key);
+
+                            break;
+
+                        default:
+                            assert false;
+                    }
+                }
+            }, 10, "tx-thread");
+
+            long stopTime = System.currentTimeMillis() + 3 * 60_000;
+
+            long topVer = ignite0.cluster().topologyVersion();
+
+            boolean failed = false;
+
+            while (System.currentTimeMillis() < stopTime) {
+                log.info("Start node.");
+
+                IgniteKernal ignite = (IgniteKernal)startGrid(GRID_CNT);
+
+                assertFalse(ignite.configuration().isClientMode());
+
+                topVer++;
+
+                IgniteInternalFuture<?> affFut = ignite.context().cache().context().exchange().affinityReadyFuture(
+                    new AffinityTopologyVersion(topVer));
+
+                try {
+                    if (affFut != null)
+                        affFut.get(30_000);
+                }
+                catch (IgniteFutureTimeoutCheckedException e) {
+                    log.error("Failed to wait for affinity future after start: " + topVer);
+
+                    failed = true;
+
+                    break;
+                }
+
+                Thread.sleep(500);
+
+                log.info("Stop node.");
+
+                stopGrid(GRID_CNT);
+
+                topVer++;
+
+                affFut = ignite0.context().cache().context().exchange().affinityReadyFuture(
+                    new AffinityTopologyVersion(topVer));
+
+                try {
+                    if (affFut != null)
+                        affFut.get(30_000);
+                }
+                catch (IgniteFutureTimeoutCheckedException e) {
+                    log.error("Failed to wait for affinity future after stop: " + topVer);
+
+                    failed = true;
+
+                    break;
+                }
+            }
+
+            stop.set(true);
+
+            fut.get();
+
+            assertFalse("Test failed, see log for details.", failed);
+        }
+        finally {
+            stop.set(true);
+
+            ignite0.destroyCache(CACHE1);
+            ignite0.destroyCache(CACHE2);
+
+            awaitPartitionMapExchange();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestKey implements Serializable {
+        /** */
+        private long key;
+
+        /**
+         * @param key Key.
+         */
+        public TestKey(long key) {
+            this.key = key;
+        }
+
+        /**
+         * @return Key.
+         */
+        public long key() {
+            return key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestKey testKey = (TestKey)o;
+
+            return key == testKey.key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return (int)(key ^ (key >>> 32));
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestValue implements Serializable {
+        /** */
+        private long val;
+
+        /**
+         * @param val Value.
+         */
+        public TestValue(long val) {
+            this.val = val;
+        }
+
+        /**
+         * @return Value.
+         */
+        public long value() {
+            return val;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestEntryProcessor implements CacheEntryProcessor<TestKey, TestValue, TestValue> {
+        /** */
+        private Long val;
+
+        /**
+         * @param val Value.
+         */
+        public TestEntryProcessor(@Nullable Long val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public TestValue process(MutableEntry<TestKey, TestValue> e, Object... args) {
+            TestValue old = e.getValue();
+
+            if (val != null)
+                e.setValue(new TestValue(val));
+
+            return old;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f928dc0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java
index 6138022..6ceded3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java
@@ -32,6 +32,8 @@ import java.util.*;
 
 import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Test for issue GG-3997 Total Hits and Misses display wrong value for in-memory database.
@@ -50,18 +52,18 @@ public class GridCachePartitionedHitsAndMissesSelfTest extends GridCommonAbstrac
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        // DiscoverySpi
+        // DiscoverySpi.
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
         disco.setIpFinder(IP_FINDER);
         cfg.setDiscoverySpi(disco);
 
         // Cache.
-        cfg.setCacheConfiguration(cacheConfiguration(gridName));
+        cfg.setCacheConfiguration(cacheConfiguration());
 
         TransactionConfiguration tCfg = new TransactionConfiguration();
 
-        tCfg.setDefaultTxConcurrency(TransactionConcurrency.PESSIMISTIC);
-        tCfg.setDefaultTxIsolation(TransactionIsolation.REPEATABLE_READ);
+        tCfg.setDefaultTxConcurrency(PESSIMISTIC);
+        tCfg.setDefaultTxIsolation(REPEATABLE_READ);
 
         cfg.setTransactionConfiguration(tCfg);
 
@@ -71,20 +73,18 @@ public class GridCachePartitionedHitsAndMissesSelfTest extends GridCommonAbstrac
     /**
      * Cache configuration.
      *
-     * @param gridName Grid name.
      * @return Cache configuration.
      * @throws Exception In case of error.
      */
-    protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+    protected CacheConfiguration cacheConfiguration() throws Exception {
         CacheConfiguration cfg = defaultCacheConfiguration();
+
         cfg.setCacheMode(PARTITIONED);
         cfg.setStartSize(700000);
         cfg.setWriteSynchronizationMode(FULL_ASYNC);
         cfg.setEvictionPolicy(null);
         cfg.setBackups(1);
         cfg.setNearConfiguration(null);
-        cfg.setRebalanceDelay(-1);
-        cfg.setBackups(1);
         cfg.setStatisticsEnabled(true);
 
         return cfg;
@@ -96,10 +96,10 @@ public class GridCachePartitionedHitsAndMissesSelfTest extends GridCommonAbstrac
      * @throws Exception If failed.
      */
     public void testHitsAndMisses() throws Exception {
-        assert(GRID_CNT > 0);
-
         startGrids(GRID_CNT);
 
+        awaitPartitionMapExchange();
+
         try {
             final Ignite g = grid(0);
 


[06/34] incubator-ignite git commit: # ignite-1265

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


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

Branch: refs/heads/master
Commit: 8b3fed850ccc4527a6593c5ec4e596ca6c08b61f
Parents: 5065a1e
Author: sboikov <sb...@gridgain.com>
Authored: Wed Aug 19 09:32:54 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Aug 19 10:26:10 2015 +0300

----------------------------------------------------------------------
 .../CachePartialUpdateCheckedException.java     | 11 ++++---
 ...teAtomicCacheEntryProcessorNodeJoinTest.java | 32 ++++++++++++++++++++
 .../IgniteCacheEntryProcessorNodeJoinTest.java  | 25 ++++++++-------
 .../testsuites/IgniteCacheTestSuite2.java       |  3 ++
 4 files changed, 55 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8b3fed85/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
index c2259df..fc846f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
@@ -47,8 +47,9 @@ public class CachePartialUpdateCheckedException extends IgniteCheckedException {
      * Gets collection of failed keys.
      * @return Collection of failed keys.
      */
-    public <K> Collection<K> failedKeys() {
-        return (Collection<K>)failedKeys;
+    @SuppressWarnings("unchecked")
+    public synchronized <K> Collection<K> failedKeys() {
+        return new HashSet<>((Collection<K>)failedKeys);
     }
 
     /**
@@ -56,7 +57,7 @@ public class CachePartialUpdateCheckedException extends IgniteCheckedException {
      * @param err Error.
      * @param topVer Topology version for failed update.
      */
-    public void add(Collection<?> failedKeys, Throwable err, AffinityTopologyVersion topVer) {
+    public synchronized void add(Collection<?> failedKeys, Throwable err, AffinityTopologyVersion topVer) {
         if (topVer != null) {
             AffinityTopologyVersion topVer0 = this.topVer;
 
@@ -72,7 +73,7 @@ public class CachePartialUpdateCheckedException extends IgniteCheckedException {
     /**
      * @return Topology version.
      */
-    public AffinityTopologyVersion topologyVersion() {
+    public synchronized AffinityTopologyVersion topologyVersion() {
         return topVer;
     }
 
@@ -80,7 +81,7 @@ public class CachePartialUpdateCheckedException extends IgniteCheckedException {
      * @param failedKeys Failed keys.
      * @param err Error.
      */
-    public void add(Collection<?> failedKeys, Throwable err) {
+    public synchronized void add(Collection<?> failedKeys, Throwable err) {
         add(failedKeys, err, null);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8b3fed85/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteAtomicCacheEntryProcessorNodeJoinTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteAtomicCacheEntryProcessorNodeJoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteAtomicCacheEntryProcessorNodeJoinTest.java
new file mode 100644
index 0000000..af87a7d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteAtomicCacheEntryProcessorNodeJoinTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public class IgniteAtomicCacheEntryProcessorNodeJoinTest extends IgniteCacheEntryProcessorNodeJoinTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8b3fed85/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
index 94bfd8f..955a792 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -51,9 +52,6 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
     /** Number of increment iterations. */
     private static final int NUM_SETS = 50;
 
-    /** Helper for excluding stopped node from iteration logic. */
-    private AtomicReferenceArray<Ignite> grids;
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -61,7 +59,7 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
         CacheConfiguration cache = new CacheConfiguration();
 
         cache.setCacheMode(PARTITIONED);
-        cache.setAtomicityMode(TRANSACTIONAL);
+        cache.setAtomicityMode(atomicityMode());
         cache.setWriteSynchronizationMode(FULL_SYNC);
         cache.setBackups(1);
         cache.setRebalanceMode(SYNC);
@@ -83,21 +81,21 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
         return cfg;
     }
 
+    /**
+     * @return Atomicity mode.
+     */
+    protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         startGrids(GRID_CNT);
-
-        grids = new AtomicReferenceArray<>(GRID_CNT);
-
-        for (int i = 0; i < GRID_CNT; i++)
-            grids.set(i, grid(i));
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
-
-        grids = null;
     }
 
     /**
@@ -115,6 +113,7 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
     }
 
     /**
+     * @param invokeAll If {@code true} tests invokeAll operation.
      * @throws Exception If failed.
      */
     private void checkEntryProcessorNodeJoin(boolean invokeAll) throws Exception {
@@ -163,6 +162,7 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
     }
 
     /**
+     * @param invokeAll If {@code true} tests invokeAll operation.
      * @throws Exception If failed.
      */
     private void checkIncrement(boolean invokeAll) throws Exception {
@@ -201,6 +201,9 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
         /** */
         private String val;
 
+        /**
+         * @param val Value.
+         */
         private Processor(String val) {
             this.val = val;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8b3fed85/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 dcbab07..ec50399 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
@@ -140,6 +140,9 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(IgniteCacheClientNodeChangingTopologyTest.class));
         suite.addTest(new TestSuite(IgniteCacheClientNodeConcurrentStart.class));
 
+        suite.addTest(new TestSuite(IgniteCacheEntryProcessorNodeJoinTest.class));
+        suite.addTest(new TestSuite(IgniteAtomicCacheEntryProcessorNodeJoinTest.class));
+
         return suite;
     }
 }


[34/34] incubator-ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/master
Commit: 0399ccd832d5156dc6fad683f81f07bafb3a094a
Parents: 1a03348 ebb5d4a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Aug 21 11:12:50 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Aug 21 11:12:50 2015 +0300

----------------------------------------------------------------------
 examples/config/example-cache.xml | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------



[28/34] incubator-ignite git commit: IGNITE-1275 - Use topology-safe method in marshaller context to prevent deadlocks.

Posted by sb...@apache.org.
IGNITE-1275 - Use topology-safe method in marshaller context to prevent deadlocks.


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

Branch: refs/heads/master
Commit: 6b93ee7a39b94b6edb52de7543fb222ef44a1bd3
Parents: abbd308
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Aug 20 16:19:01 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Aug 20 16:19:01 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/MarshallerContextImpl.java  |   2 +-
 .../processors/cache/GridCacheAdapter.java      |  82 +++++--
 .../distributed/dht/GridDhtCacheAdapter.java    |  12 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |  12 +-
 .../dht/GridPartitionedGetFuture.java           |  86 +++++---
 .../dht/atomic/GridDhtAtomicCache.java          |  16 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  19 +-
 .../distributed/near/GridNearAtomicCache.java   |   6 +-
 .../distributed/near/GridNearCacheAdapter.java  |  15 +-
 .../distributed/near/GridNearCacheEntry.java    |   4 +-
 .../distributed/near/GridNearGetFuture.java     | 101 ++++++---
 .../near/GridNearTransactionalCache.java        |   9 +-
 .../cache/distributed/near/GridNearTxLocal.java |   7 +-
 .../local/atomic/GridLocalAtomicCache.java      |  17 +-
 .../IgniteCacheTopologySafeGetSelfTest.java     | 215 +++++++++++++++++++
 ...gniteCachePutRetryTransactionalSelfTest.java |   2 +
 .../IgniteCacheFailoverTestSuite.java           |   2 +
 17 files changed, 494 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 87bd3b6..dc0fd57 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -136,7 +136,7 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
                 throw new IllegalStateException("Failed to initialize marshaller context (grid is stopping).");
         }
 
-        String clsName = cache0.get(id);
+        String clsName = cache0.getTopologySafe(id);
 
         if (clsName == null) {
             File file = new File(workDir, id + ".classname");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 992edd8..c7fbbfc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -526,7 +526,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             /*subj id*/null,
             /*task name*/null,
             /*deserialize portable*/false,
-            /*skip values*/true
+            /*skip values*/true,
+            /*can remap*/true
         ).chain(new CX1<IgniteInternalFuture<Map<K, V>>, Boolean>() {
             @Override public Boolean applyx(IgniteInternalFuture<Map<K, V>> fut) throws IgniteCheckedException {
                 Map<K, V> map = fut.get();
@@ -560,7 +561,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             /*subj id*/null,
             /*task name*/null,
             /*deserialize portable*/false,
-            /*skip values*/true
+            /*skip values*/true,
+            /*can remap*/true
         ).chain(new CX1<IgniteInternalFuture<Map<K, V>>, Boolean>() {
             @Override public Boolean applyx(IgniteInternalFuture<Map<K, V>> fut) throws IgniteCheckedException {
                 Map<K, V> kvMap = fut.get();
@@ -894,7 +896,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<Cache.Entry<K, V>> entrySet() {
-        return entrySet((CacheEntryPredicate[]) null);
+        return entrySet((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
@@ -919,12 +921,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<K> primaryKeySet() {
-        return primaryKeySet((CacheEntryPredicate[]) null);
+        return primaryKeySet((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<V> values() {
-        return values((CacheEntryPredicate[]) null);
+        return values((CacheEntryPredicate[])null);
     }
 
     /**
@@ -1210,22 +1212,57 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     @Override public V getForcePrimary(K key) throws IgniteCheckedException {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
-        return getAllAsync(F.asList(key), /*force primary*/true, /*skip tx*/false, null, null, taskName, true, false)
-            .get().get(key);
+        return getAllAsync(
+            F.asList(key),
+            /*force primary*/true,
+            /*skip tx*/false,
+            /*cached entry*/null,
+            /*subject id*/null,
+            taskName,
+            /*deserialize cache objects*/true,
+            /*skip values*/false,
+            /*can remap*/true
+        ).get().get(key);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<V> getForcePrimaryAsync(final K key) {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
-        return getAllAsync(Collections.singletonList(key), /*force primary*/true, /*skip tx*/false, null, null,
-            taskName, true, false).chain(new CX1<IgniteInternalFuture<Map<K, V>>, V>() {
-            @Override public V applyx(IgniteInternalFuture<Map<K, V>> e) throws IgniteCheckedException {
+        return getAllAsync(
+            Collections.singletonList(key),
+            /*force primary*/true,
+            /*skip tx*/false,
+            null,
+            null,
+            taskName,
+            true,
+            false,
+            /*can remap*/true
+        ).chain(new CX1<IgniteInternalFuture<Map<K, V>>, V>() {
+            @Override
+            public V applyx(IgniteInternalFuture<Map<K, V>> e) throws IgniteCheckedException {
                 return e.get().get(key);
             }
         });
     }
 
+    public V getTopologySafe(K key) throws IgniteCheckedException {
+        String taskName = ctx.kernalContext().job().currentTaskName();
+
+        return getAllAsync(
+            F.asList(key),
+            /*force primary*/false,
+            /*skip tx*/false,
+            /*cached entry*/null,
+            /*subject id*/null,
+            taskName,
+            /*deserialize cache objects*/true,
+            /*skip values*/false,
+            /*can remap*/false
+        ).get().get(key);
+    }
+
     /** {@inheritDoc} */
     @Nullable @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
         return getAllOutTxAsync(keys).get();
@@ -1242,7 +1279,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             null,
             taskName,
             !ctx.keepPortable(),
-            false);
+            /*skip values*/false,
+            /*can remap*/true);
     }
 
     /**
@@ -1582,7 +1620,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         @Nullable UUID subjId,
         String taskName,
         boolean deserializePortable,
-        boolean skipVals
+        boolean skipVals,
+        boolean canRemap
     ) {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
@@ -1597,7 +1636,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             deserializePortable,
             forcePrimary,
             skipVals ? null : expiryPolicy(opCtx != null ? opCtx.expiry() : null),
-            skipVals);
+            skipVals,
+            canRemap);
     }
 
     /**
@@ -1623,7 +1663,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         final boolean deserializePortable,
         final boolean forcePrimary,
         @Nullable IgniteCacheExpiryPolicy expiry,
-        final boolean skipVals
+        final boolean skipVals,
+        boolean canRemap
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -1638,7 +1679,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             deserializePortable,
             expiry,
             skipVals,
-            false);
+            false,
+            canRemap);
     }
 
     /**
@@ -1661,7 +1703,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         final boolean deserializePortable,
         @Nullable IgniteCacheExpiryPolicy expiry,
         final boolean skipVals,
-        final boolean keepCacheObjects
+        final boolean keepCacheObjects,
+        boolean canRemap
         ) {
         if (F.isEmpty(keys))
             return new GridFinishedFuture<>(Collections.<K1, V1>emptyMap());
@@ -1684,7 +1727,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 assert keys != null;
 
                 final AffinityTopologyVersion topVer = tx == null
-                    ? ctx.affinity().affinityTopologyVersion()
+                    ? (canRemap ? ctx.affinity().affinityTopologyVersion(): ctx.shared().exchange().readyAffinityVersion())
                     : tx.topologyVersion();
 
                 final Map<K1, V1> map = new GridLeanMap<>(keys.size());
@@ -4461,7 +4504,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             null,
             taskName,
             deserializePortable,
-            false);
+            false,
+            /*can remap*/true);
     }
 
     /**
@@ -4682,7 +4726,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
 
         /**
-         * @param tx Transaction.
+         *
          */
         public void execute() {
             tx = ctx.tm().newTx(

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/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 adea9e0..a7b3b1e 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
@@ -527,7 +527,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
         @Nullable UUID subjId,
         String taskName,
         boolean deserializePortable,
-        boolean skipVals
+        boolean skipVals,
+        boolean canRemap
     ) {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
@@ -540,7 +541,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
             deserializePortable,
             forcePrimary,
             null,
-            skipVals);
+            skipVals,
+            canRemap);
     }
 
     /**
@@ -558,7 +560,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
         @Nullable UUID subjId,
         String taskName,
         @Nullable IgniteCacheExpiryPolicy expiry,
-        boolean skipVals
+        boolean skipVals,
+        boolean canRemap
         ) {
         return getAllAsync0(keys,
             readThrough,
@@ -568,7 +571,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
             false,
             expiry,
             skipVals,
-            /*keep cache objects*/true);
+            /*keep cache objects*/true,
+            canRemap);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index 742fbfe..9005541 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -349,12 +349,14 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
             }
             else {
                 if (tx == null) {
-                    fut = cache().getDhtAllAsync(keys.keySet(),
+                    fut = cache().getDhtAllAsync(
+                        keys.keySet(),
                         readThrough,
                         subjId,
                         taskName,
                         expiryPlc,
-                        skipVals);
+                        skipVals,
+                        /*can remap*/true);
                 }
                 else {
                     fut = tx.getAllAsync(cctx,
@@ -387,12 +389,14 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
                         }
                         else {
                             if (tx == null) {
-                                return cache().getDhtAllAsync(keys.keySet(),
+                                return cache().getDhtAllAsync(
+                                    keys.keySet(),
                                     readThrough,
                                     subjId,
                                     taskName,
                                     expiryPlc,
-                                    skipVals);
+                                    skipVals,
+                                    /*can remap*/true);
                             }
                             else {
                                 return tx.getAllAsync(cctx,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 79d5e75..a85962f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -61,7 +61,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
         DFLT_MAX_REMAP_CNT);
 
     /** Context. */
-    private GridCacheContext<K, V> cctx;
+    private final GridCacheContext<K, V> cctx;
 
     /** Keys. */
     private Collection<KeyCacheObject> keys;
@@ -105,6 +105,9 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
     /** Skip values flag. */
     private boolean skipVals;
 
+    /** Flag indicating whether future can be remapped on a newer topology version. */
+    private final boolean canRemap;
+
     /**
      * @param cctx Context.
      * @param keys Keys.
@@ -130,7 +133,8 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
         String taskName,
         boolean deserializePortable,
         @Nullable IgniteCacheExpiryPolicy expiryPlc,
-        boolean skipVals
+        boolean skipVals,
+        boolean canRemap
     ) {
         super(cctx.kernalContext(), CU.<K, V>mapsReducer(keys.size()));
 
@@ -147,6 +151,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
         this.taskName = taskName;
         this.expiryPlc = expiryPlc;
         this.skipVals = skipVals;
+        this.canRemap = canRemap;
 
         futId = IgniteUuid.randomUuid();
 
@@ -160,7 +165,8 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
      * Initializes future.
      */
     public void init() {
-        AffinityTopologyVersion topVer = this.topVer.topologyVersion() > 0 ? this.topVer : cctx.affinity().affinityTopologyVersion();
+        AffinityTopologyVersion topVer = this.topVer.topologyVersion() > 0 ? this.topVer :
+            canRemap ? cctx.affinity().affinityTopologyVersion() : cctx.shared().exchange().readyAffinityVersion();
 
         map(keys, Collections.<ClusterNode, LinkedHashMap<KeyCacheObject, Boolean>>emptyMap(), topVer);
 
@@ -334,7 +340,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
                             remapKeys.add(key);
                     }
 
-                    AffinityTopologyVersion updTopVer = new AffinityTopologyVersion(cctx.discovery().topologyVersion());
+                    AffinityTopologyVersion updTopVer = cctx.discovery().topologyVersionEx();
 
                     assert updTopVer.compareTo(topVer) > 0 : "Got invalid partitions for local node but topology version did " +
                         "not change [topVer=" + topVer + ", updTopVer=" + updTopVer +
@@ -461,7 +467,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
                     }
                 }
 
-                ClusterNode node = cctx.affinity().primary(key, topVer);
+                ClusterNode node = affinityNode(key, topVer);
 
                 if (node == null) {
                     onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
@@ -522,6 +528,28 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
     }
 
     /**
+     * Finds affinity node to send get request to.
+     *
+     * @param key Key to get.
+     * @param topVer Topology version.
+     * @return Affinity node from which the key will be requested.
+     */
+    private ClusterNode affinityNode(KeyCacheObject key, AffinityTopologyVersion topVer) {
+        if (!canRemap) {
+            List<ClusterNode> nodes = cctx.affinity().nodes(key, topVer);
+
+            for (ClusterNode node : nodes) {
+                if (cctx.discovery().alive(node))
+                    return node;
+            }
+
+            return null;
+        }
+        else
+            return cctx.affinity().primary(key, topVer);
+    }
+
+    /**
      * @param infos Entry infos.
      * @return Result map.
      */
@@ -557,14 +585,14 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
         private final IgniteUuid futId = IgniteUuid.randomUuid();
 
         /** Node ID. */
-        private ClusterNode node;
+        private final ClusterNode node;
 
         /** Keys. */
         @GridToStringInclude
-        private LinkedHashMap<KeyCacheObject, Boolean> keys;
+        private final LinkedHashMap<KeyCacheObject, Boolean> keys;
 
         /** Topology version on which this future was mapped. */
-        private AffinityTopologyVersion topVer;
+        private final AffinityTopologyVersion topVer;
 
         /** {@code True} if remapped after node left. */
         private boolean remapped;
@@ -625,30 +653,38 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
             if (log.isDebugEnabled())
                 log.debug("Remote node left grid while sending or waiting for reply (will retry): " + this);
 
-            final AffinityTopologyVersion updTopVer =
-                new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
+            // Try getting from existing nodes.
+            if (!canRemap) {
+                map(keys.keySet(), F.t(node, keys), topVer);
+
+                onDone(Collections.<K, V>emptyMap());
+            }
+            else {
+                final AffinityTopologyVersion updTopVer =
+                    new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
 
-            final GridFutureRemapTimeoutObject timeout = new GridFutureRemapTimeoutObject(this,
-                cctx.kernalContext().config().getNetworkTimeout(),
-                updTopVer,
-                e);
+                final GridFutureRemapTimeoutObject timeout = new GridFutureRemapTimeoutObject(this,
+                    cctx.kernalContext().config().getNetworkTimeout(),
+                    updTopVer,
+                    e);
 
-            cctx.affinity().affinityReadyFuture(updTopVer).listen(
-                new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                    @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
-                        if (timeout.finish()) {
-                            cctx.kernalContext().timeout().removeTimeoutObject(timeout);
+                cctx.affinity().affinityReadyFuture(updTopVer).listen(
+                    new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
+                            if (timeout.finish()) {
+                                cctx.kernalContext().timeout().removeTimeoutObject(timeout);
 
-                            // Remap.
-                            map(keys.keySet(), F.t(node, keys), updTopVer);
+                                // Remap.
+                                map(keys.keySet(), F.t(node, keys), updTopVer);
 
-                            onDone(Collections.<K, V>emptyMap());
+                                onDone(Collections.<K, V>emptyMap());
+                            }
                         }
                     }
-                }
-            );
+                );
 
-            cctx.kernalContext().timeout().addTimeoutObject(timeout);
+                cctx.kernalContext().timeout().addTimeoutObject(timeout);
+            }
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/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 96e6edc..5b82162 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
@@ -248,7 +248,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         @Nullable UUID subjId,
         final String taskName,
         final boolean deserializePortable,
-        final boolean skipVals
+        final boolean skipVals,
+        final boolean canRemap
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -278,7 +279,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     deserializePortable,
                     expiryPlc,
                     skipVals,
-                    skipStore);
+                    skipStore,
+                    canRemap);
             }
         });
     }
@@ -870,8 +872,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         boolean deserializePortable,
         @Nullable ExpiryPolicy expiryPlc,
         boolean skipVals,
-        boolean skipStore) {
-        AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
+        boolean skipStore,
+        boolean canRemap
+    ) {
+        AffinityTopologyVersion topVer = canRemap ? ctx.affinity().affinityTopologyVersion() :
+            ctx.shared().exchange().readyAffinityVersion();
 
         final IgniteCacheExpiryPolicy expiry = skipVals ? null : expiryPolicy(expiryPlc);
 
@@ -971,7 +976,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             taskName,
             deserializePortable,
             expiry,
-            skipVals);
+            skipVals,
+            canRemap);
 
         fut.init();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index 221b230..eb7c78f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -155,7 +155,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         @Nullable UUID subjId,
         String taskName,
         final boolean deserializePortable,
-        final boolean skipVals
+        final boolean skipVals,
+        boolean canRemap
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -183,7 +184,9 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             });
         }
 
-        AffinityTopologyVersion topVer = tx == null ? ctx.affinity().affinityTopologyVersion() : tx.topologyVersion();
+        AffinityTopologyVersion topVer = tx == null ?
+            (canRemap ? ctx.affinity().affinityTopologyVersion() : ctx.shared().exchange().readyAffinityVersion()) :
+            tx.topologyVersion();
 
         subjId = ctx.subjectIdPerCall(subjId, opCtx);
 
@@ -197,7 +200,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             taskName,
             deserializePortable,
             skipVals ? null : expiryPolicy(opCtx != null ? opCtx.expiry() : null),
-            skipVals);
+            skipVals,
+            canRemap);
     }
 
     /** {@inheritDoc} */
@@ -226,7 +230,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
      * @param skipVals Skip values flag.
      * @return Loaded values.
      */
-    public IgniteInternalFuture<Map<K, V>> loadAsync(@Nullable Collection<KeyCacheObject> keys,
+    public IgniteInternalFuture<Map<K, V>> loadAsync(
+        @Nullable Collection<KeyCacheObject> keys,
         boolean readThrough,
         boolean reload,
         boolean forcePrimary,
@@ -235,7 +240,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         String taskName,
         boolean deserializePortable,
         @Nullable IgniteCacheExpiryPolicy expiryPlc,
-        boolean skipVals
+        boolean skipVals,
+        boolean canRemap
     ) {
         if (keys == null || keys.isEmpty())
             return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
@@ -340,7 +346,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             taskName,
             deserializePortable,
             expiryPlc,
-            skipVals);
+            skipVals,
+            canRemap);
 
         fut.init();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index 041f83a..2bf5365 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -364,7 +364,8 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
         @Nullable UUID subjId,
         String taskName,
         boolean deserializePortable,
-        boolean skipVals
+        boolean skipVals,
+        boolean canRemap
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -387,7 +388,8 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
             deserializePortable,
             skipVals ? null : opCtx != null ? opCtx.expiry() : null,
             skipVals,
-            opCtx != null && opCtx.skipStore());
+            opCtx != null && opCtx.skipStore(),
+            canRemap);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index 351d6cd..ba0692c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -195,13 +195,14 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
         return (IgniteInternalFuture)loadAsync(tx,
             keys,
             reload,
-            false,
+            /*force primary*/false,
             subjId,
             taskName,
-            true,
-            null,
+            /*deserialize portable*/true,
+            /*expiry policy*/null,
             skipVals,
-            /*skip store*/false);
+            /*skip store*/false,
+            /*can remap*/true);
     }
 
     /**
@@ -226,7 +227,8 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
         boolean deserializePortable,
         @Nullable ExpiryPolicy expiryPlc,
         boolean skipVal,
-        boolean skipStore
+        boolean skipStore,
+        boolean canRemap
     ) {
         if (F.isEmpty(keys))
             return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
@@ -245,7 +247,8 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
             taskName,
             deserializePortable,
             expiry,
-            skipVal);
+            skipVal,
+            canRemap);
 
         // init() will register future for responses if future has remote mappings.
         fut.init();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/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 194c68a..6f4f15e 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
@@ -333,7 +333,9 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
             true,
             null,
             false,
-            /*skip store*/false).get().get(keyValue(false));
+            /*skip store*/false,
+            /*can remap*/true
+        ).get().get(keyValue(false));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index d109d2b..ca460c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -62,7 +62,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
     private static final int MAX_REMAP_CNT = getInteger(IGNITE_NEAR_GET_MAX_REMAPS, DFLT_MAX_REMAP_CNT);
 
     /** Context. */
-    private GridCacheContext<K, V> cctx;
+    private final GridCacheContext<K, V> cctx;
 
     /** Keys. */
     private Collection<KeyCacheObject> keys;
@@ -106,6 +106,9 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
     /** Expiry policy. */
     private IgniteCacheExpiryPolicy expiryPlc;
 
+    /** Flag indicating that get should be done on a locked topology version. */
+    private final boolean canRemap;
+
     /**
      * @param cctx Context.
      * @param keys Keys.
@@ -131,7 +134,8 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
         String taskName,
         boolean deserializePortable,
         @Nullable IgniteCacheExpiryPolicy expiryPlc,
-        boolean skipVals
+        boolean skipVals,
+        boolean canRemap
     ) {
         super(cctx.kernalContext(), CU.<K, V>mapsReducer(keys.size()));
 
@@ -148,6 +152,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
         this.deserializePortable = deserializePortable;
         this.expiryPlc = expiryPlc;
         this.skipVals = skipVals;
+        this.canRemap = canRemap;
 
         futId = IgniteUuid.randomUuid();
 
@@ -161,7 +166,9 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
      * Initializes future.
      */
     public void init() {
-        AffinityTopologyVersion topVer = tx == null ? cctx.affinity().affinityTopologyVersion() : tx.topologyVersion();
+        AffinityTopologyVersion topVer = tx == null ?
+            (canRemap ? cctx.affinity().affinityTopologyVersion() : cctx.shared().exchange().readyAffinityVersion()) :
+            tx.topologyVersion();
 
         map(keys, Collections.<ClusterNode, LinkedHashMap<KeyCacheObject, Boolean>>emptyMap(), topVer);
 
@@ -327,7 +334,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                             remapKeys.add(key);
                     }
 
-                    AffinityTopologyVersion updTopVer = new AffinityTopologyVersion(cctx.discovery().topologyVersion());
+                    AffinityTopologyVersion updTopVer = cctx.discovery().topologyVersionEx();
 
                     assert updTopVer.compareTo(topVer) > 0 : "Got invalid partitions for local node but topology version did " +
                         "not change [topVer=" + topVer + ", updTopVer=" + updTopVer +
@@ -435,7 +442,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                         taskName,
                         expiryPlc);
 
-                ClusterNode primary = null;
+                ClusterNode affNode = null;
 
                 if (v == null && allowLocRead && cctx.affinityNode()) {
                     GridDhtCacheAdapter<K, V> dht = cache().dht();
@@ -472,16 +479,16 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                                 near.metrics0().onRead(true);
                         }
                         else {
-                            primary = cctx.affinity().primary(key, topVer);
+                            affNode = affinityNode(key, topVer);
 
-                            if (primary == null) {
+                            if (affNode == null) {
                                 onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
                                     "(all partition nodes left the grid)."));
 
                                 return savedVers;
                             }
 
-                            if (!primary.isLocal() && cctx.cache().configuration().isStatisticsEnabled() && !skipVals)
+                            if (!affNode.isLocal() && cctx.cache().configuration().isStatisticsEnabled() && !skipVals)
                                 near.metrics0().onRead(false);
                         }
                     }
@@ -507,10 +514,10 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                     add(new GridFinishedFuture<>(Collections.singletonMap(key0, val0)));
                 }
                 else {
-                    if (primary == null) {
-                        primary = cctx.affinity().primary(key, topVer);
+                    if (affNode == null) {
+                        affNode = affinityNode(key, topVer);
 
-                        if (primary == null) {
+                        if (affNode == null) {
                             onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
                                 "(all partition nodes left the grid)."));
 
@@ -527,13 +534,13 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
 
                     savedVers.put(key, nearEntry == null ? null : nearEntry.dhtVersion());
 
-                    LinkedHashMap<KeyCacheObject, Boolean> keys = mapped.get(primary);
+                    LinkedHashMap<KeyCacheObject, Boolean> keys = mapped.get(affNode);
 
                     if (keys != null && keys.containsKey(key)) {
                         if (remapCnt.incrementAndGet() > MAX_REMAP_CNT) {
                             onDone(new ClusterTopologyCheckedException("Failed to remap key to a new node after " +
                                 MAX_REMAP_CNT + " attempts (key got remapped to the same node) " +
-                                "[key=" + key + ", node=" + U.toShortString(primary) + ", mappings=" + mapped + ']'));
+                                "[key=" + key + ", node=" + U.toShortString(affNode) + ", mappings=" + mapped + ']'));
 
                             return savedVers;
                         }
@@ -545,10 +552,10 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                     if (!addRdr && tx.readCommitted() && !tx.writeSet().contains(cctx.txKey(key)))
                         addRdr = true;
 
-                    LinkedHashMap<KeyCacheObject, Boolean> old = mappings.get(primary);
+                    LinkedHashMap<KeyCacheObject, Boolean> old = mappings.get(affNode);
 
                     if (old == null)
-                        mappings.put(primary, old = new LinkedHashMap<>(3, 1f));
+                        mappings.put(affNode, old = new LinkedHashMap<>(3, 1f));
 
                     old.put(key, addRdr);
                 }
@@ -579,6 +586,28 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
     }
 
     /**
+     * Affinity node to send get request to.
+     *
+     * @param key Key to get.
+     * @param topVer Topology version.
+     * @return Affinity node to get key from.
+     */
+    private ClusterNode affinityNode(KeyCacheObject key, AffinityTopologyVersion topVer) {
+        if (!canRemap) {
+            List<ClusterNode> affNodes = cctx.affinity().nodes(key, topVer);
+
+            for (ClusterNode node : affNodes) {
+                if (cctx.discovery().alive(node))
+                    return node;
+            }
+
+            return null;
+        }
+        else
+            return cctx.affinity().primary(key, topVer);
+    }
+
+    /**
      * @return Near cache.
      */
     private GridNearCacheAdapter<K, V> cache() {
@@ -752,30 +781,38 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
             if (log.isDebugEnabled())
                 log.debug("Remote node left grid while sending or waiting for reply (will retry): " + this);
 
-            final AffinityTopologyVersion updTopVer =
-                new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
+            // Try getting value from alive nodes.
+            if (!canRemap) {
+                // Remap
+                map(keys.keySet(), F.t(node, keys), topVer);
+
+                onDone(Collections.<K, V>emptyMap());
+            } else {
+                final AffinityTopologyVersion updTopVer =
+                    new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
 
-            final GridFutureRemapTimeoutObject timeout = new GridFutureRemapTimeoutObject(this,
-                cctx.kernalContext().config().getNetworkTimeout(),
-                updTopVer,
-                e);
+                final GridFutureRemapTimeoutObject timeout = new GridFutureRemapTimeoutObject(this,
+                    cctx.kernalContext().config().getNetworkTimeout(),
+                    updTopVer,
+                    e);
 
-            cctx.affinity().affinityReadyFuture(updTopVer).listen(
-                new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                    @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
-                        if (timeout.finish()) {
-                            cctx.kernalContext().timeout().removeTimeoutObject(timeout);
+                cctx.affinity().affinityReadyFuture(updTopVer).listen(
+                    new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
+                            if (timeout.finish()) {
+                                cctx.kernalContext().timeout().removeTimeoutObject(timeout);
 
-                            // Remap.
-                            map(keys.keySet(), F.t(node, keys), updTopVer);
+                                // Remap.
+                                map(keys.keySet(), F.t(node, keys), updTopVer);
 
-                            onDone(Collections.<K, V>emptyMap());
+                                onDone(Collections.<K, V>emptyMap());
+                            }
                         }
                     }
-                }
-            );
+                );
 
-            cctx.kernalContext().timeout().addTimeoutObject(timeout);
+                cctx.kernalContext().timeout().addTimeoutObject(timeout);
+            }
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
index 696acfb..a1f1383 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
@@ -101,7 +101,8 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
         @Nullable UUID subjId,
         String taskName,
         final boolean deserializePortable,
-        final boolean skipVals
+        final boolean skipVals,
+        boolean canRemap
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -142,7 +143,8 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
             deserializePortable,
             skipVals ? null : opCtx != null ? opCtx.expiry() : null,
             skipVals,
-            skipStore);
+            skipStore,
+            canRemap);
     }
 
     /**
@@ -172,7 +174,8 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
             tx.resolveTaskName(),
             deserializePortable,
             expiryPlc,
-            skipVals);
+            skipVals,
+            /*can remap*/true);
 
         // init() will register future for responses if it has remote mappings.
         fut.init();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/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 cb391e4..5ff7345 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
@@ -313,7 +313,8 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
             });
         }
         else if (cacheCtx.isColocated()) {
-            return cacheCtx.colocated().loadAsync(keys,
+            return cacheCtx.colocated().loadAsync(
+                keys,
                 readThrough,
                 /*reload*/false,
                 /*force primary*/false,
@@ -322,7 +323,9 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
                 resolveTaskName(),
                 deserializePortable,
                 accessPolicy(cacheCtx, keys),
-                skipVals).chain(new C1<IgniteInternalFuture<Map<Object, Object>>, Boolean>() {
+                skipVals,
+                /*can remap*/true
+            ).chain(new C1<IgniteInternalFuture<Map<Object, Object>>, Boolean>() {
                     @Override public Boolean apply(IgniteInternalFuture<Map<Object, Object>> f) {
                         try {
                             Map<Object, Object> map = f.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index bcbdec4..c648f11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -458,7 +458,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         @Nullable UUID subjId,
         final String taskName,
         final boolean deserializePortable,
-        final boolean skipVals
+        final boolean skipVals,
+        boolean canRemap
     ) {
         A.notNull(keys, "keys");
 
@@ -570,8 +571,18 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         if (success || !storeEnabled)
             return vals;
 
-        return getAllAsync(keys, opCtx == null || !opCtx.skipStore(), null, false, subjId, taskName, deserializePortable,
-            false, expiry, skipVals).get();
+        return getAllAsync(
+            keys,
+            opCtx == null || !opCtx.skipStore(),
+            null,
+            false,
+            subjId,
+            taskName,
+            deserializePortable,
+            /*force primary*/false,
+            expiry,
+            skipVals,
+            /*can remap*/true).get();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java
new file mode 100644
index 0000000..ef031f6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java
@@ -0,0 +1,215 @@
+/*
+ * 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.cache.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static  org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
+/**
+ *
+ */
+public class IgniteCacheTopologySafeGetSelfTest extends GridCommonAbstractTest {
+    /** Number of initial grids. */
+    public static final int GRID_CNT = 4;
+
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** TX commit latch. */
+    private CountDownLatch releaseLatch;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCacheConfiguration(
+            cacheCfg("tx", TRANSACTIONAL, false),
+            cacheCfg("atomic", ATOMIC, false),
+            cacheCfg("tx_near", TRANSACTIONAL, true),
+            cacheCfg("atomic_near", ATOMIC, true));
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /**
+     * @param name Cache name.
+     * @param cacheMode Cache mode.
+     * @param near Near enabled flag.
+     * @return Cache configuration.
+     */
+    @SuppressWarnings("unchecked")
+    private CacheConfiguration cacheCfg(String name, CacheAtomicityMode cacheMode, boolean near) {
+        CacheConfiguration cfg = new CacheConfiguration(name);
+
+        cfg.setAtomicityMode(cacheMode);
+        cfg.setBackups(1);
+
+        if (near)
+            cfg.setNearConfiguration(new NearCacheConfiguration());
+        else
+            cfg.setNearConfiguration(null);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetTopologySafeNodeJoin() throws Exception {
+        checkGetTopologySafeNodeJoin(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetTopologySafeNodeJoinPrimaryLeave() throws Exception {
+        checkGetTopologySafeNodeJoin(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void checkGetTopologySafeNodeJoin(boolean failPrimary) throws Exception {
+        startGrids(GRID_CNT);
+
+        awaitPartitionMapExchange();
+
+        try {
+            ClusterNode targetNode = ignite(1).cluster().localNode();
+
+            info(">>> Target node: " + targetNode.id());
+
+            // Populate caches with a key that does not belong to ignite(0).
+            int key = -1;
+            for (int i = 0; i < 100; i++) {
+                Collection<ClusterNode> nodes = ignite(0).affinity("tx").mapKeyToPrimaryAndBackups(i);
+                ClusterNode primaryNode = F.first(nodes);
+
+                if (!nodes.contains(ignite(0).cluster().localNode()) && primaryNode.id().equals(targetNode.id())) {
+                    ignite(1).cache("tx").put(i, i);
+                    ignite(1).cache("atomic").put(i, i);
+                    ignite(1).cache("tx_near").put(i, i);
+                    ignite(1).cache("atomic_near").put(i, i);
+
+                    key = i;
+
+
+                    break;
+                }
+            }
+
+            assertTrue(key != -1);
+
+            IgniteInternalFuture<?> txFut = startBlockingTxAsync();
+
+            IgniteInternalFuture<?> nodeFut = startNodeAsync();
+
+            if (failPrimary)
+                stopGrid(1);
+
+            assertEquals(key, ((IgniteKernal)ignite(0)).internalCache("tx").getTopologySafe(key));
+            assertEquals(key, ((IgniteKernal)ignite(0)).internalCache("atomic").getTopologySafe(key));
+            assertEquals(key, ((IgniteKernal)ignite(0)).internalCache("tx_near").getTopologySafe(key));
+            assertEquals(key, ((IgniteKernal)ignite(0)).internalCache("atomic_near").getTopologySafe(key));
+
+            releaseTx();
+
+            txFut.get();
+            nodeFut.get();
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    private IgniteInternalFuture<?> startNodeAsync() throws Exception {
+        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override
+            public Object call() throws Exception {
+                startGrid(GRID_CNT);
+
+                return null;
+            }
+        });
+
+        U.sleep(1000);
+
+        return fut;
+    }
+
+    /**
+     * @return TX release future.
+     * @throws Exception If failed.
+     */
+    private IgniteInternalFuture<?> startBlockingTxAsync() throws Exception {
+        final CountDownLatch lockLatch = new CountDownLatch(1);
+
+        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try (Transaction ignore = ignite(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                    for (int i = 0; i < 30; i++)
+                        ignite(0).cache("tx").get("value-" + i);
+
+                    releaseLatch = new CountDownLatch(1);
+
+                    lockLatch.countDown();
+
+                    releaseLatch.await();
+                }
+
+                return null;
+            }
+        });
+
+        lockLatch.await();
+
+        return fut;
+    }
+
+    /**
+     *
+     */
+    private void releaseTx() {
+        assert releaseLatch != null;
+
+        releaseLatch.countDown();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index 9c4446d..c2fc46c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@ -120,6 +120,8 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
 
             stopGrid(stopIdx);
 
+            U.sleep(500);
+
             startGrid(stopIdx);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b93ee7a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
index af2b85c..b64471b 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
@@ -80,6 +80,8 @@ public class IgniteCacheFailoverTestSuite extends TestSuite {
 
         suite.addTestSuite(IgniteCacheSizeFailoverTest.class);
 
+        suite.addTestSuite(IgniteCacheTopologySafeGetSelfTest.class);
+
         return suite;
     }
 }


[21/34] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-1.3.3-p3' into ignite-1.3.3-p3

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


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

Branch: refs/heads/master
Commit: 2054f197fbb12ab6671699f7b4819d2ef1b82b08
Parents: 05fda0c 2673eca
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 20 13:14:03 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 20 13:14:03 2015 +0300

----------------------------------------------------------------------
 .../store/jdbc/CacheAbstractJdbcStore.java      |  45 ++++-----
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  20 +++-
 .../store/jdbc/dialect/BasicJdbcDialect.java    |   3 +
 .../cache/store/jdbc/dialect/DB2Dialect.java    |   3 +
 .../cache/store/jdbc/dialect/H2Dialect.java     |   3 +
 .../cache/store/jdbc/dialect/JdbcDialect.java   |   3 +-
 .../cache/store/jdbc/dialect/MySQLDialect.java  |   3 +
 .../cache/store/jdbc/dialect/OracleDialect.java |   3 +
 .../store/jdbc/dialect/SQLServerDialect.java    |   3 +
 .../config/store/jdbc/ignite-type-metadata.xml  |   8 ++
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |  11 +-
 ...eJdbcStoreAbstractMultithreadedSelfTest.java |  16 +--
 .../ignite/cache/store/jdbc/model/Person.java   |  26 ++++-
 .../ignite/schema/model/PojoDescriptor.java     |   6 +-
 .../parser/dialect/OracleMetadataDialect.java   | 101 ++++++++++---------
 .../ignite/schema/ui/SchemaImportApp.java       |  36 +++++--
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |  17 ++--
 17 files changed, 198 insertions(+), 109 deletions(-)
----------------------------------------------------------------------



[18/34] incubator-ignite git commit: IGNITE-843: Fixed POJO store null value and made dialects serializable.

Posted by sb...@apache.org.
IGNITE-843: Fixed POJO store null value and made dialects serializable.


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

Branch: refs/heads/master
Commit: 939afff97f007d53f1fdee11e974beca4b75bf2f
Parents: c04fcea
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Aug 20 16:53:18 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 20 16:53:18 2015 +0700

----------------------------------------------------------------------
 .../store/jdbc/CacheAbstractJdbcStore.java      |  45 ++++-----
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  20 +++-
 .../store/jdbc/dialect/BasicJdbcDialect.java    |   3 +
 .../cache/store/jdbc/dialect/DB2Dialect.java    |   3 +
 .../cache/store/jdbc/dialect/H2Dialect.java     |   3 +
 .../cache/store/jdbc/dialect/JdbcDialect.java   |   3 +-
 .../cache/store/jdbc/dialect/MySQLDialect.java  |   3 +
 .../cache/store/jdbc/dialect/OracleDialect.java |   3 +
 .../store/jdbc/dialect/SQLServerDialect.java    |   3 +
 .../config/store/jdbc/ignite-type-metadata.xml  |   8 ++
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |  11 +-
 ...eJdbcStoreAbstractMultithreadedSelfTest.java |  16 +--
 .../ignite/cache/store/jdbc/model/Person.java   |  26 ++++-
 .../ignite/schema/model/PojoDescriptor.java     |   6 +-
 .../parser/dialect/OracleMetadataDialect.java   | 101 ++++++++++---------
 .../ignite/schema/ui/SchemaImportApp.java       |  36 +++++--
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |  17 ++--
 17 files changed, 198 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
index b1e223b..b2be8c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
@@ -355,6 +355,11 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
      * @throws SQLException If a database access error occurs or this method is called.
      */
     protected Object getColumnValue(ResultSet rs, int colIdx, Class<?> type) throws SQLException {
+        Object val = rs.getObject(colIdx);
+
+        if (val == null)
+            return null;
+
         if (type == int.class)
             return rs.getInt(colIdx);
 
@@ -364,7 +369,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
         if (type == double.class)
             return rs.getDouble(colIdx);
 
-        if (type == boolean.class)
+        if (type == boolean.class || type == Boolean.class)
             return rs.getBoolean(colIdx);
 
         if (type == byte.class)
@@ -378,31 +383,23 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
 
         if (type == Integer.class || type == Long.class || type == Double.class ||
             type == Byte.class || type == Short.class ||  type == Float.class) {
-            Object val = rs.getObject(colIdx);
-
-            if (val != null) {
-                Number num = (Number)val;
-
-                if (type == Integer.class)
-                    return num.intValue();
-                else if (type == Long.class)
-                    return num.longValue();
-                else if (type == Double.class)
-                    return num.doubleValue();
-                else if (type == Byte.class)
-                    return num.byteValue();
-                else if (type == Short.class)
-                    return num.shortValue();
-                else if (type == Float.class)
-                    return num.floatValue();
-            }
-            else
-                return EMPTY_COLUMN_VALUE;
+            Number num = (Number)val;
+
+            if (type == Integer.class)
+                return num.intValue();
+            else if (type == Long.class)
+                return num.longValue();
+            else if (type == Double.class)
+                return num.doubleValue();
+            else if (type == Byte.class)
+                return num.byteValue();
+            else if (type == Short.class)
+                return num.shortValue();
+            else if (type == Float.class)
+                return num.floatValue();
         }
 
-        Object val = rs.getObject(colIdx);
-
-        if (type == UUID.class && val != null) {
+        if (type == UUID.class) {
             if (val instanceof UUID)
                 return val;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
index 7b78bda..428485f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
@@ -167,15 +167,25 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
             Object obj = mc.ctor.newInstance();
 
             for (CacheTypeFieldMetadata field : fields) {
-                Method setter = mc.setters.get(field.getJavaName());
+                String fldJavaName = field.getJavaName();
+
+                Method setter = mc.setters.get(fldJavaName);
 
                 if (setter == null)
-                    throw new CacheLoaderException("Failed to find setter in POJO class [class name=" + typeName +
-                        ", property=" + field.getJavaName() + "]");
+                    throw new IllegalStateException("Failed to find setter in POJO class [class name=" + typeName +
+                        ", property=" + fldJavaName + "]");
+
+                String fldDbName = field.getDatabaseName();
 
-                Integer colIdx = loadColIdxs.get(field.getDatabaseName());
+                Integer colIdx = loadColIdxs.get(fldDbName);
 
-                setter.invoke(obj, getColumnValue(rs, colIdx, field.getJavaType()));
+                try {
+                    setter.invoke(obj, getColumnValue(rs, colIdx, field.getJavaType()));
+                }
+                catch (Exception e) {
+                    throw new IllegalStateException("Failed to set property in POJO class [class name=" + typeName +
+                        ", property=" + fldJavaName + ", column=" + colIdx + ", db name=" + fldDbName + "]", e);
+                }
             }
 
             return (R)obj;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
index d0dd6f4..b43c7d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
@@ -26,6 +26,9 @@ import java.util.*;
  * Basic implementation of dialect based on JDBC specification.
  */
 public class BasicJdbcDialect implements JdbcDialect {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** Default max query parameters count. */
     protected static final int DFLT_MAX_PARAMS_CNT = 2000;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java
index fe1d876..2a08557 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java
@@ -25,6 +25,9 @@ import java.util.*;
  * A dialect compatible with the IBM DB2 database.
  */
 public class DB2Dialect extends BasicJdbcDialect {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** {@inheritDoc} */
     @Override public boolean hasMerge() {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java
index a97e144..8091e1a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java
@@ -25,6 +25,9 @@ import java.util.*;
  * A dialect compatible with the H2 database.
  */
 public class H2Dialect extends BasicJdbcDialect {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** {@inheritDoc} */
     @Override public boolean hasMerge() {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java
index be1cc67..32adcc4 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java
@@ -17,12 +17,13 @@
 
 package org.apache.ignite.cache.store.jdbc.dialect;
 
+import java.io.*;
 import java.util.*;
 
 /**
  * Represents a dialect of SQL implemented by a particular RDBMS.
  */
-public interface JdbcDialect {
+public interface JdbcDialect extends Serializable {
     /**
      * Construct select count query.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java
index df16841..def2fe7 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java
@@ -25,6 +25,9 @@ import java.util.*;
  * A dialect compatible with the MySQL database.
  */
 public class MySQLDialect extends BasicJdbcDialect {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** {@inheritDoc} */
     @Override public boolean hasMerge() {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java
index 351f10a..e155fb4 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java
@@ -25,6 +25,9 @@ import java.util.*;
  * A dialect compatible with the Oracle database.
  */
 public class OracleDialect extends BasicJdbcDialect {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** {@inheritDoc} */
     @Override public boolean hasMerge() {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java
index e781e98..7fdda6b 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java
@@ -25,6 +25,9 @@ import java.util.*;
  * A dialect compatible with the Microsoft SQL Server database.
  */
 public class SQLServerDialect extends BasicJdbcDialect {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** {@inheritDoc} */
     @Override public boolean hasMerge() {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/test/config/store/jdbc/ignite-type-metadata.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/store/jdbc/ignite-type-metadata.xml b/modules/core/src/test/config/store/jdbc/ignite-type-metadata.xml
index 91d77cd..3e3d6e0 100644
--- a/modules/core/src/test/config/store/jdbc/ignite-type-metadata.xml
+++ b/modules/core/src/test/config/store/jdbc/ignite-type-metadata.xml
@@ -180,6 +180,14 @@
                     <property name="javaName" value="name"/>
                     <property name="javaType" value="java.lang.String"/>
                 </bean>
+                <bean class="org.apache.ignite.cache.CacheTypeFieldMetadata">
+                    <property name="databaseName" value="salary"/>
+                    <property name="databaseType">
+                        <util:constant static-field="java.sql.Types.INTEGER"/>
+                    </property>
+                    <property name="javaName" value="salary"/>
+                    <property name="javaType" value="java.lang.Integer"/>
+                </bean>
             </list>
         </property>
     </bean>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
index 182d3bc..b1efb0d 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
@@ -188,7 +188,7 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
         stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Timestamp_Entries (key timestamp not null, val integer, PRIMARY KEY(key))");
         stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Organization (id integer not null, name varchar(50), city varchar(50), PRIMARY KEY(id))");
         stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Person (id integer not null, org_id integer, name varchar(50), PRIMARY KEY(id))");
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Person_Complex (id integer not null, org_id integer not null, city_id integer not null, name varchar(50), PRIMARY KEY(id))");
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Person_Complex (id integer not null, org_id integer not null, city_id integer not null, name varchar(50), salary integer, PRIMARY KEY(id))");
 
         conn.commit();
 
@@ -238,7 +238,7 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
 
         U.closeQuiet(prnStmt);
 
-        PreparedStatement prnComplexStmt = conn.prepareStatement("INSERT INTO Person_Complex(id, org_id, city_id, name) VALUES (?, ?, ?, ?)");
+        PreparedStatement prnComplexStmt = conn.prepareStatement("INSERT INTO Person_Complex(id, org_id, city_id, name, salary) VALUES (?, ?, ?, ?, ?)");
 
         for (int i = 0; i < PERSON_CNT; i++) {
             prnComplexStmt.setInt(1, i);
@@ -246,6 +246,11 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
             prnComplexStmt.setInt(3, i % 100);
             prnComplexStmt.setString(4, "name" + i);
 
+            if (i > 0)
+                prnComplexStmt.setInt(5, 1000 + i * 500);
+            else // Add person with null salary
+                prnComplexStmt.setNull(5, java.sql.Types.INTEGER);
+
             prnComplexStmt.addBatch();
         }
 
@@ -274,7 +279,7 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
 
                     assert key.getId() == val.getId();
                     assert key.getOrgId() == val.getOrgId();
-                    assert ("name"  + key.getId()).equals(val.getName());
+                    assertEquals("name"  + key.getId(), val.getName());
 
                     prnComplexKeys.add((PersonComplexKey)k);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreAbstractMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreAbstractMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreAbstractMultithreadedSelfTest.java
index eac7669..9483545 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreAbstractMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreAbstractMultithreadedSelfTest.java
@@ -190,7 +190,7 @@ public abstract class CacheJdbcStoreAbstractMultithreadedSelfTest<T extends Cach
                     if (rnd.nextBoolean())
                         cache.put(new OrganizationKey(id), new Organization(id, "Name" + id, "City" + id));
                     else
-                        cache.put(new PersonKey(id), new Person(id, rnd.nextInt(), "Name" + id));
+                        cache.put(new PersonKey(id), new Person(id, rnd.nextInt(), "Name" + id, 1));
                 }
 
                 return null;
@@ -209,7 +209,7 @@ public abstract class CacheJdbcStoreAbstractMultithreadedSelfTest<T extends Cach
                     if (rnd.nextBoolean())
                         cache.putIfAbsent(new OrganizationKey(id), new Organization(id, "Name" + id, "City" + id));
                     else
-                        cache.putIfAbsent(new PersonKey(id), new Person(id, rnd.nextInt(), "Name" + id));
+                        cache.putIfAbsent(new PersonKey(id), new Person(id, rnd.nextInt(), "Name" + id, i));
                 }
 
                 return null;
@@ -248,7 +248,7 @@ public abstract class CacheJdbcStoreAbstractMultithreadedSelfTest<T extends Cach
                         if (rnd.nextBoolean())
                             map.put(new OrganizationKey(id), new Organization(id, "Name" + id, "City" + id));
                         else
-                            map.put(new PersonKey(id), new Person(id, rnd.nextInt(), "Name" + id));
+                            map.put(new PersonKey(id), new Person(id, rnd.nextInt(), "Name" + id, 1));
                     }
 
                     IgniteCache<Object, Object> cache = jcache();
@@ -273,17 +273,17 @@ public abstract class CacheJdbcStoreAbstractMultithreadedSelfTest<T extends Cach
                     IgniteCache<PersonKey, Person> cache = jcache();
 
                     try (Transaction tx = grid().transactions().txStart()) {
-                        cache.put(new PersonKey(1), new Person(1, rnd.nextInt(), "Name" + 1));
-                        cache.put(new PersonKey(2), new Person(2, rnd.nextInt(), "Name" + 2));
-                        cache.put(new PersonKey(3), new Person(3, rnd.nextInt(), "Name" + 3));
+                        cache.put(new PersonKey(1), new Person(1, rnd.nextInt(), "Name" + 1, 1));
+                        cache.put(new PersonKey(2), new Person(2, rnd.nextInt(), "Name" + 2, 2));
+                        cache.put(new PersonKey(3), new Person(3, rnd.nextInt(), "Name" + 3, 3));
 
                         cache.get(new PersonKey(1));
                         cache.get(new PersonKey(4));
 
                         Map<PersonKey, Person> map =  U.newHashMap(2);
 
-                        map.put(new PersonKey(5), new Person(5, rnd.nextInt(), "Name" + 5));
-                        map.put(new PersonKey(6), new Person(6, rnd.nextInt(), "Name" + 6));
+                        map.put(new PersonKey(5), new Person(5, rnd.nextInt(), "Name" + 5, 5));
+                        map.put(new PersonKey(6), new Person(6, rnd.nextInt(), "Name" + 6, 6));
 
                         cache.putAll(map);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
index 1c4b9a7..95c83b9 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
@@ -37,6 +37,9 @@ public class Person implements Serializable {
     /** Value for name. */
     private String name;
 
+    /** Value for salary. */
+    private Integer salary;
+
     /**
      * Empty constructor.
      */
@@ -50,11 +53,13 @@ public class Person implements Serializable {
     public Person(
         Integer id,
         Integer orgId,
-        String name
+        String name,
+        Integer salary
     ) {
         this.id = id;
         this.orgId = orgId;
         this.name = name;
+        this.salary = salary;
     }
 
     /**
@@ -111,6 +116,25 @@ public class Person implements Serializable {
         this.name = name;
     }
 
+
+    /**
+     * Gets salary.
+     *
+     * @return Value for salary.
+     */
+    public Integer getSalary() {
+        return salary;
+    }
+
+    /**
+     * Sets salary.
+     *
+     * @param salary New value for salary.
+     */
+    public void setSalary(Integer salary) {
+        this.salary = salary;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         if (this == o)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
index 5e1da06..5a4a1fa 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
@@ -504,7 +504,11 @@ public class PojoDescriptor {
             case TIMESTAMP:
                 return java.sql.Timestamp.class;
 
-            // BINARY, VARBINARY, LONGVARBINARY, ARRAY, BLOB, CLOB, NCLOB, NULL, DATALINK
+            case CLOB:
+            case NCLOB:
+                return String.class;
+
+            // BINARY, VARBINARY, LONGVARBINARY, ARRAY, BLOB, NULL, DATALINK
             // OTHER, JAVA_OBJECT, DISTINCT, STRUCT, REF, ROWID, SQLXML
             default:
                 return Object.class;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
index 860ff68..30dda5d 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
@@ -87,73 +87,76 @@ public class OracleMetadataDialect extends DatabaseMetadataDialect {
      * @throws SQLException If failed to decode type.
      */
     private int decodeType(ResultSet rs) throws SQLException {
-        switch (rs.getString(DATA_TYPE_IDX)) {
-            case "CHAR":
-            case "NCHAR":
-                return CHAR;
+        String type = rs.getString(DATA_TYPE_IDX);
 
-            case "VARCHAR2":
-            case "NVARCHAR2":
-                return VARCHAR;
+        if (type.startsWith("TIMESTAMP"))
+            return TIMESTAMP;
+        else {
+            switch (type) {
+                case "CHAR":
+                case "NCHAR":
+                    return CHAR;
 
-            case "LONG":
-                return LONGVARCHAR;
+                case "VARCHAR2":
+                case "NVARCHAR2":
+                    return VARCHAR;
 
-            case "LONG RAW":
-                return LONGVARBINARY;
+                case "LONG":
+                    return LONGVARCHAR;
 
-            case "FLOAT":
-                return FLOAT;
+                case "LONG RAW":
+                    return LONGVARBINARY;
 
-            case "NUMBER":
-                int precision = rs.getInt(DATA_PRECISION_IDX);
-                int scale = rs.getInt(DATA_SCALE_IDX);
+                case "FLOAT":
+                    return FLOAT;
 
-                if (scale > 0) {
-                    if (scale < 4 && precision < 19)
-                        return FLOAT;
+                case "NUMBER":
+                    int precision = rs.getInt(DATA_PRECISION_IDX);
+                    int scale = rs.getInt(DATA_SCALE_IDX);
 
-                    if (scale > 4 || precision > 19)
-                        return DOUBLE;
+                    if (scale > 0) {
+                        if (scale < 4 && precision < 19)
+                            return FLOAT;
 
-                    return NUMERIC;
-                }
-                else {
-                    if (precision < 1)
-                        return INTEGER;
+                        if (scale > 4 || precision > 19)
+                            return DOUBLE;
 
-                    if (precision < 2)
-                        return BOOLEAN;
+                        return NUMERIC;
+                    }
+                    else {
+                        if (precision < 1)
+                            return INTEGER;
 
-                    if (precision < 4)
-                        return TINYINT;
+                        if (precision < 2)
+                            return BOOLEAN;
 
-                    if (precision < 6)
-                        return SMALLINT;
+                        if (precision < 4)
+                            return TINYINT;
 
-                    if (precision < 11)
-                        return INTEGER;
+                        if (precision < 6)
+                            return SMALLINT;
 
-                    if (precision < 20)
-                        return BIGINT;
+                        if (precision < 11)
+                            return INTEGER;
 
-                    return NUMERIC;
-                }
+                        if (precision < 20)
+                            return BIGINT;
 
-            case "DATE":
-                return DATE;
+                        return NUMERIC;
+                    }
 
-            case "TIMESTAMP":
-                return TIMESTAMP;
+                case "DATE":
+                    return DATE;
 
-            case "BFILE":
-            case "BLOB":
-                return BLOB;
+                case "BFILE":
+                case "BLOB":
+                    return BLOB;
 
-            case "CLOB":
-            case "NCLOB":
-            case "XMLTYPE":
-                return CLOB;
+                case "CLOB":
+                case "NCLOB":
+                case "XMLTYPE":
+                    return CLOB;
+            }
         }
 
         return OTHER;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
index 00d1bc8..7b9c220 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
@@ -109,7 +109,7 @@ public class SchemaImportApp extends Application {
             "jdbc:mysql://[host]:[port]/[database]", "root"),
         new Preset("mssql", "Microsoft SQL Server", "sqljdbc41.jar", "com.microsoft.sqlserver.jdbc.SQLServerDriver",
             "jdbc:sqlserver://[host]:[port][;databaseName=database]", "sa"),
-        new Preset("posgresql", "PostgreSQL", "postgresql-9.3.jdbc4.jar", "org.postgresql.Driver",
+        new Preset("postgresql", "PostgreSQL", "postgresql-9.3.jdbc4.jar", "org.postgresql.Driver",
             "jdbc:postgresql://[host]:[port]/[database]", "sa"),
         new Preset("custom", "Custom server...", "custom-jdbc.jar", "org.custom.Driver", "jdbc:custom", "sa")
     };
@@ -444,6 +444,11 @@ public class SchemaImportApp extends Application {
         final boolean singleXml = xmlSingleFileCh.isSelected();
 
         Runnable task = new Task<Void>() {
+            /**
+             * @param pojo POJO descriptor to check.
+             * @param selected Selected flag.
+             * @param msg Message to show in case of check failed.
+             */
             private void checkEmpty(final PojoDescriptor pojo, boolean selected, String msg) {
                 if (!selected) {
                     Platform.runLater(new Runnable() {
@@ -694,6 +699,8 @@ public class SchemaImportApp extends Application {
 
     /**
      * Create connection pane with controls.
+     *
+     * @return Pane with connection controls.
      */
     private Pane createConnectionPane() {
         connPnl = paneEx(10, 10, 0, 10);
@@ -1110,10 +1117,10 @@ public class SchemaImportApp extends Application {
                 if (curPojo != null) {
                     TableView.TableViewSelectionModel<PojoDescriptor> selMdl = pojosTbl.getSelectionModel();
 
-                    List<Integer> idxs = new ArrayList<>(selMdl.getSelectedIndices());
+                    List<Integer> selIndices = new ArrayList<>(selMdl.getSelectedIndices());
 
-                    if (idxs.size() > 1) {
-                        for (Integer idx : idxs) {
+                    if (selIndices.size() > 1) {
+                        for (Integer idx : selIndices) {
                             if (pojos.get(idx) != curPojo)
                                 selMdl.clearSelection(idx);
                         }
@@ -1242,6 +1249,7 @@ public class SchemaImportApp extends Application {
      *
      * @param key Property key.
      * @param dflt Default value.
+     * @return Property value as string.
      */
     private String getStringProp(String key, String dflt) {
         String val = prefs.getProperty(key);
@@ -1267,6 +1275,7 @@ public class SchemaImportApp extends Application {
      *
      * @param key Property key.
      * @param dflt Default value.
+     * @return Property value as int.
      */
     private int getIntProp(String key, int dflt) {
         String val = prefs.getProperty(key);
@@ -1297,6 +1306,7 @@ public class SchemaImportApp extends Application {
      *
      * @param key Property key.
      * @param dflt Default value.
+     * @return Property value as boolean.
      */
     private boolean getBoolProp(String key, boolean dflt) {
         String val = prefs.getProperty(key);
@@ -1563,7 +1573,11 @@ public class SchemaImportApp extends Application {
         /** Combo box. */
         private final ComboBox<String> comboBox;
 
-        /** Creates a ComboBox cell factory for use in TableColumn controls. */
+        /**
+         * Creates a ComboBox cell factory for use in TableColumn controls.
+         *
+         * @return Cell factory for cell with java types combobox.
+         */
         public static Callback<TableColumn<PojoField, String>, TableCell<PojoField, String>> cellFactory() {
             return new Callback<TableColumn<PojoField, String>, TableCell<PojoField, String>>() {
                 @Override public TableCell<PojoField, String> call(TableColumn<PojoField, String> col) {
@@ -1637,7 +1651,11 @@ public class SchemaImportApp extends Application {
      * Special table cell to select schema or table.
      */
     private static class PojoDescriptorCell extends TableCell<PojoDescriptor, Boolean> {
-        /** Creates a ComboBox cell factory for use in TableColumn controls. */
+        /**
+         * Creates a ComboBox cell factory for use in TableColumn controls.
+         *
+         * @return Cell factory for schema / table selection.
+         */
         public static Callback<TableColumn<PojoDescriptor, Boolean>, TableCell<PojoDescriptor, Boolean>> cellFactory() {
             return new Callback<TableColumn<PojoDescriptor, Boolean>, TableCell<PojoDescriptor, Boolean>>() {
                 @Override public TableCell<PojoDescriptor, Boolean> call(TableColumn<PojoDescriptor, Boolean> col) {
@@ -1694,7 +1712,11 @@ public class SchemaImportApp extends Application {
      * Special table cell to select &quot;used&quot; fields for code generation.
      */
     private static class PojoFieldUseCell extends TableCell<PojoField, Boolean> {
-        /** Creates a ComboBox cell factory for use in TableColumn controls. */
+        /**
+         * Creates a ComboBox cell factory for use in TableColumn controls.
+         *
+         * @return Cell factory for used fields selection.
+         */
         public static Callback<TableColumn<PojoField, Boolean>, TableCell<PojoField, Boolean>> cellFactory() {
             return new Callback<TableColumn<PojoField, Boolean>, TableCell<PojoField, Boolean>>() {
                 @Override public TableCell<PojoField, Boolean> call(TableColumn<PojoField, Boolean> col) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/939afff9/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
index c456339..faaf28f 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
@@ -57,17 +57,14 @@ public class CacheJdbcPojoStoreFactorySelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testSerializable() throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                try (Ignite ignite = Ignition.start("modules/spring/src/test/config/node.xml")) {
-                    try (IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheConfigurationH2Dialect())) {
-                        checkStore(cache, JdbcDataSource.class);
-                    }
-                }
-
-                return null;
+        try (Ignite ignite = Ignition.start("modules/spring/src/test/config/node.xml")) {
+            try (IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheConfigurationH2Dialect())) {
+                checkStore(cache, JdbcDataSource.class);
             }
-        }, CacheException.class, "Failed to validate cache configuration. Cache store factory is not serializable.");
+        }
+        catch (Exception e) {
+            fail("Failed to validate cache configuration. Cache store factory is not serializable.");
+        }
     }
 
     /**



[24/34] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-1.3.3-p3' into ignite-1.3.3-p3

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


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

Branch: refs/heads/master
Commit: 3acc01471c613b7b6de4c4c2b20980db2d589c52
Parents: d93e1db 2054f19
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Aug 20 17:44:39 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 20 17:44:39 2015 +0700

----------------------------------------------------------------------
 .../cache/CrossCacheTxRandomOperationsTest.java | 490 +++++++++++++++++++
 ...gniteCachePutRetryTransactionalSelfTest.java |  15 +-
 2 files changed, 500 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[31/34] incubator-ignite git commit: # ignite-1.3.3-p3 fixed CachePartialUpdateCheckedException

Posted by sb...@apache.org.
# ignite-1.3.3-p3 fixed CachePartialUpdateCheckedException


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

Branch: refs/heads/master
Commit: 9603b251ab22226747a384d0e220580a69c44d31
Parents: d31a8df
Author: sboikov <sb...@gridgain.com>
Authored: Fri Aug 21 09:35:29 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Aug 21 09:35:29 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9603b251/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
index fc846f1..ab38e5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
@@ -49,7 +49,7 @@ public class CachePartialUpdateCheckedException extends IgniteCheckedException {
      */
     @SuppressWarnings("unchecked")
     public synchronized <K> Collection<K> failedKeys() {
-        return new HashSet<>((Collection<K>)failedKeys);
+        return new LinkedHashSet<>((Collection<K>)failedKeys);
     }
 
     /**


[26/34] incubator-ignite git commit: # ignite-1.3.3-p3 exchange manager can return null affinityReadyFuture

Posted by sb...@apache.org.
# ignite-1.3.3-p3 exchange manager can return null affinityReadyFuture


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

Branch: refs/heads/master
Commit: d78e8bcc820268f7244dea021258978d46d4b411
Parents: 2054f19
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 20 14:25:11 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 20 14:25:11 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheSharedContext.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d78e8bcc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index f7763ec..6562257 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -455,7 +455,9 @@ public class GridCacheSharedContext<K, V> {
 
         AffinityTopologyVersion nextVer = new AffinityTopologyVersion(curVer.topologyVersion() + 1);
 
-        return exchMgr.affinityReadyFuture(nextVer);
+        IgniteInternalFuture<?> fut = exchMgr.affinityReadyFuture(nextVer);
+
+        return fut == null ? new GridFinishedFuture<>() : fut;
     }
 
     /**


[12/34] incubator-ignite git commit: IGNITE-1265 - Corrected assertion in marshaller cache.

Posted by sb...@apache.org.
IGNITE-1265 - Corrected assertion in marshaller 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/d384d290
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d384d290
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d384d290

Branch: refs/heads/master
Commit: d384d2901250f64e470feda9e4b34763b9420976
Parents: 0041737
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Aug 19 11:47:19 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Aug 19 11:47:19 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/MarshallerContextImpl.java  | 22 ++++++++++++--------
 1 file changed, 13 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d384d290/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 9f7c983..87bd3b6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.plugin.*;
 
@@ -177,18 +178,21 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
         @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends String>> events)
             throws CacheEntryListenerException {
             for (CacheEntryEvent<? extends Integer, ? extends String> evt : events) {
-                assert evt.getOldValue() == null : "Received non-null old value for system marshaller cache: " + evt;
+                assert evt.getOldValue() == null || F.eq(evt.getOldValue(), evt.getValue()):
+                    "Received cache entry update for system marshaller cache: " + evt;
 
-                File file = new File(workDir, evt.getKey() + ".classname");
+                if (evt.getOldValue() == null) {
+                    File file = new File(workDir, evt.getKey() + ".classname");
 
-                try (Writer writer = new FileWriter(file)) {
-                    writer.write(evt.getValue());
+                    try (Writer writer = new FileWriter(file)) {
+                        writer.write(evt.getValue());
 
-                    writer.flush();
-                }
-                catch (IOException e) {
-                    U.error(log, "Failed to write class name to file [id=" + evt.getKey() +
-                        ", clsName=" + evt.getValue() + ", file=" + file.getAbsolutePath() + ']', e);
+                        writer.flush();
+                    }
+                    catch (IOException e) {
+                        U.error(log, "Failed to write class name to file [id=" + evt.getKey() +
+                            ", clsName=" + evt.getValue() + ", file=" + file.getAbsolutePath() + ']', e);
+                    }
                 }
             }
         }


[13/34] incubator-ignite git commit: IGNITE-1265 - Limit the node log output in a case of partition exchange timeout.

Posted by sb...@apache.org.
IGNITE-1265 - Limit the node log output in a case of partition exchange timeout.


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

Branch: refs/heads/master
Commit: 8ced20733da65bbd224d7dea61ab09d8dcfa2efc
Parents: d384d29
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Aug 19 12:21:24 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Aug 19 12:21:24 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  3 ++
 .../GridDhtPartitionsExchangeFuture.java        | 32 +++++++++++++++-----
 2 files changed, 27 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8ced2073/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 7e96b29..7c808df 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -354,6 +354,9 @@ public final class IgniteSystemProperties {
     /** Number of cache operation retries in case of topology exceptions. */
     public static final String IGNITE_CACHE_RETRIES_COUNT = "IGNITE_CACHE_RETRIES_COUNT";
 
+    /** Number of times pending cache objects will be dumped to the log in case of partition exchange timeout. */
+    public static final String IGNITE_DUMP_PENDING_OBJECTS_THRESHOLD = "IGNITE_DUMP_PENDING_OBJECTS_THRESHOLD";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8ced2073/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 5701749..4971ca6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -54,6 +54,10 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.*;
 public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityTopologyVersion>
     implements Comparable<GridDhtPartitionsExchangeFuture>, GridDhtTopologyFuture {
     /** */
+    private static final int DUMP_PENDING_OBJECTS_THRESHOLD =
+        IgniteSystemProperties.getInteger(IgniteSystemProperties.IGNITE_DUMP_PENDING_OBJECTS_THRESHOLD, 10);
+
+    /** */
     private static final long serialVersionUID = 0L;
 
     /** Dummy flag. */
@@ -722,6 +726,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 if (log.isDebugEnabled())
                     log.debug("Before waiting for partition release future: " + this);
 
+                int dumpedObjects = 0;
+
                 while (true) {
                     try {
                         partReleaseFut.get(2 * cctx.gridConfig().getNetworkTimeout(), TimeUnit.MILLISECONDS);
@@ -730,7 +736,11 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                     }
                     catch (IgniteFutureTimeoutCheckedException ignored) {
                         // Print pending transactions and locks that might have led to hang.
-                        dumpPendingObjects();
+                        if (dumpedObjects < DUMP_PENDING_OBJECTS_THRESHOLD) {
+                            dumpPendingObjects();
+
+                            dumpedObjects++;
+                        }
                     }
                 }
 
@@ -742,6 +752,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
                 IgniteInternalFuture<?> locksFut = cctx.mvcc().finishLocks(exchId.topologyVersion());
 
+                dumpedObjects = 0;
+
                 while (true) {
                     try {
                         locksFut.get(2 * cctx.gridConfig().getNetworkTimeout(), TimeUnit.MILLISECONDS);
@@ -749,16 +761,20 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                         break;
                     }
                     catch (IgniteFutureTimeoutCheckedException ignored) {
-                        U.warn(log, "Failed to wait for locks release future. " +
-                            "Dumping pending objects that might be the cause: " + cctx.localNodeId());
+                        if (dumpedObjects < DUMP_PENDING_OBJECTS_THRESHOLD) {
+                            U.warn(log, "Failed to wait for locks release future. " +
+                                "Dumping pending objects that might be the cause: " + cctx.localNodeId());
 
-                        U.warn(log, "Locked entries:");
+                            U.warn(log, "Locked entries:");
 
-                        Map<IgniteTxKey, Collection<GridCacheMvccCandidate>> locks =
-                            cctx.mvcc().unfinishedLocks(exchId.topologyVersion());
+                            Map<IgniteTxKey, Collection<GridCacheMvccCandidate>> locks =
+                                cctx.mvcc().unfinishedLocks(exchId.topologyVersion());
 
-                        for (Map.Entry<IgniteTxKey, Collection<GridCacheMvccCandidate>> e : locks.entrySet())
-                            U.warn(log, "Locked entry [key=" + e.getKey() + ", mvcc=" + e.getValue() + ']');
+                            for (Map.Entry<IgniteTxKey, Collection<GridCacheMvccCandidate>> e : locks.entrySet())
+                                U.warn(log, "Locked entry [key=" + e.getKey() + ", mvcc=" + e.getValue() + ']');
+
+                            dumpedObjects++;
+                        }
                     }
                 }
 


[17/34] incubator-ignite git commit: # ignite-1.3.3-p3 retryReadyFuture can be null for ClusterTopologyCheckedException

Posted by sb...@apache.org.
# ignite-1.3.3-p3 retryReadyFuture can be null for ClusterTopologyCheckedException


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

Branch: refs/heads/master
Commit: ef1495006fc011e998462ef14804efabdfeb7020
Parents: 5e81456
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 20 11:06:00 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 20 11:06:00 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/internal/util/IgniteUtils.java    | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef149500/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index e259084..404c806 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -585,7 +585,8 @@ public abstract class IgniteUtils {
 
                 ClusterTopologyCheckedException checked = (ClusterTopologyCheckedException)e;
 
-                topEx.retryReadyFuture(new IgniteFutureImpl<>(checked.retryReadyFuture()));
+                if (checked.retryReadyFuture() != null)
+                    topEx.retryReadyFuture(new IgniteFutureImpl<>(checked.retryReadyFuture()));
 
                 return topEx;
             }


[09/34] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1265' into ignite-1.3.3-p3

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1265' into ignite-1.3.3-p3


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

Branch: refs/heads/master
Commit: 7d1a550dcac96dc2028aaeab5632c9d35f99489b
Parents: ac670f9 b55365d
Author: sboikov <sb...@gridgain.com>
Authored: Wed Aug 19 15:20:13 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Aug 19 15:20:13 2015 +0300

----------------------------------------------------------------------
 .../CachePartialUpdateCheckedException.java     |  11 +-
 .../processors/cache/GridCacheIoManager.java    |   1 -
 .../processors/cache/GridCacheUtils.java        |  23 ++
 .../distributed/GridDistributedCacheEntry.java  |   7 +
 .../dht/GridClientPartitionTopology.java        |  20 ++
 .../dht/GridDhtPartitionTopology.java           |   7 +
 .../dht/GridDhtPartitionTopologyImpl.java       |  20 ++
 .../cache/distributed/dht/GridDhtTxLocal.java   |   4 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java | 166 ++++++++++++--
 .../dht/GridDhtTxPrepareResponse.java           |  42 +++-
 .../distributed/near/GridNearCacheEntry.java    |   6 +
 .../distributed/near/GridNearLockFuture.java    |   4 +-
 .../near/GridNearTxPrepareResponse.java         |   3 -
 .../cache/transactions/IgniteInternalTx.java    |   2 +-
 .../cache/transactions/IgniteTxAdapter.java     |  19 +-
 .../cache/transactions/IgniteTxEntry.java       |  18 ++
 .../cache/transactions/IgniteTxHandler.java     |   5 +-
 .../ignite/internal/util/lang/GridFunc.java     |  14 ++
 ...teAtomicCacheEntryProcessorNodeJoinTest.java |  32 +++
 .../IgniteCacheEntryProcessorNodeJoinTest.java  | 225 +++++++++++++++++++
 .../near/GridCacheNearOnlyTopologySelfTest.java |   4 +-
 .../near/GridCacheNearTxForceKeyTest.java       |  76 +++++++
 .../testsuites/IgniteCacheTestSuite2.java       |   4 +
 23 files changed, 670 insertions(+), 43 deletions(-)
----------------------------------------------------------------------



[25/34] incubator-ignite git commit: Fixed typo.

Posted by sb...@apache.org.
Fixed typo.


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

Branch: refs/heads/master
Commit: afadd36c2c23e12d9bb29374ea9eb66c32aa71f4
Parents: 3acc014
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Aug 20 18:15:58 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 20 18:15:58 2015 +0700

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/schema/ui/SchemaImportApp.java    | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/afadd36c/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
index aab36b8..e524065 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
@@ -728,12 +728,13 @@ public class SchemaImportApp extends Application {
      * Go to &quot;Generate XML And POJOs&quot; panel or generate XML and POJOs.
      */
     private void next() {
-        if (rootPane.getCenter() == connLayerPnl)
+        if (rootPane.getCenter() == connLayerPnl) {
             if (checkInput(jdbcDrvJarTf, true, "Path to JDBC driver is not specified!") &&
                 checkInput(jdbcDrvClsTf, true, "JDBC driver class name is not specified!") &&
                 checkInput(jdbcUrlTf, true, "JDBC URL connection string is not specified!") &&
                 checkInput(userTf, true, "User name is not specified!"))
                 fill();
+        }
         else
             generate();
     }


[08/34] incubator-ignite git commit: 1.3.3-p3-SNAPSHOT

Posted by sb...@apache.org.
1.3.3-p3-SNAPSHOT


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

Branch: refs/heads/master
Commit: ac670f923c84b4a20b60487354e838cd1d26e263
Parents: 5faffb9
Author: Ignite Teamcity <ig...@apache.org>
Authored: Wed Aug 19 15:09:09 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Wed Aug 19 15:09:09 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                   | 2 +-
 modules/aop/pom.xml                                | 2 +-
 modules/apache-license-gen/pom.xml                 | 2 +-
 modules/aws/pom.xml                                | 2 +-
 modules/clients/pom.xml                            | 2 +-
 modules/cloud/pom.xml                              | 2 +-
 modules/codegen/pom.xml                            | 2 +-
 modules/core/pom.xml                               | 2 +-
 modules/core/src/main/resources/ignite.properties  | 2 +-
 modules/extdata/p2p/pom.xml                        | 2 +-
 modules/extdata/uri/modules/uri-dependency/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 +-
 modules/jcl/pom.xml                                | 2 +-
 modules/jta/pom.xml                                | 2 +-
 modules/kafka/pom.xml                              | 2 +-
 modules/log4j/pom.xml                              | 2 +-
 modules/mesos/pom.xml                              | 2 +-
 modules/rest-http/pom.xml                          | 2 +-
 modules/scalar-2.10/pom.xml                        | 2 +-
 modules/scalar/pom.xml                             | 2 +-
 modules/schedule/pom.xml                           | 2 +-
 modules/schema-import/pom.xml                      | 2 +-
 modules/slf4j/pom.xml                              | 2 +-
 modules/spark-2.10/pom.xml                         | 2 +-
 modules/spark/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-2.10/pom.xml                 | 2 +-
 modules/visor-console/pom.xml                      | 2 +-
 modules/visor-plugins/pom.xml                      | 2 +-
 modules/web/pom.xml                                | 2 +-
 modules/yardstick/pom.xml                          | 2 +-
 modules/yarn/pom.xml                               | 2 +-
 pom.xml                                            | 2 +-
 41 files changed, 41 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 043689c..e57a075 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index 8a271e5..bcf2b9f 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index e1066c1..5dd7459 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -31,5 +31,5 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index e5cf518..51457cf 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 44ff225..a6e6371 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 28a071f..619cb17 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 7d5a9dc..5e894e0 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 980559d..e5bf32d 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index 1bd08d4..b67236b 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.3.3-p2-SNAPSHOT
+ignite.version=1.3.3-p3-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index d81b488..8e37a92 100644
--- a/modules/extdata/p2p/pom.xml
+++ b/modules/extdata/p2p/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-p2p</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/extdata/uri/modules/uri-dependency/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/modules/uri-dependency/pom.xml b/modules/extdata/uri/modules/uri-dependency/pom.xml
index 7430fa1..4374027 100644
--- a/modules/extdata/uri/modules/uri-dependency/pom.xml
+++ b/modules/extdata/uri/modules/uri-dependency/pom.xml
@@ -27,7 +27,7 @@
     <artifactId>ignite-extdata-uri-dep</artifactId>
     <packaging>jar</packaging>
 
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
     <modelVersion>4.0.0</modelVersion>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index 4121188..4de1399 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -32,7 +32,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index 3c1ea24..d7a5386 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index 1cb53b6..06ad2d3 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 28a0e10..c75695b 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index 14d257a..c2469a9 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 15067fa..d00d448 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index d191858..b38133f 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index 9714593..67cd147 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index ed77507..9d0d2a8 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-kafka</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index 987c8aa..707c67d 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 7879e63..ca3fd5e 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <properties>
         <mesos.version>0.22.0</mesos.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 4864b55..fffefbb 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-rest-http</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 11d9c9a..eff169c 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index d5241c3..d30ae50 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 26b2ed3..02b180d 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 957c821..e9f218b 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schema-import</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 4c81324..bd3abaa 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 3cc13ea..5e95aa9 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 0ba28d8..11f604b 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 3fdb893..978a019 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 7911ff1..64d1378 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index e31d996..2f22179 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 58fd030..0ca6d6a 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index a676e10..2985c4b 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index 6eb02e9..5a54217 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index a1936e4..32152cc 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-plugins</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 3e31812..688481b 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index aa3845b..4e037a2 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index bfa53bc..c102e1f 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yarn</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
 
     <properties>
         <hadoop.version>2.7.0</hadoop.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac670f92/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 176c14e..3be4b26 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.3.3-p2-SNAPSHOT</version>
+    <version>1.3.3-p3-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[14/34] incubator-ignite git commit: IGNITE-1265 - Added ready future to ClusterTopologyException, added test for correct explicit transaction retries.

Posted by sb...@apache.org.
IGNITE-1265 - Added ready future to ClusterTopologyException, added test for correct explicit transaction retries.


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

Branch: refs/heads/master
Commit: 8f1c1c0d7406f9b43681a6da69560c8ba6bbb737
Parents: 8ced207
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Aug 19 22:55:48 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Aug 19 22:55:48 2015 -0700

----------------------------------------------------------------------
 .../cluster/ClusterTopologyException.java       |  18 ++
 .../ClusterTopologyCheckedException.java        |  18 ++
 .../cache/GridCacheSharedContext.java           |  15 ++
 .../colocated/GridDhtColocatedLockFuture.java   |   8 +-
 .../distributed/near/GridNearLockFuture.java    |   8 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   7 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   7 +-
 .../ignite/internal/util/IgniteUtils.java       |   9 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |   1 +
 ...gniteCachePutRetryTransactionalSelfTest.java | 179 +++++++++++++++++++
 10 files changed, 263 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8f1c1c0d/modules/core/src/main/java/org/apache/ignite/cluster/ClusterTopologyException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterTopologyException.java b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterTopologyException.java
index d28c409..61bc367 100644
--- a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterTopologyException.java
+++ b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterTopologyException.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.cluster;
 
 import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -27,6 +28,9 @@ public class ClusterTopologyException extends IgniteException {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Retry ready future. */
+    private transient IgniteFuture<?> readyFut;
+
     /**
      * Creates new topology exception with given error message.
      *
@@ -46,4 +50,18 @@ public class ClusterTopologyException extends IgniteException {
     public ClusterTopologyException(String msg, @Nullable Throwable cause) {
         super(msg, cause);
     }
+
+    /**
+     * @return Retry ready future.
+     */
+    public IgniteFuture<?> retryReadyFuture() {
+        return readyFut;
+    }
+
+    /**
+     * @param readyFut Retry ready future.
+     */
+    public void retryReadyFuture(IgniteFuture<?> readyFut) {
+        this.readyFut = readyFut;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8f1c1c0d/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterTopologyCheckedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterTopologyCheckedException.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterTopologyCheckedException.java
index 8f985b4..2d7b0de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterTopologyCheckedException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterTopologyCheckedException.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.cluster;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -27,6 +28,9 @@ public class ClusterTopologyCheckedException extends IgniteCheckedException {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Next topology version to wait. */
+    private transient IgniteInternalFuture<?> readyFut;
+
     /**
      * Creates new topology exception with given error message.
      *
@@ -46,4 +50,18 @@ public class ClusterTopologyCheckedException extends IgniteCheckedException {
     public ClusterTopologyCheckedException(String msg, @Nullable Throwable cause) {
         super(msg, cause);
     }
+
+    /**
+     * @return Retry ready future.
+     */
+    public IgniteInternalFuture<?> retryReadyFuture() {
+        return readyFut;
+    }
+
+    /**
+     * @param readyFut Retry ready future.
+     */
+    public void retryReadyFuture(IgniteInternalFuture<?> readyFut) {
+        this.readyFut = readyFut;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8f1c1c0d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 7f4daff..f7763ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -444,6 +444,21 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
+     * Gets ready future for the next affinity topology version (used in cases when a node leaves grid).
+     *
+     * @param curVer Current topology version (before a node left grid).
+     * @return Ready future.
+     */
+    public IgniteInternalFuture<?> nextAffinityReadyFuture(AffinityTopologyVersion curVer) {
+        if (curVer == null)
+            return null;
+
+        AffinityTopologyVersion nextVer = new AffinityTopologyVersion(curVer.topologyVersion() + 1);
+
+        return exchMgr.affinityReadyFuture(nextVer);
+    }
+
+    /**
      * @param tx Transaction to check.
      * @param activeCacheIds Active cache IDs.
      * @param cacheCtx Cache context.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8f1c1c0d/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 c784948..90ca8df 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
@@ -1125,8 +1125,12 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
      * @return Topology exception with user-friendly message.
      */
     private ClusterTopologyCheckedException newTopologyException(@Nullable Throwable nested, UUID nodeId) {
-        return new ClusterTopologyCheckedException("Failed to acquire lock for keys (primary node left grid, " +
-            "retry transaction if possible) [keys=" + keys + ", node=" + nodeId + ']', nested);
+        ClusterTopologyCheckedException topEx = new ClusterTopologyCheckedException("Failed to acquire lock for keys " +
+            "(primary node left grid, retry transaction if possible) [keys=" + keys + ", node=" + nodeId + ']', nested);
+
+        topEx.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(topVer.get()));
+
+        return topEx;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8f1c1c0d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index b7e0d73..2815194 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -1232,8 +1232,12 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
      * @return Topology exception with user-friendly message.
      */
     private ClusterTopologyCheckedException newTopologyException(@Nullable Throwable nested, UUID nodeId) {
-        return new ClusterTopologyCheckedException("Failed to acquire lock for keys (primary node left grid, " +
-            "retry transaction if possible) [keys=" + keys + ", node=" + nodeId + ']', nested);
+        ClusterTopologyCheckedException topEx = new ClusterTopologyCheckedException("Failed to acquire lock for keys " +
+            "(primary node left grid, retry transaction if possible) [keys=" + keys + ", node=" + nodeId + ']', nested);
+
+        topEx.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(topVer.get()));
+
+        return topEx;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8f1c1c0d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 2b86672..4bb4c67 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -100,7 +100,12 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
                 MiniFuture f = (MiniFuture) fut;
 
                 if (f.node().id().equals(nodeId)) {
-                    f.onResult(new ClusterTopologyCheckedException("Remote node left grid: " + nodeId));
+                    ClusterTopologyCheckedException e = new ClusterTopologyCheckedException("Remote node left grid: " +
+                        nodeId);
+
+                    e.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion()));
+
+                    f.onResult(e);
 
                     found = true;
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8f1c1c0d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index 7006114..3d43797 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -72,7 +72,12 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
             MiniFuture f = (MiniFuture)fut;
 
             if (f.node().id().equals(nodeId)) {
-                f.onNodeLeft(new ClusterTopologyCheckedException("Remote node left grid: " + nodeId));
+                ClusterTopologyCheckedException e = new ClusterTopologyCheckedException("Remote node left grid: " +
+                    nodeId);
+
+                e.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion()));
+
+                f.onNodeLeft(e);
 
                 found = true;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8f1c1c0d/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 3366256..e259084 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.mxbean.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.transactions.*;
+import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -580,7 +581,13 @@ public abstract class IgniteUtils {
 
         m.put(ClusterTopologyCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
             @Override public IgniteException apply(IgniteCheckedException e) {
-                return new ClusterTopologyException(e.getMessage(), e);
+                ClusterTopologyException topEx = new ClusterTopologyException(e.getMessage(), e);
+
+                ClusterTopologyCheckedException checked = (ClusterTopologyCheckedException)e;
+
+                topEx.retryReadyFuture(new IgniteFutureImpl<>(checked.retryReadyFuture()));
+
+                return topEx;
             }
         });
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8f1c1c0d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
index 9abc5c8..6624f8e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
@@ -54,6 +54,7 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
 
         cfg.setAtomicWriteOrderMode(writeOrderMode());
         cfg.setBackups(1);
+        cfg.setRebalanceMode(CacheRebalanceMode.SYNC);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8f1c1c0d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index 91c454a..9a6bb31 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@ -19,17 +19,29 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.testframework.*;
+import org.apache.ignite.transactions.*;
 
+import javax.cache.*;
+import javax.cache.processor.*;
+import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
 /**
  *
  */
 public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetryAbstractSelfTest {
+    /** */
+    private static final int FACTOR = 1000;
+
     /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
         return CacheAtomicityMode.TRANSACTIONAL;
@@ -71,4 +83,171 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         finished.set(true);
         fut.get();
     }
+
+    /** {@inheritDoc} */
+    public void testExplicitTransactionRetries() throws Exception {
+        final AtomicInteger idx = new AtomicInteger();
+        int threads = 8;
+
+        final AtomicReferenceArray<Exception> err = new AtomicReferenceArray<>(threads);
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override
+            public Object call() throws Exception {
+                int th = idx.getAndIncrement();
+                int base = th * FACTOR;
+
+                Ignite ignite = ignite(0);
+                final IgniteCache<Object, Object> cache = ignite.cache(null);
+
+                try {
+                    for (int i = 0; i < FACTOR; i++) {
+                        doInTransaction(ignite, new ProcessCallable(cache, base, i));
+
+                        if (i > 0 && i % 500 == 0)
+                            info("Done: " + i);
+                    }
+                }
+                catch (Exception e) {
+                    err.set(th, e);
+                }
+
+                return null;
+            }
+        }, threads, "tx-runner");
+
+        while (!fut.isDone()) {
+            int stopIdx = ThreadLocalRandom.current().nextInt(2, 4); // Random in [2, 3].
+
+            stopGrid(stopIdx);
+
+            startGrid(stopIdx);
+        }
+
+        for (int i = 0; i < threads; i++) {
+            Exception error = err.get(i);
+
+            if (error != null)
+                throw error;
+        }
+
+        // Verify contents of the cache.
+        for (int g = 0; g < gridCount(); g++) {
+            IgniteCache<Object, Object> cache = ignite(g).cache(null);
+
+            for (int th = 0; th < threads; th++) {
+                int base = th * FACTOR;
+
+                String key = "key-" + base;
+
+                Set<String> set = (Set<String>)cache.get(key);
+
+                assertNotNull("Missing set for key: " + key, set);
+                assertEquals(FACTOR, set.size());
+
+                for (int i = 0; i < FACTOR; i++) {
+                    assertEquals("value-" + i, cache.get("key-" + base + "-" + i));
+                    assertTrue(set.contains("value-" + i));
+                }
+            }
+        }
+    }
+
+    /**
+     * @param ignite Ignite instance.
+     * @param clo Closure.
+     * @return Result of closure execution.
+     * @throws Exception
+     */
+    private <T> T doInTransaction(Ignite ignite, Callable<T> clo) throws Exception {
+        while (true) {
+            try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                T res = clo.call();
+
+                tx.commit();
+
+                return res;
+            }
+            catch (CacheException e) {
+                if (e.getCause() instanceof ClusterTopologyException) {
+                    ClusterTopologyException topEx = (ClusterTopologyException)e.getCause();
+
+                    topEx.retryReadyFuture().get();
+                }
+                else
+                    throw e;
+            }
+            catch (ClusterTopologyException e) {
+                IgniteFuture<?> fut = e.retryReadyFuture();
+
+                fut.get();
+            }
+            catch (TransactionRollbackException ignore) {
+                // Safe to retry right away.
+            }
+        }
+    }
+
+    /**
+     * Callable to process inside transaction.
+     */
+    private static class ProcessCallable implements Callable<Void> {
+        /** */
+        private IgniteCache cache;
+
+        /** */
+        private int base;
+
+        /** */
+        private int i;
+
+        /**
+         * @param cache Cache.
+         * @param base Base index.
+         * @param i Iteration index.
+         */
+        private ProcessCallable(IgniteCache<Object, Object> cache, int base, int i) {
+            this.cache = cache;
+            this.base = base;
+            this.i = i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void call() throws Exception {
+            ((IgniteCache<String, String>)cache).put("key-" + base + "-" + i, "value-" + i);
+
+            ((IgniteCache<String, Set<String>>)cache).invoke("key-" + base, new AddEntryProcessor("value-" + i));
+
+            return null;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class AddEntryProcessor implements CacheEntryProcessor<String, Set<String>, Void> {
+        /** */
+        private String addVal;
+
+        /**
+         * @param addVal Value to add.
+         */
+        private AddEntryProcessor(String addVal) {
+            this.addVal = addVal;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void process(MutableEntry<String, Set<String>> entry, Object... arguments) throws EntryProcessorException {
+            Set<String> set = entry.getValue();
+
+            if (set == null)
+                set = new HashSet<>();
+
+            set.add(addVal);
+
+            entry.setValue(set);
+
+            return null;
+        }
+    }
 }


[03/34] incubator-ignite git commit: IGNITE-1265 - EntryProcessorTest when nodes joining topology.

Posted by sb...@apache.org.
IGNITE-1265 - EntryProcessorTest when nodes joining topology.


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

Branch: refs/heads/master
Commit: 013d7075853d4728739f3dfb2647ce6001d723c8
Parents: 574c679
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Aug 18 18:18:43 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Aug 18 18:18:43 2015 -0700

----------------------------------------------------------------------
 .../IgniteCacheEntryProcessorRestartTest.java   | 185 +++++++++++++++++++
 1 file changed, 185 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/013d7075/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorRestartTest.java
new file mode 100644
index 0000000..c027ee4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorRestartTest.java
@@ -0,0 +1,185 @@
+/*
+ * 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.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.processor.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * Tests cache in-place modification logic with iterative value increment.
+ */
+public class IgniteCacheEntryProcessorRestartTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Number of nodes to test on. */
+    private static final int GRID_CNT = 2;
+
+    /** Number of increment iterations. */
+    private static final int NUM_SETS = 50;
+
+    /** Helper for excluding stopped node from iteration logic. */
+    private AtomicReferenceArray<Ignite> grids;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cache = new CacheConfiguration();
+
+        cache.setCacheMode(PARTITIONED);
+        cache.setAtomicityMode(TRANSACTIONAL);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setBackups(1);
+        cache.setRebalanceMode(SYNC);
+
+        cfg.setCacheConfiguration(cache);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGrids(GRID_CNT);
+
+        grids = new AtomicReferenceArray<>(GRID_CNT);
+
+        for (int i = 0; i < GRID_CNT; i++)
+            grids.set(i, grid(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        grids = null;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEntryProcessorRestart() throws Exception {
+        final AtomicBoolean stop = new AtomicBoolean();
+        final AtomicReference<Throwable> error = new AtomicReference<>();
+        final int started = 6;
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                try {
+                    for (int i = 0; i < started; i++) {
+                        U.sleep(1_000);
+
+                        startGrid(GRID_CNT + i);
+                    }
+                }
+                catch (Exception e) {
+                    error.compareAndSet(null, e);
+                }
+            }
+        }, 1, "starter");
+
+        try {
+            checkIncrement();
+        }
+        finally {
+            stop.set(true);
+
+            fut.get(getTestTimeout());
+        }
+
+        for (int i = 0; i < NUM_SETS; i++) {
+            for (int g = 0; g < GRID_CNT + started; g++) {
+                Set<String> vals = ignite(g).<String, Set<String>>cache(null).get("set-" + i);
+
+                assertNotNull(vals);
+                assertEquals(100, vals.size());
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkIncrement() throws Exception {
+        for (int k = 0; k < 100; k++) {
+            for (int i = 0; i < NUM_SETS; i++) {
+                String key = "set-" + i;
+
+                String val = "value-" + k;
+
+                IgniteCache<String, Set<String>> cache = ignite(0).cache(null);
+
+                cache.invoke(key, new Processor(val));
+            }
+        }
+    }
+
+    /** */
+    private static class Processor implements EntryProcessor<String, Set<String>, Void>, Serializable {
+        /** */
+        private String val;
+
+        private Processor(String val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void process(MutableEntry<String, Set<String>> e, Object... args) {
+            Set<String> vals = e.getValue();
+
+            if (vals == null)
+                vals = new HashSet<>();
+
+            vals.add(val);
+
+            e.setValue(vals);
+
+            return null;
+        }
+    }
+}