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/12/09 08:36:27 UTC

[01/50] [abbrv] ignite git commit: ignite-2008 Added test reproducing issue.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1.5.1 250aa4f90 -> ebb4aa63b


ignite-2008 Added test reproducing issue.


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

Branch: refs/heads/ignite-1.5.1
Commit: f7709543c7e6652dc276e38629f2f1d1315fa84e
Parents: e7298ad
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 3 10:42:15 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 3 10:42:15 2015 +0300

----------------------------------------------------------------------
 .../CacheLockReleaseNodeLeaveTest.java          | 51 ++++++++++++++++++++
 1 file changed, 51 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f7709543/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
index 956aaef..e84fd3f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
@@ -29,9 +29,12 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  *
@@ -107,4 +110,52 @@ public class CacheLockReleaseNodeLeaveTest extends GridCommonAbstractTest {
 
         fut2.get(5, SECONDS);
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxLockRelease() throws Exception {
+        startGrids(2);
+
+        final Ignite ignite0 = ignite(0);
+        final Ignite ignite1 = ignite(1);
+
+        final Integer key = primaryKey(ignite1.cache(null));
+
+        IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                Transaction tx = ignite0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+                ignite0.cache(null).get(key);
+
+                return null;
+            }
+        }, "lock-thread1");
+
+        fut1.get();
+
+        IgniteInternalFuture<?> fut2 = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                try (Transaction tx = ignite1.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                    log.info("Start tx lock.");
+
+                    ignite1.cache(null).get(key);
+
+                    log.info("Tx locked key.");
+
+                    tx.commit();
+                }
+
+                return null;
+            }
+        }, "lock-thread2");
+
+        U.sleep(1000);
+
+        log.info("Stop node.");
+
+        ignite0.close();
+
+        fut2.get(5, SECONDS);
+    }
 }


[03/50] [abbrv] ignite git commit: ignite-1027 Fixed early rebalance sync future completion.

Posted by sb...@apache.org.
ignite-1027 Fixed early rebalance sync future completion.


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

Branch: refs/heads/ignite-1.5.1
Commit: ad9e4db5b87b064d13db4f9251c25efd535fb9e8
Parents: 9b60c75
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 3 10:45:30 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 3 10:45:30 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      | 17 ++--
 .../processors/cache/GridCachePreloader.java    | 11 ++-
 .../dht/preloader/GridDhtPartitionDemander.java | 41 ++++----
 .../dht/preloader/GridDhtPreloader.java         |  4 +-
 .../preloader/GridDhtPreloaderAssignments.java  | 19 +++-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java | 37 +++++---
 ...cingDelayedPartitionMapExchangeSelfTest.java |  9 +-
 .../GridCacheRebalancingAsyncSelfTest.java      |  3 +-
 .../GridCacheRebalancingSyncCheckDataTest.java  | 98 ++++++++++++++++++++
 .../GridCacheRebalancingSyncSelfTest.java       | 55 +++++------
 ...eRebalancingUnmarshallingFailedSelfTest.java |  6 +-
 .../testsuites/IgniteCacheTestSuite3.java       |  2 +
 12 files changed, 223 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index b13a5af..a0f7f93 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -1399,8 +1399,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                         waitList.add(cctx.cacheContext(cId).name());
                                 }
 
-                                Callable<Boolean> r = cacheCtx.preloader().addAssignments(
-                                    assignsMap.get(cacheId), forcePreload, waitList, cnt);
+                                Callable<Boolean> r = cacheCtx.preloader().addAssignments(assignsMap.get(cacheId),
+                                    forcePreload,
+                                    waitList,
+                                    cnt);
 
                                 if (r != null) {
                                     U.log(log, "Cache rebalancing scheduled: [cache=" + cacheCtx.name() +
@@ -1425,7 +1427,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                     "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
                                     ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
 
-                                if (marshR != null)
+                                if (marshR != null) {
                                     try {
                                         marshR.call(); //Marshaller cache rebalancing launches in sync way.
                                     }
@@ -1435,6 +1437,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                                         continue;
                                     }
+                                }
 
                                 final GridFutureAdapter fut = new GridFutureAdapter();
 
@@ -1463,17 +1466,19 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                             fut.onDone();
                                         }
                                     }
-                                }, /*system pool*/ true);
+                                }, /*system pool*/true);
                             }
-                            else
+                            else {
                                 U.log(log, "Skipping rebalancing (obsolete exchange ID) " +
                                     "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
                                     ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
+                            }
                         }
-                        else
+                        else {
                             U.log(log, "Skipping rebalancing (nothing scheduled) " +
                                 "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
                                 ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
+                        }
                     }
                 }
                 catch (IgniteInterruptedCheckedException e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 8e1164b..c8fcb90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -86,9 +86,9 @@ public interface GridCachePreloader {
 
     /**
      * @param exchFut Exchange future to assign.
-     * @return Assignments.
+     * @return Assignments or {@code null} if detected that there are pending exchanges.
      */
-    public GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut);
+    @Nullable public GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut);
 
     /**
      * Adds assignments to preloader.
@@ -97,9 +97,12 @@ public interface GridCachePreloader {
      * @param forcePreload Force preload flag.
      * @param caches Rebalancing of these caches will be finished before this started.
      * @param cnt Counter.
+     * @return Rebalancing closure.
      */
-    public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
-        Collection<String> caches, int cnt);
+    public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments,
+        boolean forcePreload,
+        Collection<String> caches,
+        int cnt);
 
     /**
      * @param p Preload predicate.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index eb9e97f..ced0d10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -251,6 +251,7 @@ public class GridDhtPartitionDemander {
     /**
      * @param name Cache name.
      * @param fut Future.
+     * @throws IgniteCheckedException If failed.
      */
     private boolean waitForCacheRebalancing(String name, RebalanceFuture fut) throws IgniteCheckedException {
         if (log.isDebugEnabled())
@@ -283,7 +284,7 @@ public class GridDhtPartitionDemander {
      * @param force {@code True} if dummy reassign.
      * @param caches Rebalancing of these caches will be finished before this started.
      * @param cnt Counter.
-     * @throws IgniteCheckedException If failed.
+     * @return Rebalancing closure.
      */
     Callable<Boolean> addAssignments(final GridDhtPreloaderAssignments assigns, boolean force,
         final Collection<String> caches, int cnt) {
@@ -293,25 +294,24 @@ public class GridDhtPartitionDemander {
         long delay = cctx.config().getRebalanceDelay();
 
         if (delay == 0 || force) {
-            assert assigns != null;
-
             final RebalanceFuture oldFut = rebalanceFut;
 
             final RebalanceFuture fut = new RebalanceFuture(assigns, cctx, log, oldFut.isInitial(), cnt);
 
             if (!oldFut.isInitial())
                 oldFut.cancel();
-            else
+            else {
                 fut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
-                    @Override public void apply(IgniteInternalFuture<Boolean> future) {
+                    @Override public void apply(IgniteInternalFuture<Boolean> fut) {
                         oldFut.onDone(fut.result());
                     }
                 });
+            }
 
             rebalanceFut = fut;
 
             if (assigns.isEmpty()) {
-                fut.doneIfEmpty();
+                fut.doneIfEmpty(assigns.cancelled());
 
                 return null;
             }
@@ -357,6 +357,9 @@ public class GridDhtPartitionDemander {
 
     /**
      * @param fut Future.
+     * @param assigns Assignments.
+     * @throws IgniteCheckedException If failed.
+     * @return
      */
     private boolean requestPartitions(
         RebalanceFuture fut,
@@ -370,7 +373,7 @@ public class GridDhtPartitionDemander {
 
             GridDhtPartitionDemandMessage d = e.getValue();
 
-            fut.appendPartitions(node.id(), d.partitions());//Future preparation.
+            fut.appendPartitions(node.id(), d.partitions()); //Future preparation.
         }
 
         for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
@@ -413,7 +416,8 @@ public class GridDhtPartitionDemander {
                         initD.timeout(cctx.config().getRebalanceTimeout());
 
                         synchronized (fut) {
-                            if (!fut.isDone())// Future can be already cancelled at this moment and all failovers happened.
+                            if (!fut.isDone())
+                                // Future can be already cancelled at this moment and all failovers happened.
                                 // New requests will not be covered by failovers.
                                 cctx.io().sendOrderedMessage(node,
                                     rebalanceTopics.get(cnt), initD, cctx.ioPolicy(), initD.timeout());
@@ -427,9 +431,12 @@ public class GridDhtPartitionDemander {
                 }
             }
             else {
-                U.log(log, "Starting rebalancing (old api) [cache=" + cctx.name() + ", mode=" + cfg.getRebalanceMode() +
-                    ", fromNode=" + node.id() + ", partitionsCount=" + parts.size() +
-                    ", topology=" + fut.topologyVersion() + ", updateSeq=" + fut.updateSeq + "]");
+                U.log(log, "Starting rebalancing (old api) [cache=" + cctx.name() +
+                    ", mode=" + cfg.getRebalanceMode() +
+                    ", fromNode=" + node.id() +
+                    ", partitionsCount=" + parts.size() +
+                    ", topology=" + fut.topologyVersion() +
+                    ", updateSeq=" + fut.updateSeq + "]");
 
                 d.timeout(cctx.config().getRebalanceTimeout());
                 d.workerId(0);//old api support.
@@ -832,9 +839,9 @@ public class GridDhtPartitionDemander {
         }
 
         /**
-         *
+         * @param cancelled Is cancelled.
          */
-        private void doneIfEmpty() {
+        private void doneIfEmpty(boolean cancelled) {
             synchronized (this) {
                 if (isDone())
                     return;
@@ -845,14 +852,14 @@ public class GridDhtPartitionDemander {
                     log.debug("Rebalancing is not required [cache=" + cctx.name() +
                         ", topology=" + topVer + "]");
 
-                checkIsDone();
+                checkIsDone(cancelled);
             }
         }
 
         /**
          * Cancels this future.
          *
-         * @return {@code true}.
+         * @return {@code True}.
          */
         @Override public boolean cancel() {
             synchronized (this) {
@@ -860,7 +867,7 @@ public class GridDhtPartitionDemander {
                     return true;
 
                 U.log(log, "Cancelled rebalancing from all nodes [cache=" + cctx.name()
-                    + ", topology=" + topologyVersion());
+                    + ", topology=" + topologyVersion() + ']');
 
                 if (!cctx.kernalContext().isStopping()) {
                     for (UUID nodeId : remaining.keySet())
@@ -1012,7 +1019,7 @@ public class GridDhtPartitionDemander {
                     preloadEvent(EVT_CACHE_REBALANCE_STOPPED, exchFut.discoveryEvent());
 
                 if (log.isDebugEnabled())
-                    log.debug("Completed rebalance future.");
+                    log.debug("Completed rebalance future: " + this);
 
                 cctx.shared().exchange().scheduleResendPartitions();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 3e3cee3..9a6246f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -324,7 +324,9 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
                     log.debug("Skipping assignments creation, exchange worker has pending assignments: " +
                         exchFut.exchangeId());
 
-                break;
+                assigns.cancelled(true);
+
+                return assigns;
             }
 
             // If partition belongs to local node.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
index 3583967..3f82c9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
@@ -37,19 +37,36 @@ public class GridDhtPreloaderAssignments extends ConcurrentHashMap<ClusterNode,
     /** Last join order. */
     private final AffinityTopologyVersion topVer;
 
+    /** */
+    private boolean cancelled;
+
     /**
      * @param exchFut Exchange future.
      * @param topVer Last join order.
      */
     public GridDhtPreloaderAssignments(GridDhtPartitionsExchangeFuture exchFut, AffinityTopologyVersion topVer) {
         assert exchFut != null;
-        assert topVer.topologyVersion() > 0;
+        assert topVer.topologyVersion() > 0 : topVer;
 
         this.exchFut = exchFut;
         this.topVer = topVer;
     }
 
     /**
+     * @return {@code True} if assignments creation was cancelled.
+     */
+    public boolean cancelled() {
+        return cancelled;
+    }
+
+    /**
+     * @param cancelled {@code True} if assignments creation was cancelled.
+     */
+    public void cancelled(boolean cancelled) {
+        this.cancelled = cancelled;
+    }
+
+    /**
      * @return Exchange future.
      */
     GridDhtPartitionsExchangeFuture exchangeFuture() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
index 9d6e82f..0b610f3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
@@ -22,7 +22,6 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CachePeekMode;
@@ -35,7 +34,6 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
@@ -51,6 +49,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
@@ -107,7 +106,9 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
         stopAllGrids();
     }
 
-    /** @throws Exception If failed. */
+    /**
+     * @throws Exception If failed.
+     */
     public void testManualPreload() throws Exception {
         delay = -1;
 
@@ -184,7 +185,9 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
         checkCache(c2, cnt);
     }
 
-    /** @throws Exception If failed. */
+    /**
+     * @throws Exception If failed.
+     */
     public void testDelayedPreload() throws Exception {
         delay = PRELOAD_DELAY;
 
@@ -238,9 +241,9 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
 
         checkMaps(false, d0, d1, d2);
 
-        assert l1.await(PRELOAD_DELAY * 3 / 2, TimeUnit.MILLISECONDS);
+        assert l1.await(PRELOAD_DELAY * 3 / 2, MILLISECONDS);
 
-        assert l2.await(PRELOAD_DELAY * 3 / 2, TimeUnit.MILLISECONDS);
+        assert l2.await(PRELOAD_DELAY * 3 / 2, MILLISECONDS);
 
         U.sleep(1000);
 
@@ -253,7 +256,9 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
         checkCache(c2, cnt);
     }
 
-    /** @throws Exception If failed. */
+    /**
+     * @throws Exception If failed.
+     */
     public void testAutomaticPreload() throws Exception {
         delay = 0;
         preloadMode = CacheRebalanceMode.SYNC;
@@ -284,7 +289,9 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
         checkCache(c2, cnt);
     }
 
-    /** @throws Exception If failed. */
+    /**
+     * @throws Exception If failed.
+     */
     public void testAutomaticPreloadWithEmptyCache() throws Exception {
         preloadMode = SYNC;
 
@@ -331,7 +338,9 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
         }
     }
 
-    /** @throws Exception If failed. */
+    /**
+     * @throws Exception If failed.
+     */
     public void testManualPreloadSyncMode() throws Exception {
         preloadMode = CacheRebalanceMode.SYNC;
         delay = -1;
@@ -344,7 +353,9 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
         }
     }
 
-    /** @throws Exception If failed. */
+    /**
+     * @throws Exception If failed.
+     */
     public void testPreloadManyNodes() throws Exception {
         delay = 0;
         preloadMode = ASYNC;
@@ -419,9 +430,11 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
      *
      * @param strict Strict check flag.
      * @param caches Maps to compare.
+     * @throws Exception If failed.
      */
-    private void checkMaps(final boolean strict, final GridDhtCacheAdapter<String, Integer>... caches)
-        throws IgniteInterruptedCheckedException {
+    @SafeVarargs
+    private final void checkMaps(final boolean strict, final GridDhtCacheAdapter<String, Integer>... caches)
+        throws Exception {
         if (caches.length < 2)
             return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java
index a1ea7ad..2890fcb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java
@@ -73,20 +73,20 @@ public class GridCacheRabalancingDelayedPartitionMapExchangeSelfTest extends Gri
     public class DelayableCommunicationSpi extends TcpCommunicationSpi {
         /** {@inheritDoc} */
         @Override public void sendMessage(final ClusterNode node, final Message msg,
-            final IgniteInClosure<IgniteException> ackClosure) throws IgniteSpiException {
+            final IgniteInClosure<IgniteException> ackC) throws IgniteSpiException {
             final Object msg0 = ((GridIoMessage)msg).message();
 
             if (msg0 instanceof GridDhtPartitionsFullMessage && record &&
                 ((GridDhtPartitionsFullMessage)msg0).exchangeId() == null) {
                 rs.putIfAbsent(node.id(), new Runnable() {
                     @Override public void run() {
-                        DelayableCommunicationSpi.super.sendMessage(node, msg, ackClosure);
+                        DelayableCommunicationSpi.super.sendMessage(node, msg, ackC);
                     }
                 });
             }
             else
                 try {
-                    super.sendMessage(node, msg, ackClosure);
+                    super.sendMessage(node, msg, ackC);
                 }
                 catch (Exception e) {
                     U.log(null, e);
@@ -144,9 +144,8 @@ public class GridCacheRabalancingDelayedPartitionMapExchangeSelfTest extends Gri
 
         awaitPartitionMapExchange();
 
-        for (Runnable r : rs.values()) {
+        for (Runnable r : rs.values())
             r.run();
-        }
 
         U.sleep(10000); // Enough time to process delayed GridDhtPartitionsFullMessages.
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingAsyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingAsyncSelfTest.java
index 7759c70..bcda0da 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingAsyncSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingAsyncSelfTest.java
@@ -33,9 +33,8 @@ public class GridCacheRebalancingAsyncSelfTest extends GridCacheRebalancingSyncS
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration iCfg = super.getConfiguration(gridName);
 
-        for (CacheConfiguration cacheCfg : iCfg.getCacheConfiguration()) {
+        for (CacheConfiguration cacheCfg : iCfg.getCacheConfiguration())
             cacheCfg.setRebalanceMode(CacheRebalanceMode.ASYNC);
-        }
 
         return iCfg;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncCheckDataTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncCheckDataTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncCheckDataTest.java
new file mode 100644
index 0000000..5e4a5c4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncCheckDataTest.java
@@ -0,0 +1,98 @@
+/*
+ * 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.rebalancing;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+
+/**
+ *
+ */
+public class GridCacheRebalancingSyncCheckDataTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+        ccfg.setCacheMode(REPLICATED);
+        ccfg.setRebalanceMode(SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDataRebalancing() throws Exception {
+        Ignite ignite = startGrid(0);
+
+        final int KEYS = 10_000;
+
+        IgniteCache<Object, Object> cache = ignite.cache(null);
+
+        for (int i = 0; i < KEYS; i++)
+            cache.put(i, i);
+
+
+        for (int i = 0; i < 3; i++) {
+            log.info("Iteration: " + i);
+
+            final AtomicInteger idx = new AtomicInteger(1);
+
+            GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    try(Ignite ignite = startGrid(idx.getAndIncrement())) {
+                        IgniteCache<Object, Object> cache = ignite.cache(null);
+
+                        for (int i = 0; i < KEYS; i++)
+                            assertNotNull(cache.localPeek(i));
+                    }
+
+                    return null;
+                }
+            }, 5, "start-node");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
index 8c5cd40..3b25bd7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
@@ -45,19 +45,19 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
     protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** */
-    private static int TEST_SIZE = 100_000;
+    private static final int TEST_SIZE = 100_000;
 
     /** partitioned cache name. */
-    protected static String CACHE_NAME_DHT_PARTITIONED = "cacheP";
+    protected static final String CACHE_NAME_DHT_PARTITIONED = "cacheP";
 
     /** partitioned cache 2 name. */
-    protected static String CACHE_NAME_DHT_PARTITIONED_2 = "cacheP2";
+    protected static final String CACHE_NAME_DHT_PARTITIONED_2 = "cacheP2";
 
     /** replicated cache name. */
-    protected static String CACHE_NAME_DHT_REPLICATED = "cacheR";
+    protected static final String CACHE_NAME_DHT_REPLICATED = "cacheR";
 
     /** replicated cache 2 name. */
-    protected static String CACHE_NAME_DHT_REPLICATED_2 = "cacheR2";
+    protected static final String CACHE_NAME_DHT_REPLICATED_2 = "cacheR2";
 
     /** */
     private volatile boolean concurrentStartFinished;
@@ -122,6 +122,8 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param ignite Ignite.
+     * @param from Start from key.
+     * @param iter Iteration.
      */
     protected void generateData(Ignite ignite, int from, int iter) {
         generateData(ignite, CACHE_NAME_DHT_PARTITIONED, from, iter);
@@ -132,6 +134,9 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param ignite Ignite.
+     * @param name Cache name.
+     * @param from Start from key.
+     * @param iter Iteration.
      */
     protected void generateData(Ignite ignite, String name, int from, int iter) {
         for (int i = from; i < from + TEST_SIZE; i++) {
@@ -144,9 +149,10 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param ignite Ignite.
-     * @throws IgniteCheckedException Exception.
+     * @param from Start from key.
+     * @param iter Iteration.
      */
-    protected void checkData(Ignite ignite, int from, int iter) throws IgniteCheckedException {
+    protected void checkData(Ignite ignite, int from, int iter) {
         checkData(ignite, CACHE_NAME_DHT_PARTITIONED, from, iter);
         checkData(ignite, CACHE_NAME_DHT_PARTITIONED_2, from, iter);
         checkData(ignite, CACHE_NAME_DHT_REPLICATED, from, iter);
@@ -155,10 +161,11 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param ignite Ignite.
+     * @param from Start from key.
+     * @param iter Iteration.
      * @param name Cache name.
-     * @throws IgniteCheckedException Exception.
      */
-    protected void checkData(Ignite ignite, String name, int from, int iter) throws IgniteCheckedException {
+    protected void checkData(Ignite ignite, String name, int from, int iter) {
         for (int i = from; i < from + TEST_SIZE; i++) {
             if (i % (TEST_SIZE / 10) == 0)
                 log.info("<" + name + "> Checked " + i * 100 / (TEST_SIZE) + "% entries (" + TEST_SIZE + ").");
@@ -169,7 +176,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception Exception
+     * @throws Exception If failed.
      */
     public void testSimpleRebalancing() throws Exception {
         Ignite ignite = startGrid(0);
@@ -206,7 +213,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception Exception
+     * @throws Exception If failed.
      */
     public void testLoadRebalancing() throws Exception {
         final Ignite ignite = startGrid(0);
@@ -240,14 +247,8 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
         Thread t2 = new Thread() {
             @Override public void run() {
-                while (!concurrentStartFinished) {
-                    try {
-                        checkData(ignite, CACHE_NAME_DHT_PARTITIONED, 0, 0);
-                    }
-                    catch (IgniteCheckedException e) {
-                        e.printStackTrace();
-                    }
-                }
+                while (!concurrentStartFinished)
+                    checkData(ignite, CACHE_NAME_DHT_PARTITIONED, 0, 0);
             }
         };
 
@@ -282,7 +283,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
      * @param id Node id.
      * @param major Major ver.
      * @param minor Minor ver.
-     * @throws IgniteCheckedException Exception.
+     * @throws IgniteCheckedException If failed.
      */
     protected void waitForRebalancing(int id, int major, int minor) throws IgniteCheckedException {
         waitForRebalancing(id, new AffinityTopologyVersion(major, minor));
@@ -291,7 +292,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
     /**
      * @param id Node id.
      * @param major Major ver.
-     * @throws IgniteCheckedException Exception.
+     * @throws IgniteCheckedException If failed.
      */
     protected void waitForRebalancing(int id, int major) throws IgniteCheckedException {
         waitForRebalancing(id, new AffinityTopologyVersion(major));
@@ -300,7 +301,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
     /**
      * @param id Node id.
      * @param top Topology version.
-     * @throws IgniteCheckedException
+     * @throws IgniteCheckedException If failed.
      */
     protected void waitForRebalancing(int id, AffinityTopologyVersion top) throws IgniteCheckedException {
         boolean finished = false;
@@ -327,6 +328,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
     /**
      *
      */
+    @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
     protected void checkSupplyContextMapIsEmpty() {
         for (Ignite g : G.allGrids()) {
             for (GridCacheAdapter c : ((IgniteEx)g).context().cache().internalCaches()) {
@@ -342,12 +344,13 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         }
     }
 
+    /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
         return 5 * 60_000;
     }
 
     /**
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testComplexRebalancing() throws Exception {
         final Ignite ignite = startGrid(0);
@@ -368,9 +371,8 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
                     startGrid(1);
                     startGrid(2);
 
-                    while (!concurrentStartFinished2) {
+                    while (!concurrentStartFinished2)
                         U.sleep(10);
-                    }
 
                     waitForRebalancing(0, 5, 0);
                     waitForRebalancing(1, 5, 0);
@@ -387,9 +389,8 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
                     grid(0).getOrCreateCache(cacheRCfg);
 
-                    while (!concurrentStartFinished3) {
+                    while (!concurrentStartFinished3)
                         U.sleep(10);
-                    }
 
                     concurrentStartFinished = true;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingUnmarshallingFailedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingUnmarshallingFailedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingUnmarshallingFailedSelfTest.java
index 506f1c2..7e35906 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingUnmarshallingFailedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingUnmarshallingFailedSelfTest.java
@@ -125,9 +125,8 @@ public class GridCacheRebalancingUnmarshallingFailedSelfTest extends GridCommonA
 
         startGrid(0);
 
-        for (int i = 0; i < 100; i++) {
+        for (int i = 0; i < 100; i++)
             grid(0).cache(CACHE).put(new TestKey(String.valueOf(i)), i);
-        }
 
         readCnt.set(1);
 
@@ -135,9 +134,8 @@ public class GridCacheRebalancingUnmarshallingFailedSelfTest extends GridCommonA
 
         readCnt.set(Integer.MAX_VALUE);
 
-        for (int i = 0; i < 50; i++) {
+        for (int i = 0; i < 50; i++)
             assert grid(1).cache(CACHE).get(new TestKey(String.valueOf(i))) != null;
-        }
 
         stopGrid(0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad9e4db5/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
index b02d022..176ab3f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
@@ -51,6 +51,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePut
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteTxReentryNearSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRabalancingDelayedPartitionMapExchangeSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingAsyncSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingSyncCheckDataTest;
 import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingSyncSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingUnmarshallingFailedSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheDaemonNodeReplicatedSelfTest;
@@ -140,6 +141,7 @@ public class IgniteCacheTestSuite3 extends TestSuite {
 
         suite.addTestSuite(GridCacheOrderedPreloadingSelfTest.class);
         suite.addTestSuite(GridCacheRebalancingSyncSelfTest.class);
+        suite.addTestSuite(GridCacheRebalancingSyncCheckDataTest.class);
         suite.addTestSuite(GridCacheRebalancingUnmarshallingFailedSelfTest.class);
         suite.addTestSuite(GridCacheRebalancingAsyncSelfTest.class);
         suite.addTestSuite(GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.class);


[28/50] [abbrv] ignite git commit: IGNITE-2083 EntryProcessor is called twice on primary node in transactional cache

Posted by sb...@apache.org.
IGNITE-2083 EntryProcessor is called twice on primary node in transactional cache


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

Branch: refs/heads/ignite-1.5.1
Commit: 9a14d6432932fc1a1fdf2ddd77dea920382efe8c
Parents: c10b112
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 7 15:05:09 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 7 15:05:09 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtTxPrepareFuture.java |  51 +-
 .../cache/transactions/IgniteTxAdapter.java     |   5 +
 .../cache/transactions/IgniteTxEntry.java       |   8 +-
 .../IgniteCacheEntryProcessorCallTest.java      | 497 +++++++++++++++++++
 ...idCachePartitionedHitsAndMissesSelfTest.java |   4 +-
 .../testframework/junits/GridAbstractTest.java  |   7 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 7 files changed, 558 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9a14d643/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 9f1f8a1..3829e28 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
@@ -337,6 +337,13 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         cacheCtx.config().isLoadPreviousValue() &&
                         !txEntry.skipStore();
 
+                    boolean evt = retVal || txEntry.op() == TRANSFORM;
+
+                    EntryProcessor entryProc = null;
+
+                    if (evt && txEntry.op() == TRANSFORM)
+                        entryProc = F.first(txEntry.entryProcessors()).get1();
+
                     CacheObject val = cached.innerGet(
                         tx,
                         /*swap*/true,
@@ -344,11 +351,11 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         /*fail fast*/false,
                         /*unmarshal*/true,
                         /*metrics*/retVal,
-                        /*event*/retVal,
+                        /*event*/evt,
                         /*tmp*/false,
-                        null,
-                        null,
-                        null,
+                        tx.subjectId(),
+                        entryProc,
+                        tx.resolveTaskName(),
                         null,
                         txEntry.keepBinary());
 
@@ -364,11 +371,13 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                             Object procRes = null;
                             Exception err = null;
 
+                            boolean modified = false;
+
                              for (T2<EntryProcessor<Object, Object, Object>, Object[]> t : txEntry.entryProcessors()) {
-                                try {
-                                    CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry<>(
-                                        txEntry.context(), key, val, txEntry.cached().version(), txEntry.keepBinary());
+                                 CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry<>(
+                                     txEntry.context(), key, val, txEntry.cached().version(), txEntry.keepBinary());
 
+                                 try {
                                     EntryProcessor<Object, Object, Object> processor = t.get1();
 
                                     procRes = processor.process(invokeEntry, t.get2());
@@ -380,9 +389,27 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
                                     break;
                                 }
+
+                                 modified |= invokeEntry.modified();
                             }
 
-                            txEntry.entryProcessorCalculatedValue(val);
+                            if (modified)
+                                val = cacheCtx.toCacheObject(cacheCtx.unwrapTemporary(val));
+
+                            GridCacheOperation op = modified ? (val == null ? DELETE : UPDATE) : NOOP;
+
+                            if (op == NOOP) {
+                                if (expiry != null) {
+                                    long ttl = CU.toTtl(expiry.getExpiryForAccess());
+
+                                    txEntry.ttl(ttl);
+
+                                    if (ttl == CU.TTL_ZERO)
+                                        op = DELETE;
+                                }
+                            }
+
+                            txEntry.entryProcessorCalculatedValue(new T2<>(op, op == NOOP ? null : val));
 
                             if (retVal) {
                                 if (err != null || procRes != null)
@@ -1301,10 +1328,12 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         entry.cached().partition());
 
                     if (state != GridDhtPartitionState.OWNING && state != GridDhtPartitionState.EVICTED) {
-                        CacheObject procVal = entry.entryProcessorCalculatedValue();
+                        T2<GridCacheOperation, CacheObject> procVal = entry.entryProcessorCalculatedValue();
+
+                        assert procVal != null : entry;
 
-                        entry.op(procVal == null ? DELETE : UPDATE);
-                        entry.value(procVal, true, false);
+                        entry.op(procVal.get1());
+                        entry.value(procVal.get2(), true, false);
                         entry.entryProcessors(null);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a14d643/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 3065ac2..53f4f56 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
@@ -1233,6 +1233,11 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         if (F.isEmpty(txEntry.entryProcessors()))
             return F.t(txEntry.op(), txEntry.value());
         else {
+            T2<GridCacheOperation, CacheObject> calcVal = txEntry.entryProcessorCalculatedValue();
+
+            if (calcVal != null)
+                return calcVal;
+
             boolean recordEvt = cctx.gridEvents().isRecordable(EVT_CACHE_OBJECT_READ);
 
             CacheObject cacheVal = txEntry.hasValue() ? txEntry.value() :

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a14d643/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 fba1513..2c6c3df 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
@@ -105,7 +105,7 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
 
     /** Transient field for calculated entry processor value. */
     @GridDirectTransient
-    private CacheObject entryProcessorCalcVal;
+    private T2<GridCacheOperation, CacheObject> entryProcessorCalcVal;
 
     /** Transform closure bytes. */
     @GridToStringExclude
@@ -888,14 +888,16 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     /**
      * @return Entry processor calculated value.
      */
-    public CacheObject entryProcessorCalculatedValue() {
+    public T2<GridCacheOperation, CacheObject> entryProcessorCalculatedValue() {
         return entryProcessorCalcVal;
     }
 
     /**
      * @param entryProcessorCalcVal Entry processor calculated value.
      */
-    public void entryProcessorCalculatedValue(CacheObject entryProcessorCalcVal) {
+    public void entryProcessorCalculatedValue(T2<GridCacheOperation, CacheObject> entryProcessorCalcVal) {
+        assert entryProcessorCalcVal != null;
+
         this.entryProcessorCalcVal = entryProcessorCalcVal;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a14d643/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorCallTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorCallTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorCallTest.java
new file mode 100644
index 0000000..5163d96
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorCallTest.java
@@ -0,0 +1,497 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_GRID_NAME;
+import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
+
+/**
+ *
+ */
+public class IgniteCacheEntryProcessorCallTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    static final AtomicInteger callCnt = new AtomicInteger();
+
+    /** */
+    private static final int SRV_CNT = 4;
+
+    /** */
+    private static final int NODES = 5;
+
+    /** */
+    private boolean client;
+
+    /** */
+    private static final int OP_UPDATE = 1;
+
+    /** */
+    private static final int OP_REMOVE = 2;
+
+    /** */
+    private static final int OP_GET = 3;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+        
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(SRV_CNT);
+
+        client = true;
+
+        Ignite client = startGrid(SRV_CNT);
+
+        assertTrue(client.configuration().isClientMode());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEntryProcessorCall() throws Exception {
+        {
+            CacheConfiguration<Integer, TestValue> ccfg = new CacheConfiguration<>();
+            ccfg.setBackups(1);
+            ccfg.setWriteSynchronizationMode(FULL_SYNC);
+            ccfg.setAtomicityMode(ATOMIC);
+            ccfg.setAtomicWriteOrderMode(PRIMARY);
+
+            checkEntryProcessorCallCount(ccfg, 1);
+        }
+
+        {
+            CacheConfiguration<Integer, TestValue> ccfg = new CacheConfiguration<>();
+            ccfg.setBackups(0);
+            ccfg.setWriteSynchronizationMode(FULL_SYNC);
+            ccfg.setAtomicityMode(ATOMIC);
+            ccfg.setAtomicWriteOrderMode(PRIMARY);
+
+            checkEntryProcessorCallCount(ccfg, 1);
+        }
+
+        {
+            CacheConfiguration<Integer, TestValue> ccfg = new CacheConfiguration<>();
+            ccfg.setBackups(1);
+            ccfg.setWriteSynchronizationMode(FULL_SYNC);
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+
+            checkEntryProcessorCallCount(ccfg, 2);
+        }
+
+        {
+            CacheConfiguration<Integer, TestValue> ccfg = new CacheConfiguration<>();
+            ccfg.setBackups(0);
+            ccfg.setWriteSynchronizationMode(FULL_SYNC);
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+
+            checkEntryProcessorCallCount(ccfg, 1);
+        }
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @param expCallCnt Expected entry processor calls count.
+     * @throws Exception If failed.
+     */
+    private void checkEntryProcessorCallCount(CacheConfiguration<Integer, TestValue> ccfg,
+        int expCallCnt) throws Exception {
+        Ignite client1 = ignite(SRV_CNT);
+
+        IgniteCache<Integer, TestValue> clientCache1 = client1.createCache(ccfg);
+
+        IgniteCache<Integer, TestValue> srvCache = ignite(0).cache(ccfg.getName());
+
+        awaitPartitionMapExchange();
+
+        int key = 0;
+
+        checkEntryProcessCall(key++, clientCache1, null, null, expCallCnt);
+
+        if (ccfg.getAtomicityMode() == TRANSACTIONAL) {
+            checkEntryProcessCall(key++, clientCache1, OPTIMISTIC, REPEATABLE_READ, expCallCnt + 1);
+            checkEntryProcessCall(key++, clientCache1, PESSIMISTIC, REPEATABLE_READ, expCallCnt + 1);
+            checkEntryProcessCall(key++, clientCache1, OPTIMISTIC, SERIALIZABLE, expCallCnt + 1);
+        }
+
+        for (int i = 100; i < 110; i++) {
+            checkEntryProcessCall(key++, srvCache, null, null, expCallCnt);
+
+            if (ccfg.getAtomicityMode() == TRANSACTIONAL) {
+                checkEntryProcessCall(key++, srvCache, OPTIMISTIC, REPEATABLE_READ, expCallCnt + 1);
+                checkEntryProcessCall(key++, srvCache, PESSIMISTIC, REPEATABLE_READ, expCallCnt + 1);
+                checkEntryProcessCall(key++, srvCache, OPTIMISTIC, SERIALIZABLE, expCallCnt + 1);
+            }
+        }
+
+        for (int i = 0; i < NODES; i++)
+            ignite(i).destroyCache(ccfg.getName());
+    }
+
+    /**
+     *
+     * @param key Key.
+     * @param cache Cache.
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @param expCallCnt Expected entry processor calls count.
+     */
+    private void checkEntryProcessCall(Integer key,
+        IgniteCache<Integer, TestValue> cache,
+        @Nullable TransactionConcurrency concurrency,
+        @Nullable TransactionIsolation isolation,
+        int expCallCnt) {
+        Ignite ignite = cache.unwrap(Ignite.class);
+
+        ClusterNode primary = ignite.affinity(cache.getName()).mapKeyToNode(key);
+
+        assertNotNull(primary);
+
+        log.info("Check call [key=" + key +
+            ", primary=" + primary.attribute(ATTR_GRID_NAME) +
+            ", concurrency=" + concurrency +
+            ", isolation=" + isolation + "]");
+
+        Transaction tx;
+        TestReturnValue retVal;
+
+        log.info("Invoke: " + key);
+
+        // Update.
+        callCnt.set(0);
+
+        tx = startTx(cache, concurrency, isolation);
+
+        retVal = cache.invoke(key, new TestEntryProcessor(OP_UPDATE), new TestValue(Integer.MIN_VALUE));
+
+        if (tx != null)
+            tx.commit();
+
+        assertEquals(expCallCnt, callCnt.get());
+
+        checkReturnValue(retVal, "null");
+        checkCacheValue(cache.getName(), key, new TestValue(0));
+
+        log.info("Invoke: " + key);
+
+        // Get.
+        callCnt.set(0);
+
+        tx = startTx(cache, concurrency, isolation);
+
+        retVal = cache.invoke(key, new TestEntryProcessor(OP_GET), new TestValue(Integer.MIN_VALUE));
+
+        if (tx != null)
+            tx.commit();
+
+        assertEquals(expCallCnt, callCnt.get());
+
+        checkReturnValue(retVal, "0");
+        checkCacheValue(cache.getName(), key, new TestValue(0));
+
+        log.info("Invoke: " + key);
+
+        // Update.
+        callCnt.set(0);
+
+        tx = startTx(cache, concurrency, isolation);
+
+        retVal = cache.invoke(key, new TestEntryProcessor(OP_UPDATE), new TestValue(Integer.MIN_VALUE));
+
+        if (tx != null)
+            tx.commit();
+
+        assertEquals(expCallCnt, callCnt.get());
+
+        checkReturnValue(retVal, "0");
+        checkCacheValue(cache.getName(), key, new TestValue(1));
+
+        log.info("Invoke: " + key);
+
+        // Remove.
+        callCnt.set(0);
+
+        tx = startTx(cache, concurrency, isolation);
+
+        retVal = cache.invoke(key, new TestEntryProcessor(OP_REMOVE), new TestValue(Integer.MIN_VALUE));
+
+        if (tx != null)
+            tx.commit();
+
+        assertEquals(expCallCnt, callCnt.get());
+
+        checkReturnValue(retVal, "1");
+        checkCacheValue(cache.getName(), key, null);
+    }
+
+    /**
+     * @param retVal Return value.
+     * @param expVal Expected value.
+     */
+    private void checkReturnValue(TestReturnValue retVal, String expVal) {
+        assertNotNull(retVal);
+
+        TestValue arg = (TestValue)retVal.argument();
+        assertNotNull(arg);
+        assertEquals(Integer.MIN_VALUE, (Object)arg.value());
+
+        assertEquals(expVal, retVal.value());
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @param key Key.
+     * @param expVal Expected value.
+     */
+    private void checkCacheValue(String cacheName, Integer key, TestValue expVal) {
+        for (int i = 0; i < NODES; i++) {
+            Ignite ignite = ignite(i);
+
+            IgniteCache<Integer, TestValue> cache = ignite.cache(cacheName);
+
+            assertEquals(expVal, cache.get(key));
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @return Started transaction.
+     */
+    @Nullable private Transaction startTx(IgniteCache<Integer, TestValue> cache,
+        @Nullable TransactionConcurrency concurrency,
+        @Nullable TransactionIsolation isolation) {
+        if (concurrency != null) {
+            assert isolation != null;
+
+            return cache.unwrap(Ignite.class).transactions().txStart(concurrency, isolation);
+        }
+
+        return null;
+    }
+
+    /**
+     *
+     */
+    static class TestEntryProcessor implements EntryProcessor<Integer, TestValue, TestReturnValue> {
+        /** */
+        private int op;
+
+        /**
+         * @param op Operation.
+         */
+        public TestEntryProcessor(int op) {
+            this.op = op;
+        }
+
+        /** {@inheritDoc} */
+        @Override public TestReturnValue process(MutableEntry<Integer, TestValue> entry,
+            Object... args) {
+            Ignite ignite = entry.unwrap(Ignite.class);
+
+            ignite.log().info("TestEntryProcessor called [op=" + op + ", entry=" + entry + ']');
+
+            callCnt.incrementAndGet();
+
+            assertEquals(1, args.length);
+
+            TestReturnValue retVal;
+
+            TestValue val = entry.getValue();
+
+            if (val == null)
+                retVal = new TestReturnValue("null", args[0]);
+            else
+                retVal = new TestReturnValue(String.valueOf(val.value()), args[0]);
+
+            switch (op) {
+                case OP_GET:
+                    return retVal;
+
+                case OP_UPDATE: {
+                    if (val == null)
+                        val = new TestValue(0);
+                    else
+                        val = new TestValue(val.val + 1);
+
+                    entry.setValue(val);
+
+                    break;
+                }
+
+                case OP_REMOVE:
+                    entry.remove();
+
+                    break;
+
+                default:
+                    assert false;
+            }
+
+            return retVal;
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestValue {
+        /** */
+        private Integer val;
+
+        /**
+         * @param val Value.
+         */
+        public TestValue(Integer val) {
+            this.val = val;
+        }
+
+        /**
+         * @return Value.
+         */
+        public Integer value() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestValue testVal = (TestValue) o;
+
+            return val.equals(testVal.val);
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestValue.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestReturnValue {
+        /** */
+        private String val;
+
+        /** */
+        private Object arg;
+
+        /**
+         * @param val Value.
+         * @param arg Entry processor argument.
+         */
+        public TestReturnValue(String val, Object arg) {
+            this.val = val;
+            this.arg = arg;
+        }
+
+        /**
+         * @return Value.
+         */
+        public String value() {
+            return val;
+        }
+
+        /**
+         * @return Entry processor argument.
+         */
+        public Object argument() {
+            return arg;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestReturnValue testVal = (TestReturnValue) o;
+
+            return val.equals(testVal.val);
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestReturnValue.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a14d643/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 a2ae2e1..02eb9d8 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
@@ -121,8 +121,8 @@ public class GridCachePartitionedHitsAndMissesSelfTest extends GridCommonAbstrac
             }
 
             // Check that invoke and loader updated metrics
-            assertEquals(CNT, hits);
-            assertEquals(CNT, misses);
+            assertEquals(CNT / 2, hits);
+            assertEquals(CNT / 2, misses);
         }
         finally {
             stopAllGrids();

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a14d643/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 95661cb..eaf63d7 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -60,6 +60,7 @@ import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
 import org.apache.ignite.internal.util.GridClassLoaderCache;
 import org.apache.ignite.internal.portable.BinaryEnumCache;
@@ -119,6 +120,9 @@ public abstract class GridAbstractTest extends TestCase {
     /** Null name for execution map. */
     private static final String NULL_NAME = UUID.randomUUID().toString();
 
+    /** */
+    private static final boolean BINARY_MARSHALLER = false;
+
     /** Ip finder for TCP discovery. */
     public static final TcpDiscoveryIpFinder LOCAL_IP_FINDER = new TcpDiscoveryVmIpFinder(false) {{
         setAddresses(Collections.singleton("127.0.0.1:47500..47509"));
@@ -155,6 +159,9 @@ public abstract class GridAbstractTest extends TestCase {
         System.setProperty(IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, "10000");
         System.setProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER, "false");
 
+        if (BINARY_MARSHALLER)
+            GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+
         Thread timer = new Thread(new GridTestClockTimer(), "ignite-clock-for-tests");
 
         timer.setDaemon(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a14d643/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index ca31c28..7e45470 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -91,6 +91,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerEager
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerTxLocalTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerTxReplicatedTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerTxTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheEntryProcessorCallTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheManyAsyncOperationsTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheNearLockValueSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheTransactionalStopBusySelfTest;
@@ -167,6 +168,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheAtomicLocalInvokeTest.class);
         suite.addTestSuite(IgniteCacheAtomicLocalWithStoreInvokeTest.class);
         suite.addTestSuite(IgniteCacheTxInvokeTest.class);
+        suite.addTestSuite(IgniteCacheEntryProcessorCallTest.class);
         suite.addTestSuite(IgniteCacheTxNearEnabledInvokeTest.class);
         suite.addTestSuite(IgniteCacheTxLocalInvokeTest.class);
         suite.addTestSuite(IgniteCrossCacheTxStoreSelfTest.class);


[08/50] [abbrv] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

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


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

Branch: refs/heads/ignite-1.5.1
Commit: 5beed16e3af924e367dffec841d39d09d326d110
Parents: dc0131d 50f6c01
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 3 16:54:50 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 3 16:54:50 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryReaderExImpl.java   |   2 +-
 .../internal/portable/PortableContext.java      |  34 +-
 .../processors/cache/GridCacheContext.java      |  30 ++
 .../CacheDataStructuresManager.java             |  31 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  20 +-
 .../CacheObjectBinaryProcessorImpl.java         |   8 +
 .../cache/query/GridCacheQueryManager.java      |  12 +-
 .../transactions/IgniteTxLocalAdapter.java      |  11 +-
 .../datastructures/CollocatedQueueItemKey.java  |  75 ++++
 .../datastructures/CollocatedSetItemKey.java    |  87 +++++
 .../datastructures/DataStructuresProcessor.java |   7 +-
 .../GridAtomicCacheQueueImpl.java               |   8 +-
 .../datastructures/GridCacheQueueAdapter.java   |  30 +-
 .../datastructures/GridCacheQueueItemKey.java   |   9 +-
 .../datastructures/GridCacheSetImpl.java        |  37 +-
 .../datastructures/GridCacheSetItemKey.java     |  21 +-
 .../GridTransactionalCacheQueueImpl.java        |   2 +-
 .../processors/datastructures/QueueItemKey.java |  27 ++
 .../processors/datastructures/SetItemKey.java   |  36 ++
 .../cache/IgniteCacheInvokeAbstractTest.java    | 369 ++++++++++++++-----
 ...eAbstractDataStructuresFailoverSelfTest.java |   7 +-
 .../GridCacheQueueApiSelfAbstractTest.java      |  18 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |   6 +-
 .../GridCachePartitionedQueueApiSelfTest.java   |   5 +
 ...dCachePartitionedQueueEntryMoveSelfTest.java |   2 +-
 .../IgnitePartitionedQueueNoBackupsTest.java    |  92 +++++
 .../GridCacheReplicatedQueueApiSelfTest.java    |   5 +
 .../GridCacheWriteBehindStoreAbstractTest.java  |   2 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |   3 +
 29 files changed, 778 insertions(+), 218 deletions(-)
----------------------------------------------------------------------



[02/50] [abbrv] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

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


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

Branch: refs/heads/ignite-1.5.1
Commit: 9b60c75c62d2e352a4a65b7c5a0b17bc79499ba8
Parents: f770954 ae40e2d
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 3 10:42:50 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 3 10:42:50 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/configuration/IgniteConfiguration.java    | 4 ++--
 .../cache/GridCacheMultinodeUpdateNearEnabledSelfTest.java      | 5 +++++
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[41/50] [abbrv] ignite git commit: IGNITE-1694 : .NET: Use DllImport instead of delegates in P/Invoke.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/568f910f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
index 9a49fae..56a184d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
@@ -19,10 +19,10 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 {
     using System;
     using System.Diagnostics.CodeAnalysis;
-    using System.Globalization;
     using System.Runtime.InteropServices;
     using Apache.Ignite.Core.Common;
-    using Apache.Ignite.Core.Impl.Common;
+
+    using JNI = IgniteJniNativeMethods;
 
     /// <summary>
     /// Unmanaged utility classes.
@@ -32,544 +32,27 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         /** Interop factory ID for .Net. */
         private const int InteropFactoryId = 1;
 
-        #region PROCEDURE NAMES
-
-        private const string ProcReallocate = "IgniteReallocate";
-
-        private const string ProcIgnitionStart = "IgniteIgnitionStart";
-        private const string ProcIgnitionStop = "IgniteIgnitionStop";
-        private const string ProcIgnitionStopAll = "IgniteIgnitionStopAll";
-        
-        private const string ProcProcessorReleaseStart = "IgniteProcessorReleaseStart";
-        private const string ProcProcessorProjection = "IgniteProcessorProjection";
-        private const string ProcProcessorCache = "IgniteProcessorCache";
-        private const string ProcProcessorGetOrCreateCache = "IgniteProcessorGetOrCreateCache";
-        private const string ProcProcessorCreateCache = "IgniteProcessorCreateCache";
-        private const string ProcProcessorAffinity = "IgniteProcessorAffinity";
-        private const string ProcProcessorDataStreamer = "IgniteProcessorDataStreamer";
-        private const string ProcProcessorTransactions = "IgniteProcessorTransactions";
-        private const string ProcProcessorCompute = "IgniteProcessorCompute";
-        private const string ProcProcessorMessage = "IgniteProcessorMessage";
-        private const string ProcProcessorEvents = "IgniteProcessorEvents";
-        private const string ProcProcessorServices = "IgniteProcessorServices";
-        private const string ProcProcessorExtensions = "IgniteProcessorExtensions";
-        private const string ProcProcessorAtomicLong = "IgniteProcessorAtomicLong";
-        
-        private const string ProcTargetInStreamOutLong = "IgniteTargetInStreamOutLong";
-        private const string ProcTargetInStreamOutStream = "IgniteTargetInStreamOutStream";
-        private const string ProcTargetInStreamOutObject = "IgniteTargetInStreamOutObject";
-        private const string ProcTargetInObjectStreamOutStream = "IgniteTargetInObjectStreamOutStream";
-        private const string ProcTargetOutLong = "IgniteTargetOutLong";
-        private const string ProcTargetOutStream = "IgniteTargetOutStream";
-        private const string ProcTargetOutObject = "IgniteTargetOutObject";
-        private const string ProcTargetListenFut = "IgniteTargetListenFuture";
-        private const string ProcTargetListenFutForOp = "IgniteTargetListenFutureForOperation";
-
-        private const string ProcAffinityParts = "IgniteAffinityPartitions";
-
-        private const string ProcCacheWithSkipStore = "IgniteCacheWithSkipStore";
-        private const string ProcCacheWithNoRetries = "IgniteCacheWithNoRetries";
-        private const string ProcCacheWithExpiryPolicy = "IgniteCacheWithExpiryPolicy";
-        private const string ProcCacheWithAsync = "IgniteCacheWithAsync";
-        private const string ProcCacheWithKeepBinary = "IgniteCacheWithKeepPortable";
-        private const string ProcCacheClear = "IgniteCacheClear";
-        private const string ProcCacheRemoveAll = "IgniteCacheRemoveAll";
-        private const string ProcCacheOutOpQueryCursor = "IgniteCacheOutOpQueryCursor";
-        private const string ProcCacheOutOpContinuousQuery = "IgniteCacheOutOpContinuousQuery";
-        private const string ProcCacheIterator = "IgniteCacheIterator";
-        private const string ProcCacheLocalIterator = "IgniteCacheLocalIterator";
-        private const string ProcCacheEnterLock = "IgniteCacheEnterLock";
-        private const string ProcCacheExitLock = "IgniteCacheExitLock";
-        private const string ProcCacheTryEnterLock = "IgniteCacheTryEnterLock";
-        private const string ProcCacheCloseLock = "IgniteCacheCloseLock";
-        private const string ProcCacheRebalance = "IgniteCacheRebalance";
-        private const string ProcCacheSize = "IgniteCacheSize";
-
-        private const string ProcCacheStoreCallbackInvoke = "IgniteCacheStoreCallbackInvoke";
-
-        private const string ProcComputeWithNoFailover = "IgniteComputeWithNoFailover";
-        private const string ProcComputeWithTimeout = "IgniteComputeWithTimeout";
-        private const string ProcComputeExecuteNative = "IgniteComputeExecuteNative";
-
-        private const string ProcContinuousQryClose = "IgniteContinuousQueryClose";
-        private const string ProcContinuousQryGetInitialQueryCursor = "IgniteContinuousQueryGetInitialQueryCursor";
-
-        private const string ProcDataStreamerListenTop = "IgniteDataStreamerListenTopology";
-        private const string ProcDataStreamerAllowOverwriteGet = "IgniteDataStreamerAllowOverwriteGet";
-        private const string ProcDataStreamerAllowOverwriteSet = "IgniteDataStreamerAllowOverwriteSet";
-        private const string ProcDataStreamerSkipStoreGet = "IgniteDataStreamerSkipStoreGet";
-        private const string ProcDataStreamerSkipStoreSet = "IgniteDataStreamerSkipStoreSet";
-        private const string ProcDataStreamerPerNodeBufferSizeGet = "IgniteDataStreamerPerNodeBufferSizeGet";
-        private const string ProcDataStreamerPerNodeBufferSizeSet = "IgniteDataStreamerPerNodeBufferSizeSet";
-        private const string ProcDataStreamerPerNodeParallelOpsGet = "IgniteDataStreamerPerNodeParallelOperationsGet";
-        private const string ProcDataStreamerPerNodeParallelOpsSet = "IgniteDataStreamerPerNodeParallelOperationsSet";
-
-        private const string ProcMessagingWithAsync = "IgniteMessagingWithAsync";
-
-        private const string ProcQryCursorIterator = "IgniteQueryCursorIterator";
-        private const string ProcQryCursorClose = "IgniteQueryCursorClose";
-
-        private const string ProcProjectionForOthers = "IgniteProjectionForOthers";
-        private const string ProcProjectionForRemotes = "IgniteProjectionForRemotes";
-        private const string ProcProjectionForDaemons = "IgniteProjectionForDaemons";
-        private const string ProcProjectionForRandom = "IgniteProjectionForRandom";
-        private const string ProcProjectionForOldest = "IgniteProjectionForOldest";
-        private const string ProcProjectionForYoungest = "IgniteProjectionForYoungest";
-        private const string ProcProjectionResetMetrics = "IgniteProjectionResetMetrics";
-        private const string ProcProjectionOutOpRet = "IgniteProjectionOutOpRet";
-
-        private const string ProcAcquire = "IgniteAcquire";
-        private const string ProcRelease = "IgniteRelease";
-
-        private const string ProcTxStart = "IgniteTransactionsStart";
-        private const string ProcTxCommit = "IgniteTransactionsCommit";
-        private const string ProcTxCommitAsync = "IgniteTransactionsCommitAsync";
-        private const string ProcTxRollback = "IgniteTransactionsRollback";
-        private const string ProcTxRollbackAsync = "IgniteTransactionsRollbackAsync";
-        private const string ProcTxClose = "IgniteTransactionsClose";
-        private const string ProcTxState = "IgniteTransactionsState";
-        private const string ProcTxSetRollbackOnly = "IgniteTransactionsSetRollbackOnly";
-        private const string ProcTxResetMetrics = "IgniteTransactionsResetMetrics";
-
-        private const string ProcThrowToJava = "IgniteThrowToJava";
-
-        private const string ProcDestroyJvm = "IgniteDestroyJvm";
-
-        private const string ProcHandlersSize = "IgniteHandlersSize";
-
-        private const string ProcCreateContext = "IgniteCreateContext";
-        
-        private const string ProcEventsWithAsync = "IgniteEventsWithAsync";
-        private const string ProcEventsStopLocalListen = "IgniteEventsStopLocalListen";
-        private const string ProcEventsLocalListen = "IgniteEventsLocalListen";
-        private const string ProcEventsIsEnabled = "IgniteEventsIsEnabled";
-
-        private const string ProcDeleteContext = "IgniteDeleteContext";
-        
-        private const string ProcServicesWithAsync = "IgniteServicesWithAsync";
-        private const string ProcServicesWithServerKeepBinary = "IgniteServicesWithServerKeepPortable";
-        private const string ProcServicesCancel = "IgniteServicesCancel";
-        private const string ProcServicesCancelAll = "IgniteServicesCancelAll";
-        private const string ProcServicesGetServiceProxy = "IgniteServicesGetServiceProxy";
-
-        private const string ProcAtomicLongGet = "IgniteAtomicLongGet";
-        private const string ProcAtomicLongIncrementAndGet = "IgniteAtomicLongIncrementAndGet";
-        private const string ProcAtomicLongAddAndGet = "IgniteAtomicLongAddAndGet";
-        private const string ProcAtomicLongDecrementAndGet = "IgniteAtomicLongDecrementAndGet";
-        private const string ProcAtomicLongGetAndSet = "IgniteAtomicLongGetAndSet";
-        private const string ProcAtomicLongCompareAndSetAndGet = "IgniteAtomicLongCompareAndSetAndGet";
-        private const string ProcAtomicLongIsClosed = "IgniteAtomicLongIsClosed";
-        private const string ProcAtomicLongClose = "IgniteAtomicLongClose";
-
-        #endregion
-
-        #region DELEGATE DEFINITIONS
-
-        private delegate int ReallocateDelegate(long memPtr, int cap);
-
-        private delegate void* IgnitionStartDelegate(void* ctx, sbyte* cfgPath, sbyte* gridName, int factoryId, long dataPtr);
-        private delegate bool IgnitionStopDelegate(void* ctx, sbyte* gridName, bool cancel);
-        private delegate void IgnitionStopAllDelegate(void* ctx, bool cancel);
-
-        private delegate void ProcessorReleaseStartDelegate(void* ctx, void* obj);
-        private delegate void* ProcessorProjectionDelegate(void* ctx, void* obj);
-        private delegate void* ProcessorCacheDelegate(void* ctx, void* obj, sbyte* name);
-        private delegate void* ProcessorCreateCacheDelegate(void* ctx, void* obj, sbyte* name);
-        private delegate void* ProcessorGetOrCreateCacheDelegate(void* ctx, void* obj, sbyte* name);
-        private delegate void* ProcessorAffinityDelegate(void* ctx, void* obj, sbyte* name);
-        private delegate void* ProcessorDataStreamerDelegate(void* ctx, void* obj, sbyte* name, bool keepBinary);
-        private delegate void* ProcessorTransactionsDelegate(void* ctx, void* obj);
-        private delegate void* ProcessorComputeDelegate(void* ctx, void* obj, void* prj);
-        private delegate void* ProcessorMessageDelegate(void* ctx, void* obj, void* prj);
-        private delegate void* ProcessorEventsDelegate(void* ctx, void* obj, void* prj);
-        private delegate void* ProcessorServicesDelegate(void* ctx, void* obj, void* prj);
-        private delegate void* ProcessorExtensionsDelegate(void* ctx, void* obj);
-        private delegate void* ProcessorAtomicLongDelegate(void* ctx, void* obj, sbyte* name, long initVal, bool create);
-        
-        private delegate long TargetInStreamOutLongDelegate(void* ctx, void* target, int opType, long memPtr);
-        private delegate void TargetInStreamOutStreamDelegate(void* ctx, void* target, int opType, long inMemPtr, long outMemPtr);
-        private delegate void* TargetInStreamOutObjectDelegate(void* ctx, void* target, int opType, long memPtr);
-        private delegate void TargetInObjectStreamOutStreamDelegate(void* ctx, void* target, int opType, void* arg, long inMemPtr, long outMemPtr);
-        private delegate long TargetOutLongDelegate(void* ctx, void* target, int opType);
-        private delegate void TargetOutStreamDelegate(void* ctx, void* target, int opType, long memPtr);
-        private delegate void* TargetOutObjectDelegate(void* ctx, void* target, int opType);
-        private delegate void TargetListenFutureDelegate(void* ctx, void* target, long futId, int typ);
-        private delegate void TargetListenFutureForOpDelegate(void* ctx, void* target, long futId, int typ, int opId);
-
-        private delegate int AffinityPartitionsDelegate(void* ctx, void* target);
-
-        private delegate void* CacheWithSkipStoreDelegate(void* ctx, void* obj);
-        private delegate void* CacheNoRetriesDelegate(void* ctx, void* obj);
-        private delegate void* CacheWithExpiryPolicyDelegate(void* ctx, void* obj, long create, long update, long access);
-        private delegate void* CacheWithAsyncDelegate(void* ctx, void* obj);
-        private delegate void* CacheWithKeepBinaryDelegate(void* ctx, void* obj);
-        private delegate void CacheClearDelegate(void* ctx, void* obj);
-        private delegate void CacheRemoveAllDelegate(void* ctx, void* obj);
-        private delegate void* CacheOutOpQueryCursorDelegate(void* ctx, void* obj, int type, long memPtr);
-        private delegate void* CacheOutOpContinuousQueryDelegate(void* ctx, void* obj, int type, long memPtr);
-        private delegate void* CacheIteratorDelegate(void* ctx, void* obj);
-        private delegate void* CacheLocalIteratorDelegate(void* ctx, void* obj, int peekModes);
-        private delegate void CacheEnterLockDelegate(void* ctx, void* obj, long id);
-        private delegate void CacheExitLockDelegate(void* ctx, void* obj, long id);
-        private delegate bool CacheTryEnterLockDelegate(void* ctx, void* obj, long id, long timeout);
-        private delegate void CacheCloseLockDelegate(void* ctx, void* obj, long id);
-        private delegate void CacheRebalanceDelegate(void* ctx, void* obj, long futId);
-        private delegate int CacheSizeDelegate(void* ctx, void* obj, int peekModes, bool loc);
-
-        private delegate void CacheStoreCallbackInvokeDelegate(void* ctx, void* obj, long memPtr);
-
-        private delegate void ComputeWithNoFailoverDelegate(void* ctx, void* target);
-        private delegate void ComputeWithTimeoutDelegate(void* ctx, void* target, long timeout);
-        private delegate void ComputeExecuteNativeDelegate(void* ctx, void* target, long taskPtr, long topVer);
-
-        private delegate void ContinuousQueryCloseDelegate(void* ctx, void* target);
-        private delegate void* ContinuousQueryGetInitialQueryCursorDelegate(void* ctx, void* target);
-
-        private delegate void DataStreamerListenTopologyDelegate(void* ctx, void* obj, long ptr);
-        private delegate bool DataStreamerAllowOverwriteGetDelegate(void* ctx, void* obj);
-        private delegate void DataStreamerAllowOverwriteSetDelegate(void* ctx, void* obj, bool val);
-        private delegate bool DataStreamerSkipStoreGetDelegate(void* ctx, void* obj);
-        private delegate void DataStreamerSkipStoreSetDelegate(void* ctx, void* obj, bool val);
-        private delegate int DataStreamerPerNodeBufferSizeGetDelegate(void* ctx, void* obj);
-        private delegate void DataStreamerPerNodeBufferSizeSetDelegate(void* ctx, void* obj, int val);
-        private delegate int DataStreamerPerNodeParallelOperationsGetDelegate(void* ctx, void* obj);
-        private delegate void DataStreamerPerNodeParallelOperationsSetDelegate(void* ctx, void* obj, int val);
-
-        private delegate void* MessagingWithAsyncDelegate(void* ctx, void* target);
-
-        private delegate void* ProjectionForOthersDelegate(void* ctx, void* obj, void* prj);
-		private delegate void* ProjectionForRemotesDelegate(void* ctx, void* obj);
-		private delegate void* ProjectionForDaemonsDelegate(void* ctx, void* obj);
-		private delegate void* ProjectionForRandomDelegate(void* ctx, void* obj);
-		private delegate void* ProjectionForOldestDelegate(void* ctx, void* obj);
-		private delegate void* ProjectionForYoungestDelegate(void* ctx, void* obj);
-		private delegate void ProjectionResetMetricsDelegate(void* ctx, void* obj);
-		private delegate void* ProjectionOutOpRetDelegate(void* ctx, void* obj, int type, long memPtr);
-
-        private delegate void QueryCursorIteratorDelegate(void* ctx, void* target);
-        private delegate void QueryCursorCloseDelegate(void* ctx, void* target);
-
-        private delegate void* AcquireDelegate(void* ctx, void* target);
-        private delegate void ReleaseDelegate(void* target);
-
-        private delegate long TransactionsStartDelegate(void* ctx, void* target, int concurrency, int isolation, long timeout, int txSize);
-        private delegate int TransactionsCommitDelegate(void* ctx, void* target, long id);
-        private delegate void TransactionsCommitAsyncDelegate(void* ctx, void* target, long id, long futId);
-        private delegate int TransactionsRollbackDelegate(void* ctx, void* target, long id);
-        private delegate void TransactionsRollbackAsyncDelegate(void* ctx, void* target, long id, long futId);
-        private delegate int TransactionsCloseDelegate(void* ctx, void* target, long id);
-        private delegate int TransactionsStateDelegate(void* ctx, void* target, long id);
-        private delegate bool TransactionsSetRollbackOnlyDelegate(void* ctx, void* target, long id);
-        private delegate void TransactionsResetMetricsDelegate(void* ctx, void* target);
-
-        private delegate void ThrowToJavaDelegate(void* ctx, char* msg);
-
-        private delegate void DestroyJvmDelegate(void* ctx);
-
-        private delegate int HandlersSizeDelegate();
-
-        private delegate void* CreateContextDelegate(void* opts, int optsLen, void* cbs);
-        
-        private delegate void* EventsWithAsyncDelegate(void* ctx, void* obj);
-        private delegate bool EventsStopLocalListenDelegate(void* ctx, void* obj, long hnd);
-        private delegate void EventsLocalListenDelegate(void* ctx, void* obj, long hnd, int type);
-        private delegate bool EventsIsEnabledDelegate(void* ctx, void* obj, int type);
-
-        private delegate void DeleteContextDelegate(void* ptr);
-
-        private delegate void* ServicesWithAsyncDelegate(void* ctx, void* target);
-        private delegate void* ServicesWithServerKeepBinaryDelegate(void* ctx, void* target);
-        private delegate long ServicesCancelDelegate(void* ctx, void* target, char* name);
-        private delegate long ServicesCancelAllDelegate(void* ctx, void* target);
-        private delegate void* ServicesGetServiceProxyDelegate(void* ctx, void* target, char* name, bool sticky);
-
-        private delegate long AtomicLongGetDelegate(void* ctx, void* target);
-        private delegate long AtomicLongIncrementAndGetDelegate(void* ctx, void* target);
-        private delegate long AtomicLongAddAndGetDelegate(void* ctx, void* target, long value);
-        private delegate long AtomicLongDecrementAndGetDelegate(void* ctx, void* target);
-        private delegate long AtomicLongGetAndSetDelegate(void* ctx, void* target, long value);
-        private delegate long AtomicLongCompareAndSetAndGetDelegate(void* ctx, void* target, long expVal, long newVal);
-        private delegate bool AtomicLongIsClosedDelegate(void* ctx, void* target);
-        private delegate void AtomicLongCloseDelegate(void* ctx, void* target);
-
-        #endregion
-
-        #region DELEGATE MEMBERS
-
-        // ReSharper disable InconsistentNaming
-        private static readonly ReallocateDelegate REALLOCATE;
-
-        private static readonly IgnitionStartDelegate IGNITION_START;
-        private static readonly IgnitionStopDelegate IGNITION_STOP;
-        private static readonly IgnitionStopAllDelegate IGNITION_STOP_ALL;
-
-        private static readonly ProcessorReleaseStartDelegate PROCESSOR_RELEASE_START;
-        private static readonly ProcessorProjectionDelegate PROCESSOR_PROJECTION;
-        private static readonly ProcessorCacheDelegate PROCESSOR_CACHE;
-        private static readonly ProcessorCreateCacheDelegate PROCESSOR_CREATE_CACHE;
-        private static readonly ProcessorGetOrCreateCacheDelegate PROCESSOR_GET_OR_CREATE_CACHE;
-        private static readonly ProcessorAffinityDelegate PROCESSOR_AFFINITY;
-        private static readonly ProcessorDataStreamerDelegate PROCESSOR_DATA_STREAMER;
-        private static readonly ProcessorTransactionsDelegate PROCESSOR_TRANSACTIONS;
-        private static readonly ProcessorComputeDelegate PROCESSOR_COMPUTE;
-        private static readonly ProcessorMessageDelegate PROCESSOR_MESSAGE;
-        private static readonly ProcessorEventsDelegate PROCESSOR_EVENTS;
-        private static readonly ProcessorServicesDelegate PROCESSOR_SERVICES;
-        private static readonly ProcessorExtensionsDelegate PROCESSOR_EXTENSIONS;
-        private static readonly ProcessorAtomicLongDelegate PROCESSOR_ATOMIC_LONG;
-        
-        private static readonly TargetInStreamOutLongDelegate TARGET_IN_STREAM_OUT_LONG;
-        private static readonly TargetInStreamOutStreamDelegate TARGET_IN_STREAM_OUT_STREAM;
-        private static readonly TargetInStreamOutObjectDelegate TARGET_IN_STREAM_OUT_OBJECT;
-        private static readonly TargetInObjectStreamOutStreamDelegate TARGET_IN_OBJECT_STREAM_OUT_STREAM;
-        private static readonly TargetOutLongDelegate TARGET_OUT_LONG;
-        private static readonly TargetOutStreamDelegate TARGET_OUT_STREAM;
-        private static readonly TargetOutObjectDelegate TARGET_OUT_OBJECT;
-        private static readonly TargetListenFutureDelegate TargetListenFut;
-        private static readonly TargetListenFutureForOpDelegate TargetListenFutForOp;
-
-        private static readonly AffinityPartitionsDelegate AffinityParts;
-
-        private static readonly CacheWithSkipStoreDelegate CACHE_WITH_SKIP_STORE;
-        private static readonly CacheNoRetriesDelegate CACHE_WITH_NO_RETRIES;
-        private static readonly CacheWithExpiryPolicyDelegate CACHE_WITH_EXPIRY_POLICY;
-        private static readonly CacheWithAsyncDelegate CACHE_WITH_ASYNC;
-        private static readonly CacheWithKeepBinaryDelegate CACHE_WITH_KEEP_BINARY;
-        private static readonly CacheClearDelegate CACHE_CLEAR;
-        private static readonly CacheRemoveAllDelegate CACHE_REMOVE_ALL;
-        private static readonly CacheOutOpQueryCursorDelegate CACHE_OUT_OP_QUERY_CURSOR;
-        private static readonly CacheOutOpContinuousQueryDelegate CACHE_OUT_OP_CONTINUOUS_QUERY;
-        private static readonly CacheIteratorDelegate CACHE_ITERATOR;
-        private static readonly CacheLocalIteratorDelegate CACHE_LOCAL_ITERATOR;
-        private static readonly CacheEnterLockDelegate CACHE_ENTER_LOCK;
-        private static readonly CacheExitLockDelegate CACHE_EXIT_LOCK;
-        private static readonly CacheTryEnterLockDelegate CACHE_TRY_ENTER_LOCK;
-        private static readonly CacheCloseLockDelegate CACHE_CLOSE_LOCK;
-        private static readonly CacheRebalanceDelegate CACHE_REBALANCE;
-        private static readonly CacheSizeDelegate CACHE_SIZE;
-
-        private static readonly CacheStoreCallbackInvokeDelegate CACHE_STORE_CALLBACK_INVOKE;
-
-        private static readonly ComputeWithNoFailoverDelegate COMPUTE_WITH_NO_FAILOVER;
-        private static readonly ComputeWithTimeoutDelegate COMPUTE_WITH_TIMEOUT;
-        private static readonly ComputeExecuteNativeDelegate COMPUTE_EXECUTE_NATIVE;
-
-        private static readonly ContinuousQueryCloseDelegate ContinuousQryClose;
-        private static readonly ContinuousQueryGetInitialQueryCursorDelegate ContinuousQryGetInitialQueryCursor;
-
-        private static readonly DataStreamerListenTopologyDelegate DataStreamerListenTop;
-        private static readonly DataStreamerAllowOverwriteGetDelegate DATA_STREAMER_ALLOW_OVERWRITE_GET;
-        private static readonly DataStreamerAllowOverwriteSetDelegate DATA_STREAMER_ALLOW_OVERWRITE_SET;
-        private static readonly DataStreamerSkipStoreGetDelegate DATA_STREAMER_SKIP_STORE_GET;
-        private static readonly DataStreamerSkipStoreSetDelegate DATA_STREAMER_SKIP_STORE_SET;
-        private static readonly DataStreamerPerNodeBufferSizeGetDelegate DATA_STREAMER_PER_NODE_BUFFER_SIZE_GET;
-        private static readonly DataStreamerPerNodeBufferSizeSetDelegate DATA_STREAMER_PER_NODE_BUFFER_SIZE_SET;
-        private static readonly DataStreamerPerNodeParallelOperationsGetDelegate DataStreamerPerNodeParallelOpsGet;
-        private static readonly DataStreamerPerNodeParallelOperationsSetDelegate DataStreamerPerNodeParallelOpsSet;
-
-        private static readonly MessagingWithAsyncDelegate MessagingWithAsync;
-
-        private static readonly ProjectionForOthersDelegate PROJECTION_FOR_OTHERS;
-        private static readonly ProjectionForRemotesDelegate PROJECTION_FOR_REMOTES;
-        private static readonly ProjectionForDaemonsDelegate PROJECTION_FOR_DAEMONS;
-        private static readonly ProjectionForRandomDelegate PROJECTION_FOR_RANDOM;
-        private static readonly ProjectionForOldestDelegate PROJECTION_FOR_OLDEST;
-        private static readonly ProjectionForYoungestDelegate PROJECTION_FOR_YOUNGEST;
-        private static readonly ProjectionResetMetricsDelegate PROJECTION_RESET_METRICS;
-        private static readonly ProjectionOutOpRetDelegate PROJECTION_OUT_OP_RET;
-
-        private static readonly QueryCursorIteratorDelegate QryCursorIterator;
-        private static readonly QueryCursorCloseDelegate QryCursorClose;
-
-        private static readonly AcquireDelegate ACQUIRE;
-        private static readonly ReleaseDelegate RELEASE;
-
-        private static readonly TransactionsStartDelegate TxStart;
-        private static readonly TransactionsCommitDelegate TxCommit;
-        private static readonly TransactionsCommitAsyncDelegate TxCommitAsync;
-        private static readonly TransactionsRollbackDelegate TxRollback;
-        private static readonly TransactionsRollbackAsyncDelegate TxRollbackAsync;
-        private static readonly TransactionsCloseDelegate TxClose;
-        private static readonly TransactionsStateDelegate TxState;
-        private static readonly TransactionsSetRollbackOnlyDelegate TxSetRollbackOnly;
-        private static readonly TransactionsResetMetricsDelegate TxResetMetrics;
-
-        private static readonly ThrowToJavaDelegate THROW_TO_JAVA;
-
-        private static readonly DestroyJvmDelegate DESTROY_JVM;
-
-        private static readonly HandlersSizeDelegate HANDLERS_SIZE;
-
-        private static readonly CreateContextDelegate CREATE_CONTEXT;
-        
-        private static readonly EventsWithAsyncDelegate EVENTS_WITH_ASYNC;
-        private static readonly EventsStopLocalListenDelegate EVENTS_STOP_LOCAL_LISTEN;
-        private static readonly EventsLocalListenDelegate EVENTS_LOCAL_LISTEN;
-        private static readonly EventsIsEnabledDelegate EVENTS_IS_ENABLED;
- 
-        private static readonly DeleteContextDelegate DELETE_CONTEXT;
-        
-        private static readonly ServicesWithAsyncDelegate SERVICES_WITH_ASYNC;
-        private static readonly ServicesWithServerKeepBinaryDelegate SERVICES_WITH_SERVER_KEEP_BINARY;
-        private static readonly ServicesCancelDelegate SERVICES_CANCEL;
-        private static readonly ServicesCancelAllDelegate SERVICES_CANCEL_ALL;
-        private static readonly ServicesGetServiceProxyDelegate SERVICES_GET_SERVICE_PROXY;
-
-        private static readonly AtomicLongGetDelegate ATOMIC_LONG_GET;
-        private static readonly AtomicLongIncrementAndGetDelegate ATOMIC_LONG_INCREMENT_AND_GET;
-        private static readonly AtomicLongAddAndGetDelegate ATOMIC_LONG_ADD_AND_GET;
-        private static readonly AtomicLongDecrementAndGetDelegate ATOMIC_LONG_DECREMENT_AND_GET;
-        private static readonly AtomicLongGetAndSetDelegate ATOMIC_LONG_GET_AND_SET;
-        private static readonly AtomicLongCompareAndSetAndGetDelegate ATOMIC_LONG_COMPARE_AND_SET_AND_GET;
-        private static readonly AtomicLongIsClosedDelegate ATOMIC_LONG_IS_CLOSED;
-        private static readonly AtomicLongCloseDelegate ATOMIC_LONG_CLOSE;
-
-        // ReSharper restore InconsistentNaming
-
-        #endregion
-
-        /** Library pointer. */
-        private static readonly IntPtr Ptr;
-
         /// <summary>
         /// Initializer.
         /// </summary>
         [SuppressMessage("Microsoft.Design", "CA1065:DoNotRaiseExceptionsInUnexpectedLocations")]
-        [SuppressMessage("Microsoft.Performance", "CA1810:InitializeReferenceTypeStaticFieldsInline")]
         static UnmanagedUtils()
         {
             var path = IgniteUtils.UnpackEmbeddedResource(IgniteUtils.FileIgniteJniDll);
 
-            Ptr = NativeMethods.LoadLibrary(path);
-
-            if (Ptr == IntPtr.Zero)
-                throw new IgniteException("Failed to load " + IgniteUtils.FileIgniteJniDll + ": " + Marshal.GetLastWin32Error());
-
-            REALLOCATE = CreateDelegate<ReallocateDelegate>(ProcReallocate);
-
-            IGNITION_START = CreateDelegate<IgnitionStartDelegate>(ProcIgnitionStart);
-            IGNITION_STOP = CreateDelegate<IgnitionStopDelegate>(ProcIgnitionStop);
-            IGNITION_STOP_ALL = CreateDelegate<IgnitionStopAllDelegate>(ProcIgnitionStopAll);
-            
-            PROCESSOR_RELEASE_START = CreateDelegate<ProcessorReleaseStartDelegate>(ProcProcessorReleaseStart);
-            PROCESSOR_PROJECTION = CreateDelegate<ProcessorProjectionDelegate>(ProcProcessorProjection);
-            PROCESSOR_CACHE = CreateDelegate<ProcessorCacheDelegate>(ProcProcessorCache);
-            PROCESSOR_CREATE_CACHE = CreateDelegate<ProcessorCreateCacheDelegate>(ProcProcessorCreateCache);
-            PROCESSOR_GET_OR_CREATE_CACHE = CreateDelegate<ProcessorGetOrCreateCacheDelegate>(ProcProcessorGetOrCreateCache);
-            PROCESSOR_AFFINITY = CreateDelegate<ProcessorAffinityDelegate>(ProcProcessorAffinity);
-            PROCESSOR_DATA_STREAMER = CreateDelegate<ProcessorDataStreamerDelegate>(ProcProcessorDataStreamer);
-            PROCESSOR_TRANSACTIONS = CreateDelegate<ProcessorTransactionsDelegate>(ProcProcessorTransactions);
-            PROCESSOR_COMPUTE = CreateDelegate<ProcessorComputeDelegate>(ProcProcessorCompute);
-            PROCESSOR_MESSAGE = CreateDelegate<ProcessorMessageDelegate>(ProcProcessorMessage);
-            PROCESSOR_EVENTS = CreateDelegate<ProcessorEventsDelegate>(ProcProcessorEvents);
-            PROCESSOR_SERVICES = CreateDelegate<ProcessorServicesDelegate>(ProcProcessorServices);
-            PROCESSOR_EXTENSIONS = CreateDelegate<ProcessorExtensionsDelegate>(ProcProcessorExtensions);
-            PROCESSOR_ATOMIC_LONG = CreateDelegate<ProcessorAtomicLongDelegate>(ProcProcessorAtomicLong);
-            
-            TARGET_IN_STREAM_OUT_LONG = CreateDelegate<TargetInStreamOutLongDelegate>(ProcTargetInStreamOutLong);
-            TARGET_IN_STREAM_OUT_STREAM = CreateDelegate<TargetInStreamOutStreamDelegate>(ProcTargetInStreamOutStream);
-            TARGET_IN_STREAM_OUT_OBJECT = CreateDelegate<TargetInStreamOutObjectDelegate>(ProcTargetInStreamOutObject);
-            TARGET_IN_OBJECT_STREAM_OUT_STREAM = CreateDelegate<TargetInObjectStreamOutStreamDelegate>(ProcTargetInObjectStreamOutStream);
-            TARGET_OUT_LONG = CreateDelegate<TargetOutLongDelegate>(ProcTargetOutLong);
-            TARGET_OUT_STREAM = CreateDelegate<TargetOutStreamDelegate>(ProcTargetOutStream);
-            TARGET_OUT_OBJECT = CreateDelegate<TargetOutObjectDelegate>(ProcTargetOutObject);
-            TargetListenFut = CreateDelegate<TargetListenFutureDelegate>(ProcTargetListenFut);
-            TargetListenFutForOp = CreateDelegate<TargetListenFutureForOpDelegate>(ProcTargetListenFutForOp);
-
-            AffinityParts = CreateDelegate<AffinityPartitionsDelegate>(ProcAffinityParts);
-
-            CACHE_WITH_SKIP_STORE = CreateDelegate<CacheWithSkipStoreDelegate>(ProcCacheWithSkipStore);
-            CACHE_WITH_NO_RETRIES = CreateDelegate<CacheNoRetriesDelegate>(ProcCacheWithNoRetries);
-            CACHE_WITH_EXPIRY_POLICY = CreateDelegate<CacheWithExpiryPolicyDelegate>(ProcCacheWithExpiryPolicy);
-            CACHE_WITH_ASYNC = CreateDelegate<CacheWithAsyncDelegate>(ProcCacheWithAsync);
-            CACHE_WITH_KEEP_BINARY = CreateDelegate<CacheWithKeepBinaryDelegate>(ProcCacheWithKeepBinary);
-            CACHE_CLEAR = CreateDelegate<CacheClearDelegate>(ProcCacheClear);
-            CACHE_REMOVE_ALL = CreateDelegate<CacheRemoveAllDelegate>(ProcCacheRemoveAll);
-            CACHE_OUT_OP_QUERY_CURSOR = CreateDelegate<CacheOutOpQueryCursorDelegate>(ProcCacheOutOpQueryCursor);
-            CACHE_OUT_OP_CONTINUOUS_QUERY = CreateDelegate<CacheOutOpContinuousQueryDelegate>(ProcCacheOutOpContinuousQuery);
-            CACHE_ITERATOR = CreateDelegate<CacheIteratorDelegate>(ProcCacheIterator);
-            CACHE_LOCAL_ITERATOR = CreateDelegate<CacheLocalIteratorDelegate>(ProcCacheLocalIterator);
-            CACHE_ENTER_LOCK = CreateDelegate<CacheEnterLockDelegate>(ProcCacheEnterLock);
-            CACHE_EXIT_LOCK = CreateDelegate<CacheExitLockDelegate>(ProcCacheExitLock);
-            CACHE_TRY_ENTER_LOCK = CreateDelegate<CacheTryEnterLockDelegate>(ProcCacheTryEnterLock);
-            CACHE_CLOSE_LOCK = CreateDelegate<CacheCloseLockDelegate>(ProcCacheCloseLock);
-            CACHE_REBALANCE = CreateDelegate<CacheRebalanceDelegate>(ProcCacheRebalance);
-            CACHE_SIZE = CreateDelegate<CacheSizeDelegate>(ProcCacheSize);
-
-            CACHE_STORE_CALLBACK_INVOKE = CreateDelegate<CacheStoreCallbackInvokeDelegate>(ProcCacheStoreCallbackInvoke);
-
-            COMPUTE_WITH_NO_FAILOVER = CreateDelegate<ComputeWithNoFailoverDelegate>(ProcComputeWithNoFailover);
-            COMPUTE_WITH_TIMEOUT = CreateDelegate<ComputeWithTimeoutDelegate>(ProcComputeWithTimeout);
-            COMPUTE_EXECUTE_NATIVE = CreateDelegate<ComputeExecuteNativeDelegate>(ProcComputeExecuteNative);
-
-            ContinuousQryClose = CreateDelegate<ContinuousQueryCloseDelegate>(ProcContinuousQryClose);
-            ContinuousQryGetInitialQueryCursor = CreateDelegate<ContinuousQueryGetInitialQueryCursorDelegate>(ProcContinuousQryGetInitialQueryCursor);
-
-            DataStreamerListenTop = CreateDelegate<DataStreamerListenTopologyDelegate>(ProcDataStreamerListenTop); 
-            DATA_STREAMER_ALLOW_OVERWRITE_GET = CreateDelegate<DataStreamerAllowOverwriteGetDelegate>(ProcDataStreamerAllowOverwriteGet);
-            DATA_STREAMER_ALLOW_OVERWRITE_SET = CreateDelegate<DataStreamerAllowOverwriteSetDelegate>(ProcDataStreamerAllowOverwriteSet); 
-            DATA_STREAMER_SKIP_STORE_GET = CreateDelegate<DataStreamerSkipStoreGetDelegate>(ProcDataStreamerSkipStoreGet); 
-            DATA_STREAMER_SKIP_STORE_SET = CreateDelegate<DataStreamerSkipStoreSetDelegate>(ProcDataStreamerSkipStoreSet); 
-            DATA_STREAMER_PER_NODE_BUFFER_SIZE_GET = CreateDelegate<DataStreamerPerNodeBufferSizeGetDelegate>(ProcDataStreamerPerNodeBufferSizeGet); 
-            DATA_STREAMER_PER_NODE_BUFFER_SIZE_SET = CreateDelegate<DataStreamerPerNodeBufferSizeSetDelegate>(ProcDataStreamerPerNodeBufferSizeSet); 
-            DataStreamerPerNodeParallelOpsGet = CreateDelegate<DataStreamerPerNodeParallelOperationsGetDelegate>(ProcDataStreamerPerNodeParallelOpsGet); 
-            DataStreamerPerNodeParallelOpsSet = CreateDelegate<DataStreamerPerNodeParallelOperationsSetDelegate>(ProcDataStreamerPerNodeParallelOpsSet); 
-
-            MessagingWithAsync = CreateDelegate<MessagingWithAsyncDelegate>(ProcMessagingWithAsync);
-
-            PROJECTION_FOR_OTHERS = CreateDelegate<ProjectionForOthersDelegate>(ProcProjectionForOthers);
-            PROJECTION_FOR_REMOTES = CreateDelegate<ProjectionForRemotesDelegate>(ProcProjectionForRemotes);
-            PROJECTION_FOR_DAEMONS = CreateDelegate<ProjectionForDaemonsDelegate>(ProcProjectionForDaemons);
-            PROJECTION_FOR_RANDOM = CreateDelegate<ProjectionForRandomDelegate>(ProcProjectionForRandom);
-            PROJECTION_FOR_OLDEST = CreateDelegate<ProjectionForOldestDelegate>(ProcProjectionForOldest);
-            PROJECTION_FOR_YOUNGEST = CreateDelegate<ProjectionForYoungestDelegate>(ProcProjectionForYoungest);
-            PROJECTION_RESET_METRICS = CreateDelegate<ProjectionResetMetricsDelegate>(ProcProjectionResetMetrics);
-            PROJECTION_OUT_OP_RET = CreateDelegate<ProjectionOutOpRetDelegate>(ProcProjectionOutOpRet);
-
-            QryCursorIterator = CreateDelegate<QueryCursorIteratorDelegate>(ProcQryCursorIterator);
-            QryCursorClose = CreateDelegate<QueryCursorCloseDelegate>(ProcQryCursorClose);
-
-            ACQUIRE = CreateDelegate<AcquireDelegate>(ProcAcquire);
-            RELEASE = CreateDelegate<ReleaseDelegate>(ProcRelease);
-
-            TxStart = CreateDelegate<TransactionsStartDelegate>(ProcTxStart);
-            TxCommit = CreateDelegate<TransactionsCommitDelegate>(ProcTxCommit);
-            TxCommitAsync = CreateDelegate<TransactionsCommitAsyncDelegate>(ProcTxCommitAsync);
-            TxRollback = CreateDelegate<TransactionsRollbackDelegate>(ProcTxRollback);
-            TxRollbackAsync = CreateDelegate<TransactionsRollbackAsyncDelegate>(ProcTxRollbackAsync);
-            TxClose = CreateDelegate<TransactionsCloseDelegate>(ProcTxClose);
-            TxState = CreateDelegate<TransactionsStateDelegate>(ProcTxState);
-            TxSetRollbackOnly = CreateDelegate<TransactionsSetRollbackOnlyDelegate>(ProcTxSetRollbackOnly);
-            TxResetMetrics = CreateDelegate<TransactionsResetMetricsDelegate>(ProcTxResetMetrics);
-
-            THROW_TO_JAVA = CreateDelegate<ThrowToJavaDelegate>(ProcThrowToJava);
-
-            HANDLERS_SIZE = CreateDelegate<HandlersSizeDelegate>(ProcHandlersSize);
-
-            CREATE_CONTEXT = CreateDelegate<CreateContextDelegate>(ProcCreateContext);
-            DELETE_CONTEXT = CreateDelegate<DeleteContextDelegate>(ProcDeleteContext);
-
-            DESTROY_JVM = CreateDelegate<DestroyJvmDelegate>(ProcDestroyJvm);
-
-            EVENTS_WITH_ASYNC = CreateDelegate<EventsWithAsyncDelegate>(ProcEventsWithAsync);
-            EVENTS_STOP_LOCAL_LISTEN = CreateDelegate<EventsStopLocalListenDelegate>(ProcEventsStopLocalListen);
-            EVENTS_LOCAL_LISTEN = CreateDelegate<EventsLocalListenDelegate>(ProcEventsLocalListen);
-            EVENTS_IS_ENABLED = CreateDelegate<EventsIsEnabledDelegate>(ProcEventsIsEnabled);
-            
-            SERVICES_WITH_ASYNC = CreateDelegate<ServicesWithAsyncDelegate>(ProcServicesWithAsync);
-            SERVICES_WITH_SERVER_KEEP_BINARY = CreateDelegate<ServicesWithServerKeepBinaryDelegate>(ProcServicesWithServerKeepBinary);
-            SERVICES_CANCEL = CreateDelegate<ServicesCancelDelegate>(ProcServicesCancel);
-            SERVICES_CANCEL_ALL = CreateDelegate<ServicesCancelAllDelegate>(ProcServicesCancelAll);
-            SERVICES_GET_SERVICE_PROXY = CreateDelegate<ServicesGetServiceProxyDelegate>(ProcServicesGetServiceProxy);
-
-            ATOMIC_LONG_GET = CreateDelegate<AtomicLongGetDelegate>(ProcAtomicLongGet);
-            ATOMIC_LONG_INCREMENT_AND_GET = CreateDelegate<AtomicLongIncrementAndGetDelegate>(ProcAtomicLongIncrementAndGet);
-            ATOMIC_LONG_ADD_AND_GET = CreateDelegate<AtomicLongAddAndGetDelegate>(ProcAtomicLongAddAndGet);
-            ATOMIC_LONG_DECREMENT_AND_GET = CreateDelegate<AtomicLongDecrementAndGetDelegate>(ProcAtomicLongDecrementAndGet);
-            ATOMIC_LONG_GET_AND_SET = CreateDelegate<AtomicLongGetAndSetDelegate>(ProcAtomicLongGetAndSet);
-            ATOMIC_LONG_COMPARE_AND_SET_AND_GET = CreateDelegate<AtomicLongCompareAndSetAndGetDelegate>(ProcAtomicLongCompareAndSetAndGet);
-            ATOMIC_LONG_IS_CLOSED = CreateDelegate<AtomicLongIsClosedDelegate>(ProcAtomicLongIsClosed);
-            ATOMIC_LONG_CLOSE = CreateDelegate<AtomicLongCloseDelegate>(ProcAtomicLongClose);
+            var ptr = NativeMethods.LoadLibrary(path);
+
+            if (ptr == IntPtr.Zero)
+                throw new IgniteException(string.Format("Failed to load {0}: {1}", 
+                    IgniteUtils.FileIgniteJniDll, Marshal.GetLastWin32Error()));
+        }
+
+        /// <summary>
+        /// No-op initializer used to force type loading and static constructor call.
+        /// </summary>
+        internal static void Initialize()
+        {
+            // No-op.
         }
 
         #region NATIVE METHODS: PROCESSOR
@@ -586,7 +69,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
                 try
                 {
-                    void* res = IGNITION_START(ctx.NativeContext, cfgPath0, gridName0, InteropFactoryId,
+                    void* res = JNI.IgnitionStart(ctx.NativeContext, cfgPath0, gridName0, InteropFactoryId,
                         mem.SynchronizeOutput());
 
                     return new UnmanagedTarget(ctx, res);
@@ -605,7 +88,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
             try
             {
-                return IGNITION_STOP(ctx, gridName0, cancel);
+                return JNI.IgnitionStop(ctx, gridName0, cancel);
             }
             finally
             {
@@ -615,17 +98,17 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static void IgnitionStopAll(void* ctx, bool cancel)
         {
-            IGNITION_STOP_ALL(ctx, cancel);
+            JNI.IgnitionStopAll(ctx, cancel);
         }
         
         internal static void ProcessorReleaseStart(IUnmanagedTarget target)
         {
-            PROCESSOR_RELEASE_START(target.Context, target.Target);
+            JNI.ProcessorReleaseStart(target.Context, target.Target);
         }
 
         internal static IUnmanagedTarget ProcessorProjection(IUnmanagedTarget target)
         {
-            void* res = PROCESSOR_PROJECTION(target.Context, target.Target);
+            void* res = JNI.ProcessorProjection(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
@@ -636,7 +119,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
             try
             {
-                void* res = PROCESSOR_CACHE(target.Context, target.Target, name0);
+                void* res = JNI.ProcessorCache(target.Context, target.Target, name0);
 
                 return target.ChangeTarget(res);
             }
@@ -652,7 +135,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
             try
             {
-                void* res = PROCESSOR_CREATE_CACHE(target.Context, target.Target, name0);
+                void* res = JNI.ProcessorCreateCache(target.Context, target.Target, name0);
 
                 return target.ChangeTarget(res);
             }
@@ -668,7 +151,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
             try
             {
-                void* res = PROCESSOR_GET_OR_CREATE_CACHE(target.Context, target.Target, name0);
+                void* res = JNI.ProcessorGetOrCreateCache(target.Context, target.Target, name0);
 
                 return target.ChangeTarget(res);
             }
@@ -684,7 +167,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
             try
             {
-                void* res = PROCESSOR_AFFINITY(target.Context, target.Target, name0);
+                void* res = JNI.ProcessorAffinity(target.Context, target.Target, name0);
 
                 return target.ChangeTarget(res);
             }
@@ -700,7 +183,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
             try
             {
-                void* res = PROCESSOR_DATA_STREAMER(target.Context, target.Target, name0, keepBinary);
+                void* res = JNI.ProcessorDataStreamer(target.Context, target.Target, name0, keepBinary);
 
                 return target.ChangeTarget(res);
             }
@@ -712,42 +195,42 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         
         internal static IUnmanagedTarget ProcessorTransactions(IUnmanagedTarget target)
         {
-            void* res = PROCESSOR_TRANSACTIONS(target.Context, target.Target);
+            void* res = JNI.ProcessorTransactions(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget ProcessorCompute(IUnmanagedTarget target, IUnmanagedTarget prj)
         {
-            void* res = PROCESSOR_COMPUTE(target.Context, target.Target, prj.Target);
+            void* res = JNI.ProcessorCompute(target.Context, target.Target, prj.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget ProcessorMessage(IUnmanagedTarget target, IUnmanagedTarget prj)
         {
-            void* res = PROCESSOR_MESSAGE(target.Context, target.Target, prj.Target);
+            void* res = JNI.ProcessorMessage(target.Context, target.Target, prj.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget ProcessorEvents(IUnmanagedTarget target, IUnmanagedTarget prj)
         {
-            void* res = PROCESSOR_EVENTS(target.Context, target.Target, prj.Target);
+            void* res = JNI.ProcessorEvents(target.Context, target.Target, prj.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget ProcessorServices(IUnmanagedTarget target, IUnmanagedTarget prj)
         {
-            void* res = PROCESSOR_SERVICES(target.Context, target.Target, prj.Target);
+            void* res = JNI.ProcessorServices(target.Context, target.Target, prj.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget ProcessorExtensions(IUnmanagedTarget target)
         {
-            void* res = PROCESSOR_EXTENSIONS(target.Context, target.Target);
+            void* res = JNI.ProcessorExtensions(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
@@ -759,7 +242,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
             try
             {
-                var res = PROCESSOR_ATOMIC_LONG(target.Context, target.Target, name0, initialValue, create);
+                var res = JNI.ProcessorAtomicLong(target.Context, target.Target, name0, initialValue, create);
 
                 return res == null ? null : target.ChangeTarget(res);
             }
@@ -775,51 +258,51 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static long TargetInStreamOutLong(IUnmanagedTarget target, int opType, long memPtr)
         {
-            return TARGET_IN_STREAM_OUT_LONG(target.Context, target.Target, opType, memPtr);
+            return JNI.TargetInStreamOutLong(target.Context, target.Target, opType, memPtr);
         }
 
         internal static void TargetInStreamOutStream(IUnmanagedTarget target, int opType, long inMemPtr, long outMemPtr)
         {
-            TARGET_IN_STREAM_OUT_STREAM(target.Context, target.Target, opType, inMemPtr, outMemPtr);
+            JNI.TargetInStreamOutStream(target.Context, target.Target, opType, inMemPtr, outMemPtr);
         }
 
         internal static IUnmanagedTarget TargetInStreamOutObject(IUnmanagedTarget target, int opType, long inMemPtr)
         {
-            void* res = TARGET_IN_STREAM_OUT_OBJECT(target.Context, target.Target, opType, inMemPtr);
+            void* res = JNI.TargetInStreanOutObject(target.Context, target.Target, opType, inMemPtr);
 
             return target.ChangeTarget(res);
         }
 
         internal static void TargetInObjectStreamOutStream(IUnmanagedTarget target, int opType, void* arg, long inMemPtr, long outMemPtr)
         {
-            TARGET_IN_OBJECT_STREAM_OUT_STREAM(target.Context, target.Target, opType, arg, inMemPtr, outMemPtr);
+            JNI.TargetInObjectStreamOutStream(target.Context, target.Target, opType, arg, inMemPtr, outMemPtr);
         }
 
         internal static long TargetOutLong(IUnmanagedTarget target, int opType)
         {
-            return TARGET_OUT_LONG(target.Context, target.Target, opType);
+            return JNI.TargetOutLong(target.Context, target.Target, opType);
         }
 
         internal static void TargetOutStream(IUnmanagedTarget target, int opType, long memPtr)
         {
-            TARGET_OUT_STREAM(target.Context, target.Target, opType, memPtr);
+            JNI.TargetOutStream(target.Context, target.Target, opType, memPtr);
         }
 
         internal static IUnmanagedTarget TargetOutObject(IUnmanagedTarget target, int opType)
         {
-            void* res = TARGET_OUT_OBJECT(target.Context, target.Target, opType);
+            void* res = JNI.TargetOutObject(target.Context, target.Target, opType);
 
             return target.ChangeTarget(res);
         }
 
         internal static void TargetListenFuture(IUnmanagedTarget target, long futId, int typ)
         {
-            TargetListenFut(target.Context, target.Target, futId, typ);
+            JNI.TargetListenFut(target.Context, target.Target, futId, typ);
         }
 
         internal static void TargetListenFutureForOperation(IUnmanagedTarget target, long futId, int typ, int opId)
         {
-            TargetListenFutForOp(target.Context, target.Target, futId, typ, opId);
+            JNI.TargetListenFutForOp(target.Context, target.Target, futId, typ, opId);
         }
 
         #endregion
@@ -828,7 +311,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static int AffinityPartitions(IUnmanagedTarget target)
         {
-            return AffinityParts(target.Context, target.Target);
+            return JNI.AffinityParts(target.Context, target.Target);
         }
 
         #endregion
@@ -837,110 +320,110 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static IUnmanagedTarget CacheWithSkipStore(IUnmanagedTarget target)
         {
-            void* res = CACHE_WITH_SKIP_STORE(target.Context, target.Target);
+            void* res = JNI.CacheWithSkipStore(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget CacheWithNoRetries(IUnmanagedTarget target)
         {
-            void* res = CACHE_WITH_NO_RETRIES(target.Context, target.Target);
+            void* res = JNI.CacheWithNoRetries(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget CacheWithExpiryPolicy(IUnmanagedTarget target, long create, long update, long access)
         {
-            void* res = CACHE_WITH_EXPIRY_POLICY(target.Context, target.Target, create, update, access);
+            void* res = JNI.CacheWithExpiryPolicy(target.Context, target.Target, create, update, access);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget CacheWithAsync(IUnmanagedTarget target)
         {
-            void* res = CACHE_WITH_ASYNC(target.Context, target.Target);
+            void* res = JNI.CacheWithAsync(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget CacheWithKeepBinary(IUnmanagedTarget target)
         {
-            void* res = CACHE_WITH_KEEP_BINARY(target.Context, target.Target);
+            void* res = JNI.CacheWithKeepBinary(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static void CacheClear(IUnmanagedTarget target)
         {
-            CACHE_CLEAR(target.Context, target.Target);
+            JNI.CacheClear(target.Context, target.Target);
         }
 
         internal static void CacheRemoveAll(IUnmanagedTarget target)
         {
-            CACHE_REMOVE_ALL(target.Context, target.Target);
+            JNI.CacheRemoveAll(target.Context, target.Target);
         }
 
         internal static IUnmanagedTarget CacheOutOpQueryCursor(IUnmanagedTarget target, int type, long memPtr)
         {
-            void* res = CACHE_OUT_OP_QUERY_CURSOR(target.Context, target.Target, type, memPtr);
+            void* res = JNI.CacheOutOpQueryCursor(target.Context, target.Target, type, memPtr);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget CacheOutOpContinuousQuery(IUnmanagedTarget target, int type, long memPtr)
         {
-            void* res = CACHE_OUT_OP_CONTINUOUS_QUERY(target.Context, target.Target, type, memPtr);
+            void* res = JNI.CacheOutOpContinuousQuery(target.Context, target.Target, type, memPtr);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget CacheIterator(IUnmanagedTarget target)
         {
-            void* res = CACHE_ITERATOR(target.Context, target.Target);
+            void* res = JNI.CacheIterator(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget CacheLocalIterator(IUnmanagedTarget target, int peekModes)
         {
-            void* res = CACHE_LOCAL_ITERATOR(target.Context, target.Target, peekModes);
+            void* res = JNI.CacheLocalIterator(target.Context, target.Target, peekModes);
 
             return target.ChangeTarget(res);
         }
 
         internal static void CacheEnterLock(IUnmanagedTarget target, long id)
         {
-            CACHE_ENTER_LOCK(target.Context, target.Target, id);
+            JNI.CacheEnterLock(target.Context, target.Target, id);
         }
 
         internal static void CacheExitLock(IUnmanagedTarget target, long id)
         {
-            CACHE_EXIT_LOCK(target.Context, target.Target, id);
+            JNI.CacheExitLock(target.Context, target.Target, id);
         }
 
         internal static bool CacheTryEnterLock(IUnmanagedTarget target, long id, long timeout)
         {
-            return CACHE_TRY_ENTER_LOCK(target.Context, target.Target, id, timeout);
+            return JNI.CacheTryEnterLock(target.Context, target.Target, id, timeout);
         }
 
         internal static void CacheCloseLock(IUnmanagedTarget target, long id)
         {
-            CACHE_CLOSE_LOCK(target.Context, target.Target, id);
+            JNI.CacheCloseLock(target.Context, target.Target, id);
         }
 
         internal static void CacheRebalance(IUnmanagedTarget target, long futId)
         {
-            CACHE_REBALANCE(target.Context, target.Target, futId);
+            JNI.CacheRebalance(target.Context, target.Target, futId);
         }
 
         internal static void CacheStoreCallbackInvoke(IUnmanagedTarget target, long memPtr)
         {
-            CACHE_STORE_CALLBACK_INVOKE(target.Context, target.Target, memPtr);
+            JNI.CacheStoreCallbackInvoke(target.Context, target.Target, memPtr);
         }
 
         internal static int CacheSize(IUnmanagedTarget target, int modes, bool loc)
         {
-            return CACHE_SIZE(target.Context, target.Target, modes, loc);
+            return JNI.CacheSize(target.Context, target.Target, modes, loc);
         }
 
         #endregion
@@ -949,17 +432,17 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static void ComputeWithNoFailover(IUnmanagedTarget target)
         {
-            COMPUTE_WITH_NO_FAILOVER(target.Context, target.Target);
+            JNI.ComputeWithNoFailover(target.Context, target.Target);
         }
 
         internal static void ComputeWithTimeout(IUnmanagedTarget target, long timeout)
         {
-            COMPUTE_WITH_TIMEOUT(target.Context, target.Target, timeout);
+            JNI.ComputeWithTimeout(target.Context, target.Target, timeout);
         }
 
         internal static void ComputeExecuteNative(IUnmanagedTarget target, long taskPtr, long topVer)
         {
-            COMPUTE_EXECUTE_NATIVE(target.Context, target.Target, taskPtr, topVer);
+            JNI.ComputeExecuteNative(target.Context, target.Target, taskPtr, topVer);
         }
 
         #endregion
@@ -968,12 +451,12 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static void ContinuousQueryClose(IUnmanagedTarget target)
         {
-            ContinuousQryClose(target.Context, target.Target);
+            JNI.ContinuousQryClose(target.Context, target.Target);
         }
 
         internal static IUnmanagedTarget ContinuousQueryGetInitialQueryCursor(IUnmanagedTarget target)
         {
-            void* res = ContinuousQryGetInitialQueryCursor(target.Context, target.Target);
+            void* res = JNI.ContinuousQryGetInitialQueryCursor(target.Context, target.Target);
 
             return res == null ? null : target.ChangeTarget(res);
         }
@@ -984,47 +467,47 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static void DataStreamerListenTopology(IUnmanagedTarget target, long ptr)
         {
-            DataStreamerListenTop(target.Context, target.Target, ptr);
+            JNI.DataStreamerListenTop(target.Context, target.Target, ptr);
         }
 
         internal static bool DataStreamerAllowOverwriteGet(IUnmanagedTarget target)
         {
-            return DATA_STREAMER_ALLOW_OVERWRITE_GET(target.Context, target.Target);
+            return JNI.DataStreamerAllowOverwriteGet(target.Context, target.Target);
         }
 
         internal static void DataStreamerAllowOverwriteSet(IUnmanagedTarget target, bool val)
         {
-            DATA_STREAMER_ALLOW_OVERWRITE_SET(target.Context, target.Target, val);
+            JNI.DataStreamerAllowOverwriteSet(target.Context, target.Target, val);
         }
 
         internal static bool DataStreamerSkipStoreGet(IUnmanagedTarget target)
         {
-            return DATA_STREAMER_SKIP_STORE_GET(target.Context, target.Target);
+            return JNI.DataStreamerSkipStoreGet(target.Context, target.Target);
         }
 
         internal static void DataStreamerSkipStoreSet(IUnmanagedTarget target, bool val)
         {
-            DATA_STREAMER_SKIP_STORE_SET(target.Context, target.Target, val);
+            JNI.DataStreamerSkipStoreSet(target.Context, target.Target, val);
         }
 
         internal static int DataStreamerPerNodeBufferSizeGet(IUnmanagedTarget target)
         {
-            return DATA_STREAMER_PER_NODE_BUFFER_SIZE_GET(target.Context, target.Target);
+            return JNI.DataStreamerPerNodeBufferSizeGet(target.Context, target.Target);
         }
 
         internal static void DataStreamerPerNodeBufferSizeSet(IUnmanagedTarget target, int val)
         {
-            DATA_STREAMER_PER_NODE_BUFFER_SIZE_SET(target.Context, target.Target, val);
+            JNI.DataStreamerPerNodeBufferSizeSet(target.Context, target.Target, val);
         }
 
         internal static int DataStreamerPerNodeParallelOperationsGet(IUnmanagedTarget target)
         {
-            return DataStreamerPerNodeParallelOpsGet(target.Context, target.Target);
+            return JNI.DataStreamerPerNodeParallelOpsGet(target.Context, target.Target);
         }
 
         internal static void DataStreamerPerNodeParallelOperationsSet(IUnmanagedTarget target, int val)
         {
-            DataStreamerPerNodeParallelOpsSet(target.Context, target.Target, val);
+            JNI.DataStreamerPerNodeParallelOpsSet(target.Context, target.Target, val);
         }
 
         #endregion
@@ -1033,7 +516,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static IUnmanagedTarget MessagingWithASync(IUnmanagedTarget target)
         {
-            void* res = MessagingWithAsync(target.Context, target.Target);
+            void* res = JNI.MessagingWithAsync(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
@@ -1044,54 +527,54 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static IUnmanagedTarget ProjectionForOthers(IUnmanagedTarget target, IUnmanagedTarget prj)
         {
-            void* res = PROJECTION_FOR_OTHERS(target.Context, target.Target, prj.Target);
+            void* res = JNI.ProjectionForOthers(target.Context, target.Target, prj.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget ProjectionForRemotes(IUnmanagedTarget target)
         {
-            void* res = PROJECTION_FOR_REMOTES(target.Context, target.Target);
+            void* res = JNI.ProjectionForRemotes(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget ProjectionForDaemons(IUnmanagedTarget target)
         {
-            void* res = PROJECTION_FOR_DAEMONS(target.Context, target.Target);
+            void* res = JNI.ProjectionForDaemons(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget ProjectionForRandom(IUnmanagedTarget target)
         {
-            void* res = PROJECTION_FOR_RANDOM(target.Context, target.Target);
+            void* res = JNI.ProjectionForRandom(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget ProjectionForOldest(IUnmanagedTarget target)
         {
-            void* res = PROJECTION_FOR_OLDEST(target.Context, target.Target);
+            void* res = JNI.ProjectionForOldest(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
 
         internal static IUnmanagedTarget ProjectionForYoungest(IUnmanagedTarget target)
         {
-            void* res = PROJECTION_FOR_YOUNGEST(target.Context, target.Target);
+            void* res = JNI.ProjectionForYoungest(target.Context, target.Target);
 
             return target.ChangeTarget(res);
         }
         
         internal static void ProjectionResetMetrics(IUnmanagedTarget target)
         {
-            PROJECTION_RESET_METRICS(target.Context, target.Target);
+            JNI.ProjectionResetMetrics(target.Context, target.Target);
         }
 
         internal static IUnmanagedTarget ProjectionOutOpRet(IUnmanagedTarget target, int type, long memPtr)
         {
-            void* res = PROJECTION_OUT_OP_RET(target.Context, target.Target, type, memPtr);
+            void* res = JNI.ProjectionOutOpRet(target.Context, target.Target, type, memPtr);
 
             return target.ChangeTarget(res);
         }
@@ -1102,12 +585,12 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static void QueryCursorIterator(IUnmanagedTarget target)
         {
-            QryCursorIterator(target.Context, target.Target);
+            JNI.QryCursorIterator(target.Context, target.Target);
         }
 
         internal static void QueryCursorClose(IUnmanagedTarget target)
         {
-            QryCursorClose(target.Context, target.Target);
+            JNI.QryCursorClose(target.Context, target.Target);
         }
 
         #endregion
@@ -1116,47 +599,47 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static long TransactionsStart(IUnmanagedTarget target, int concurrency, int isolation, long timeout, int txSize)
         {
-            return TxStart(target.Context, target.Target, concurrency, isolation, timeout, txSize);
+            return JNI.TxStart(target.Context, target.Target, concurrency, isolation, timeout, txSize);
         }
 
         internal static int TransactionsCommit(IUnmanagedTarget target, long id)
         {
-            return TxCommit(target.Context, target.Target, id);
+            return JNI.TxCommit(target.Context, target.Target, id);
         }
 
         internal static void TransactionsCommitAsync(IUnmanagedTarget target, long id, long futId)
         {
-            TxCommitAsync(target.Context, target.Target, id, futId);
+            JNI.TxCommitAsync(target.Context, target.Target, id, futId);
         }
 
         internal static int TransactionsRollback(IUnmanagedTarget target, long id)
         {
-            return TxRollback(target.Context, target.Target, id);
+            return JNI.TxRollback(target.Context, target.Target, id);
         }
 
         internal static void TransactionsRollbackAsync(IUnmanagedTarget target, long id, long futId)
         {
-            TxRollbackAsync(target.Context, target.Target, id, futId);
+            JNI.TxRollbackAsync(target.Context, target.Target, id, futId);
         }
 
         internal static int TransactionsClose(IUnmanagedTarget target, long id)
         {
-            return TxClose(target.Context, target.Target, id);
+            return JNI.TxClose(target.Context, target.Target, id);
         }
 
         internal static int TransactionsState(IUnmanagedTarget target, long id)
         {
-            return TxState(target.Context, target.Target, id);
+            return JNI.TxState(target.Context, target.Target, id);
         }
 
         internal static bool TransactionsSetRollbackOnly(IUnmanagedTarget target, long id)
         {
-            return TxSetRollbackOnly(target.Context, target.Target, id);
+            return JNI.TxSetRollbackOnly(target.Context, target.Target, id);
         }
 
         internal static void TransactionsResetMetrics(IUnmanagedTarget target)
         {
-            TxResetMetrics(target.Context, target.Target);
+            JNI.TxResetMetrics(target.Context, target.Target);
         }
 
         #endregion
@@ -1165,7 +648,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static void Reallocate(long memPtr, int cap)
         {
-            int res = REALLOCATE(memPtr, cap);
+            int res = JNI.Reallocate(memPtr, cap);
 
             if (res != 0)
                 throw new IgniteException("Failed to reallocate external memory [ptr=" + memPtr + 
@@ -1174,14 +657,14 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static IUnmanagedTarget Acquire(UnmanagedContext ctx, void* target)
         {
-            void* target0 = ACQUIRE(ctx.NativeContext, target);
+            void* target0 = JNI.Acquire(ctx.NativeContext, target);
 
             return new UnmanagedTarget(ctx, target0);
         }
 
         internal static void Release(IUnmanagedTarget target)
         {
-            RELEASE(target.Target);
+            JNI.Release(target.Target);
         }
 
         internal static void ThrowToJava(void* ctx, Exception e)
@@ -1190,7 +673,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
             try
             {
-                THROW_TO_JAVA(ctx, msgChars);
+                JNI.ThrowToJava(ctx, msgChars);
             }
             finally
             {
@@ -1198,26 +681,24 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             }
         }
 
-        
-
         internal static int HandlersSize()
         {
-            return HANDLERS_SIZE();
+            return JNI.HandlersSize();
         }
 
         internal static void* CreateContext(void* opts, int optsLen, void* cbs)
         {
-            return CREATE_CONTEXT(opts, optsLen, cbs);
+            return JNI.CreateContext(opts, optsLen, cbs);
         }
 
         internal static void DeleteContext(void* ctx)
         {
-            DELETE_CONTEXT(ctx);
+            JNI.DeleteContext(ctx);
         }
 
         internal static void DestroyJvm(void* ctx)
         {
-            DESTROY_JVM(ctx);
+            JNI.DestroyJvm(ctx);
         }
 
         #endregion
@@ -1226,22 +707,22 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static IUnmanagedTarget EventsWithAsync(IUnmanagedTarget target)
         {
-            return target.ChangeTarget(EVENTS_WITH_ASYNC(target.Context, target.Target));
+            return target.ChangeTarget(JNI.EventsWithAsync(target.Context, target.Target));
         }
 
         internal static bool EventsStopLocalListen(IUnmanagedTarget target, long handle)
         {
-            return EVENTS_STOP_LOCAL_LISTEN(target.Context, target.Target, handle);
+            return JNI.EventsStopLocalListen(target.Context, target.Target, handle);
         }
 
         internal static bool EventsIsEnabled(IUnmanagedTarget target, int type)
         {
-            return EVENTS_IS_ENABLED(target.Context, target.Target, type);
+            return JNI.EventsIsEnabled(target.Context, target.Target, type);
         }
 
         internal static void EventsLocalListen(IUnmanagedTarget target, long handle, int type)
         {
-            EVENTS_LOCAL_LISTEN(target.Context, target.Target, handle, type);
+            JNI.EventsLocalListen(target.Context, target.Target, handle, type);
         }
 
         #endregion
@@ -1250,12 +731,12 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static IUnmanagedTarget ServicesWithAsync(IUnmanagedTarget target)
         {
-            return target.ChangeTarget(SERVICES_WITH_ASYNC(target.Context, target.Target));
+            return target.ChangeTarget(JNI.ServicesWithAsync(target.Context, target.Target));
         }
 
         internal static IUnmanagedTarget ServicesWithServerKeepBinary(IUnmanagedTarget target)
         {
-            return target.ChangeTarget(SERVICES_WITH_SERVER_KEEP_BINARY(target.Context, target.Target));
+            return target.ChangeTarget(JNI.ServicesWithServerKeepBinary(target.Context, target.Target));
         }
 
         internal static void ServicesCancel(IUnmanagedTarget target, string name)
@@ -1264,7 +745,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
             try
             {
-                SERVICES_CANCEL(target.Context, target.Target, nameChars);
+                JNI.ServicesCancel(target.Context, target.Target, nameChars);
             }
             finally
             {
@@ -1274,7 +755,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static void ServicesCancelAll(IUnmanagedTarget target)
         {
-            SERVICES_CANCEL_ALL(target.Context, target.Target);
+            JNI.ServicesCancelAll(target.Context, target.Target);
         }
 
         internal static IUnmanagedTarget ServicesGetServiceProxy(IUnmanagedTarget target, string name, bool sticky)
@@ -1283,8 +764,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
             try
             {
-                return target.ChangeTarget(
-                    SERVICES_GET_SERVICE_PROXY(target.Context, target.Target, nameChars, sticky));
+                return target.ChangeTarget(JNI.ServicesGetServiceProxy(target.Context, target.Target, nameChars, sticky));
             }
             finally
             {
@@ -1298,74 +778,44 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal static long AtomicLongGet(IUnmanagedTarget target)
         {
-            return ATOMIC_LONG_GET(target.Context, target.Target);
+            return JNI.AtomicLongGet(target.Context, target.Target);
         }
 
         internal static long AtomicLongIncrementAndGet(IUnmanagedTarget target)
         {
-            return ATOMIC_LONG_INCREMENT_AND_GET(target.Context, target.Target);
+            return JNI.AtomicLongIncrementAndGet(target.Context, target.Target);
         }
 
         internal static long AtomicLongAddAndGet(IUnmanagedTarget target, long value)
         {
-            return ATOMIC_LONG_ADD_AND_GET(target.Context, target.Target, value);
+            return JNI.AtomicLongAddAndGet(target.Context, target.Target, value);
         }
 
         internal static long AtomicLongDecrementAndGet(IUnmanagedTarget target)
         {
-            return ATOMIC_LONG_DECREMENT_AND_GET(target.Context, target.Target);
+            return JNI.AtomicLongDecrementAndGet(target.Context, target.Target);
         }
 
         internal static long AtomicLongGetAndSet(IUnmanagedTarget target, long value)
         {
-            return ATOMIC_LONG_GET_AND_SET(target.Context, target.Target, value);
+            return JNI.AtomicLongGetAndSet(target.Context, target.Target, value);
         }
 
         internal static long AtomicLongCompareAndSetAndGet(IUnmanagedTarget target, long expVal, long newVal)
         {
-            return ATOMIC_LONG_COMPARE_AND_SET_AND_GET(target.Context, target.Target, expVal, newVal);
+            return JNI.AtomicLongCompareAndSetAndGet(target.Context, target.Target, expVal, newVal);
         }
 
         internal static bool AtomicLongIsClosed(IUnmanagedTarget target)
         {
-            return ATOMIC_LONG_IS_CLOSED(target.Context, target.Target);
+            return JNI.AtomicLongIsClosed(target.Context, target.Target);
         }
 
         internal static void AtomicLongClose(IUnmanagedTarget target)
         {
-            ATOMIC_LONG_CLOSE(target.Context, target.Target);
+            JNI.AtomicLongClose(target.Context, target.Target);
         }
 
         #endregion
-
-        /// <summary>
-        /// No-op initializer used to force type loading and static constructor call.
-        /// </summary>
-        internal static void Initialize()
-        {
-            // No-op.
-        }
-
-        /// <summary>
-        /// Create delegate for the given procedure.
-        /// </summary>
-        /// <typeparam name="T">Delegate type.</typeparam>
-        /// <param name="procName">Procedure name.</param>
-        /// <returns></returns>
-        private static T CreateDelegate<T>(string procName)
-        {
-            var procPtr = NativeMethods.GetProcAddress(Ptr, procName);
-
-            if (procPtr == IntPtr.Zero)
-            {
-                var error = Marshal.GetLastWin32Error();
-
-                throw new IgniteException(string.Format(CultureInfo.InvariantCulture,
-                    "Unable to find native function: {0} (Error code: {1}). Make sure that module.def is up to date",
-                    procName, error));
-            }
-
-            return TypeCaster<T>.Cast(Marshal.GetDelegateForFunctionPointer(procPtr, typeof (T)));
-        }
     }
 }


[19/50] [abbrv] ignite git commit: Javadoc hotfix

Posted by sb...@apache.org.
Javadoc hotfix


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

Branch: refs/heads/ignite-1.5.1
Commit: 6a109ebd4c637b3ec6e298ae8cf785c422abebc1
Parents: d5fa3c1
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Dec 4 13:42:25 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Dec 4 13:42:25 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/6a109ebd/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 4fd7156..68ba62b 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -428,6 +428,10 @@
                                 <title>Spark Integration</title>
                                 <packages>org.apache.ignite.spark.examples.java</packages>
                             </group>
+                            <group>
+                                <title>OSGi integration</title>
+                                <packages>org.apache.ignite.osgi*</packages>
+                            </group>
                         </groups>
                         <header>
                             <![CDATA[


[49/50] [abbrv] ignite git commit: ignite-10 Fix ignite instance [de]serialization

Posted by sb...@apache.org.
ignite-10 Fix ignite instance [de]serialization


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

Branch: refs/heads/ignite-1.5.1
Commit: 0adee3ae09a86334375b521871539b086215ba92
Parents: 6904724
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 9 09:56:53 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 9 09:56:53 2015 +0300

----------------------------------------------------------------------
 .../messaging/MessagingPingPongExample.java     |  4 --
 .../examples/MessagingExamplesSelfTest.java     |  2 -
 .../main/java/org/apache/ignite/Ignition.java   | 16 +++++++
 .../store/jdbc/CacheAbstractJdbcStore.java      |  5 +--
 .../configuration/CacheConfiguration.java       |  1 -
 .../configuration/IgniteConfiguration.java      |  2 -
 .../apache/ignite/internal/IgniteKernal.java    |  2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 39 +++++++++++++----
 .../ignite/internal/MarshallerContextImpl.java  | 23 +++++-----
 .../managers/communication/GridIoManager.java   |  3 +-
 .../internal/portable/BinaryFieldAccessor.java  |  9 ++--
 .../internal/portable/BinaryReaderExImpl.java   | 19 ++++----
 .../internal/portable/BinaryWriterExImpl.java   | 19 ++++----
 .../internal/portable/PortableContext.java      | 12 ++---
 .../CacheDefaultBinaryAffinityKeyMapper.java    |  2 +-
 .../processors/cache/GridCacheProcessor.java    |  2 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |  2 +-
 .../CacheObjectBinaryProcessorImpl.java         | 41 +++++++++--------
 .../cache/query/GridCacheQueryManager.java      | 46 ++++++++++----------
 .../util/nio/GridNioBackPressureControl.java    | 14 +-----
 .../org/apache/ignite/spi/IgniteSpiThread.java  | 10 ++---
 .../communication/tcp/TcpCommunicationSpi.java  |  6 +--
 .../org/apache/ignite/thread/IgniteThread.java  | 43 ++++++++++--------
 .../ignite/thread/IgniteThreadPoolExecutor.java | 37 ++--------------
 .../portable/BinaryMarshallerSelfTest.java      |  2 +-
 ...cheAbstractFullApiMultithreadedSelfTest.java |  2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  4 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |  2 -
 .../marshaller/GridMarshallerAbstractTest.java  | 32 +++++++++++++-
 .../testframework/junits/GridAbstractTest.java  |  9 ++--
 .../junits/IgniteTestResources.java             |  2 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |  4 +-
 .../IgnitePortableCacheTestSuite.java           |  3 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |  5 +++
 .../ignite/osgi/AbstractIgniteKarafTest.java    |  2 -
 .../ignite/internal/GridFactorySelfTest.java    | 29 ++++++++++++
 .../GridSpringBeanSerializationSelfTest.java    |  2 +
 37 files changed, 252 insertions(+), 205 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java b/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
index a2ba227..8571f66 100644
--- a/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
@@ -26,7 +26,6 @@ import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.examples.ExampleNodeStartup;
 import org.apache.ignite.examples.ExamplesUtils;
 import org.apache.ignite.lang.IgniteBiPredicate;
-import org.apache.ignite.resources.IgniteInstanceResource;
 
 /**
  * Demonstrates simple message exchange between local and remote nodes.
@@ -65,9 +64,6 @@ public class MessagingPingPongExample {
 
             // Set up remote player.
             ignite.message(nodeB).remoteListen(null, new IgniteBiPredicate<UUID, String>() {
-                @IgniteInstanceResource
-                private Ignite ignite;
-
                 @Override public boolean apply(UUID nodeId, String rcvMsg) {
                     System.out.println("Received message [msg=" + rcvMsg + ", sender=" + nodeId + ']');
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
index c77e4b6..e15159d 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
@@ -41,8 +41,6 @@ public class MessagingExamplesSelfTest extends GridAbstractExamplesTest {
      * @throws Exception If failed.
      */
     public void testMessagingPingPongExample() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-10");
-
         MessagingPingPongExample.main(EMPTY_ARGS);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/Ignition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignition.java b/modules/core/src/main/java/org/apache/ignite/Ignition.java
index 2c06881..4fdc849 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignition.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignition.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.DiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -516,6 +517,21 @@ public class Ignition {
     }
 
     /**
+     * This method is used to address a local {@link Ignite} instance, principally from closure.
+     * <p>
+     * According to contract this method has to be called only under {@link IgniteThread}.
+     * An {@link IllegalArgumentException} will be thrown otherwise.
+     *
+     * @return A current {@link Ignite} instance to address from closure.
+     * @throws IgniteIllegalStateException Thrown if grid was not properly
+     *      initialized or grid instance was stopped or was not started
+     * @throws IllegalArgumentException Thrown if current thread is not an {@link IgniteThread}.
+     */
+    public static Ignite localIgnite() throws IgniteIllegalStateException, IllegalArgumentException {
+        return IgnitionEx.localIgnite();
+    }
+
+    /**
      * Adds a lsnr for grid life cycle events.
      * <p>
      * Note that unlike other listeners in Ignite this listener will be

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/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 366262c..060ab89 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
@@ -62,6 +62,7 @@ import org.apache.ignite.cache.store.jdbc.dialect.MySQLDialect;
 import org.apache.ignite.cache.store.jdbc.dialect.OracleDialect;
 import org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.F;
@@ -69,7 +70,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lifecycle.LifecycleAware;
-import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.resources.CacheStoreSessionResource;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.resources.LoggerResource;
@@ -78,10 +78,9 @@ import org.jetbrains.annotations.Nullable;
 
 import static java.sql.Statement.EXECUTE_FAILED;
 import static java.sql.Statement.SUCCESS_NO_INFO;
-
 import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_BATCH_SIZE;
-import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_WRITE_ATTEMPTS;
 import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_PARALLEL_LOAD_CACHE_MINIMUM_THRESHOLD;
+import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_WRITE_ATTEMPTS;
 
 /**
  * Implementation of {@link CacheStore} backed by JDBC.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index af3530a..67f7b2e 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -45,7 +45,6 @@ import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheInterceptor;
-import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheRebalanceMode;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 07fb3a8..7d8f5be 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -46,8 +46,6 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.lifecycle.LifecycleEventType;
 import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.jdk.JdkMarshaller;
-import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.plugin.PluginConfiguration;
 import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.plugin.segmentation.SegmentationPolicy;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index ab62c13..d231ad3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -3227,7 +3227,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      */
     protected Object readResolve() throws ObjectStreamException {
         try {
-            return IgnitionEx.gridx(gridName);
+            return IgnitionEx.localIgnite();
         }
         catch (IllegalStateException e) {
             throw U.withCause(new InvalidObjectException(e.getMessage()), e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index be06f85..a73fdeb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -94,6 +94,7 @@ import org.apache.ignite.spi.indexing.noop.NoopIndexingSpi;
 import org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi;
 import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
 import org.apache.ignite.spi.swapspace.noop.NoopSwapSpaceSpi;
+import org.apache.ignite.thread.IgniteThread;
 import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
@@ -1194,6 +1195,20 @@ public class IgnitionEx {
     }
 
     /**
+     * Gets a name of the grid, which is owner of current thread. An Exception is thrown if
+     * current thread is not an {@link IgniteThread}.
+     *
+     * @return Grid instance related to current thread
+     * @throws IllegalArgumentException Thrown to indicate, that current thread is not an {@link IgniteThread}.
+     */
+    public static IgniteKernal localIgnite() throws IllegalArgumentException {
+        if (Thread.currentThread() instanceof IgniteThread)
+            return gridx(((IgniteThread)Thread.currentThread()).getGridName());
+        else
+            throw new IllegalArgumentException("This method should be accessed under " + IgniteThread.class.getName());
+    }
+
+    /**
      * Gets grid instance without waiting its initialization.
      *
      * @param name Grid name.
@@ -1528,7 +1543,8 @@ public class IgnitionEx {
             }
 
             execSvc = new IgniteThreadPoolExecutor(
-                "pub-" + cfg.getGridName(),
+                "pub",
+                cfg.getGridName(),
                 cfg.getPublicThreadPoolSize(),
                 cfg.getPublicThreadPoolSize(),
                 DFLT_PUBLIC_KEEP_ALIVE_TIME,
@@ -1541,7 +1557,8 @@ public class IgnitionEx {
             // Note that since we use 'LinkedBlockingQueue', number of
             // maximum threads has no effect.
             sysExecSvc = new IgniteThreadPoolExecutor(
-                "sys-" + cfg.getGridName(),
+                "sys",
+                cfg.getGridName(),
                 cfg.getSystemThreadPoolSize(),
                 cfg.getSystemThreadPoolSize(),
                 DFLT_SYSTEM_KEEP_ALIVE_TIME,
@@ -1555,7 +1572,8 @@ public class IgnitionEx {
             // Note, that we do not pre-start threads here as management pool may
             // not be needed.
             mgmtExecSvc = new IgniteThreadPoolExecutor(
-                "mgmt-" + cfg.getGridName(),
+                "mgmt",
+                cfg.getGridName(),
                 cfg.getManagementThreadPoolSize(),
                 cfg.getManagementThreadPoolSize(),
                 0,
@@ -1566,7 +1584,8 @@ public class IgnitionEx {
             // Note, that we do not pre-start threads here as class loading pool may
             // not be needed.
             p2pExecSvc = new IgniteThreadPoolExecutor(
-                "p2p-" + cfg.getGridName(),
+                "p2p",
+                cfg.getGridName(),
                 cfg.getPeerClassLoadingThreadPoolSize(),
                 cfg.getPeerClassLoadingThreadPoolSize(),
                 0,
@@ -1574,7 +1593,8 @@ public class IgnitionEx {
 
             // Note that we do not pre-start threads here as igfs pool may not be needed.
             igfsExecSvc = new IgniteThreadPoolExecutor(
-                "igfs-" + cfg.getGridName(),
+                "igfs",
+                cfg.getGridName(),
                 cfg.getIgfsThreadPoolSize(),
                 cfg.getIgfsThreadPoolSize(),
                 0,
@@ -1582,7 +1602,8 @@ public class IgnitionEx {
 
             if (myCfg.getConnectorConfiguration() != null) {
                 restExecSvc = new IgniteThreadPoolExecutor(
-                    "rest-" + myCfg.getGridName(),
+                    "rest",
+                    myCfg.getGridName(),
                     myCfg.getConnectorConfiguration().getThreadPoolSize(),
                     myCfg.getConnectorConfiguration().getThreadPoolSize(),
                     ConnectorConfiguration.DFLT_KEEP_ALIVE_TIME,
@@ -1591,14 +1612,16 @@ public class IgnitionEx {
             }
 
             utilityCacheExecSvc = new IgniteThreadPoolExecutor(
-                "utility-" + cfg.getGridName(),
+                "utility",
+                cfg.getGridName(),
                 myCfg.getUtilityCacheThreadPoolSize(),
                 DFLT_SYSTEM_MAX_THREAD_CNT,
                 myCfg.getUtilityCacheKeepAliveTime(),
                 new LinkedBlockingQueue<Runnable>(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP));
 
             marshCacheExecSvc = new IgniteThreadPoolExecutor(
-                "marshaller-cache-" + cfg.getGridName(),
+                "marshaller-cache",
+                cfg.getGridName(),
                 myCfg.getMarshallerCacheThreadPoolSize(),
                 DFLT_SYSTEM_MAX_THREAD_CNT,
                 myCfg.getMarshallerCacheKeepAliveTime(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/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 276cdc3..c7fa902 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
@@ -17,18 +17,6 @@
 
 package org.apache.ignite.internal;
 
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheTryPutFailedException;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.plugin.PluginProvider;
-
-import javax.cache.event.CacheEntryEvent;
-import javax.cache.event.CacheEntryListenerException;
-import javax.cache.event.CacheEntryUpdatedListener;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
@@ -37,6 +25,17 @@ import java.io.IOException;
 import java.io.Writer;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryListenerException;
+import javax.cache.event.CacheEntryUpdatedListener;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheTryPutFailedException;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.PluginProvider;
 
 /**
  * Marshaller context implementation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index a8557cc..bf7c7e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -242,7 +242,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         utilityCachePool = ctx.utilityCachePool();
         marshCachePool = ctx.marshallerCachePool();
         affPool = new IgniteThreadPoolExecutor(
-            "aff-" + ctx.gridName(),
+            "aff",
+            ctx.gridName(),
             1,
             1,
             0,

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldAccessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldAccessor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldAccessor.java
index eece245..2f6b73c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldAccessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldAccessor.java
@@ -17,11 +17,6 @@
 
 package org.apache.ignite.internal.portable;
 
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import sun.misc.Unsafe;
-
 import java.lang.reflect.Field;
 import java.math.BigDecimal;
 import java.sql.Timestamp;
@@ -29,6 +24,10 @@ import java.util.Collection;
 import java.util.Date;
 import java.util.Map;
 import java.util.UUID;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import sun.misc.Unsafe;
 
 /**
  * Field accessor to speedup access.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
index 91b67f6..a0aa2e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
@@ -17,6 +17,15 @@
 
 package org.apache.ignite.internal.portable;
 
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
 import org.apache.ignite.binary.BinaryIdMapper;
 import org.apache.ignite.binary.BinaryInvalidTypeException;
 import org.apache.ignite.binary.BinaryObject;
@@ -28,16 +37,6 @@ import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-import java.util.UUID;
-
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN_ARR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
index 59632f1..6bb493e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
@@ -17,16 +17,6 @@
 
 package org.apache.ignite.internal.portable;
 
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryIdMapper;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryRawWriter;
-import org.apache.ignite.binary.BinaryWriter;
-import org.apache.ignite.internal.portable.streams.PortableHeapOutputStream;
-import org.apache.ignite.internal.portable.streams.PortableOutputStream;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.jetbrains.annotations.Nullable;
-
 import java.io.IOException;
 import java.io.ObjectOutput;
 import java.lang.reflect.InvocationTargetException;
@@ -37,6 +27,15 @@ import java.util.Collection;
 import java.util.Date;
 import java.util.Map;
 import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.internal.portable.streams.PortableHeapOutputStream;
+import org.apache.ignite.internal.portable.streams.PortableOutputStream;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.jetbrains.annotations.Nullable;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index fd6c41d..8bad737 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -53,16 +53,16 @@ import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinarySerializer;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.cache.affinity.AffinityKeyMapped;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryIdMapper;
-import org.apache.ignite.binary.BinaryInvalidTypeException;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinarySerializer;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDefaultBinaryAffinityKeyMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDefaultBinaryAffinityKeyMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDefaultBinaryAffinityKeyMapper.java
index d244a16..828e2ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDefaultBinaryAffinityKeyMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDefaultBinaryAffinityKeyMapper.java
@@ -19,10 +19,10 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.binary.BinaryObject;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 02e6403..3bd9466 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -70,6 +70,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.IgniteTransactionsEx;
 import org.apache.ignite.internal.managers.discovery.CustomEventListener;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager;
@@ -113,7 +114,6 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
-import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.jetbrains.annotations.Nullable;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
index 42358d9..e731406 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
@@ -116,7 +116,7 @@ public class GridDhtAffinityAssignmentResponse extends GridCacheMessage {
         if (affAssignmentBytes != null) {
             affAssignment = ctx.marshaller().unmarshal(affAssignmentBytes, ldr);
 
-            // TODO IGNITE-10: setting 'local' for nodes not needed when IGNITE-10 is implemented.
+            // TODO IGNITE-2110: setting 'local' for nodes not needed when IGNITE-2110 is implemented.
             int assignments = affAssignment.size();
 
             for (int n = 0; n < assignments; n++) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index d172bca..cd65681 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -17,6 +17,25 @@
 
 package org.apache.ignite.internal.processors.cache.portable;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryListenerException;
+import javax.cache.event.CacheEntryUpdatedListener;
+import javax.cache.event.EventType;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteBinary;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -31,6 +50,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.portable.BinaryEnumObjectImpl;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.portable.BinaryMetadata;
 import org.apache.ignite.internal.portable.BinaryMetadataHandler;
 import org.apache.ignite.internal.portable.BinaryObjectEx;
@@ -73,31 +93,10 @@ import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import sun.misc.Unsafe;
 
-import javax.cache.Cache;
-import javax.cache.CacheException;
-import javax.cache.event.CacheEntryEvent;
-import javax.cache.event.CacheEntryListenerException;
-import javax.cache.event.CacheEntryUpdatedListener;
-import javax.cache.event.EventType;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.MutableEntry;
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
-
 /**
  * Portable processor implementation.
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index bb5d230..4681e47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -17,6 +17,28 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.sql.SQLException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentMap;
+import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -91,30 +113,6 @@ import org.apache.ignite.spi.indexing.IndexingSpi;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
-import javax.cache.Cache;
-import javax.cache.expiry.ExpiryPolicy;
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.sql.SQLException;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Queue;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentMap;
-
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java
index b15e4a6..96a1ab3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.util.nio;
 
-import org.apache.ignite.thread.IgniteThread;
-
 /**
  * Utility class that allows to ignore back-pressure control for threads that are processing messages.
  */
@@ -34,11 +32,6 @@ public class GridNioBackPressureControl {
      * @return Flag indicating whether current thread is processing message.
      */
     public static boolean threadProcessingMessage() {
-        Thread th = Thread.currentThread();
-
-        if (th instanceof IgniteThread)
-            return ((IgniteThread)th).processingMessage();
-
         return threadProcMsg.get();
     }
 
@@ -46,11 +39,6 @@ public class GridNioBackPressureControl {
      * @param processing Flag indicating whether current thread is processing message.
      */
     public static void threadProcessingMessage(boolean processing) {
-        Thread th = Thread.currentThread();
-
-        if (th instanceof IgniteThread)
-            ((IgniteThread)th).processingMessage(processing);
-        else
-            threadProcMsg.set(processing);
+        threadProcMsg.set(processing);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiThread.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiThread.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiThread.java
index 25b81e7..1667891 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiThread.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiThread.java
@@ -21,16 +21,16 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThread;
 
 /**
  * This class provides convenient adapter for threads used by SPIs.
- * This class adds necessary plumbing on top of the {@link Thread} class:
+ * This class adds necessary plumbing on top of the {@link IgniteThread} class:
  * <ul>
- * <li>Consistent naming of threads</li>
- * <li>Dedicated parent thread group</li>
+ *      <li>Proper exception handling in {@link #body()}</li>
  * </ul>
  */
-public abstract class IgniteSpiThread extends Thread {
+public abstract class IgniteSpiThread extends IgniteThread {
     /** Default thread's group. */
     public static final ThreadGroup DFLT_GRP = new ThreadGroup("ignite-spi");
 
@@ -48,7 +48,7 @@ public abstract class IgniteSpiThread extends Thread {
      * @param log Grid logger to use.
      */
     protected IgniteSpiThread(String gridName, String name, IgniteLogger log) {
-        super(DFLT_GRP, name + "-#" + cntr.incrementAndGet() + '%' + gridName);
+        super(gridName, DFLT_GRP, createName(cntr.incrementAndGet(), name, gridName));
 
         assert log != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 09e4877..bf6e869 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -1510,7 +1510,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
         nioSrvr.start();
 
-        commWorker = new CommunicationWorker();
+        commWorker = new CommunicationWorker(gridName);
 
         commWorker.start();
 
@@ -3024,9 +3024,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         private final BlockingQueue<GridNioRecoveryDescriptor> q = new LinkedBlockingQueue<>();
 
         /**
-         *
+         * @param gridName Grid name.
          */
-        private CommunicationWorker() {
+        private CommunicationWorker(String gridName) {
             super(gridName, "tcp-comm-worker", log);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
index 6ea77ae..974fa94 100644
--- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
+++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.util.worker.GridWorker;
  *      <li>Consistent naming of threads</li>
  *      <li>Dedicated parent thread group</li>
  *      <li>Backing interrupted flag</li>
+ *      <li>Name of the grid this thread belongs to</li>
  * </ul>
  * <b>Note</b>: this class is intended for internal use only.
  */
@@ -36,10 +37,10 @@ public class IgniteThread extends Thread {
     private static final ThreadGroup DFLT_GRP = new ThreadGroup("ignite");
 
     /** Number of all grid threads in the system. */
-    private static final AtomicLong threadCntr = new AtomicLong(0);
+    private static final AtomicLong cntr = new AtomicLong();
 
-    /** Boolean flag indicating of this thread is currently processing message. */
-    private boolean procMsg;
+    /** The name of the grid this thread belongs to. */
+    protected final String gridName;
 
     /**
      * Creates thread with given worker.
@@ -71,7 +72,25 @@ public class IgniteThread extends Thread {
      * @param r Runnable to execute.
      */
     public IgniteThread(ThreadGroup grp, String gridName, String threadName, Runnable r) {
-        super(grp, r, createName(threadCntr.incrementAndGet(), threadName, gridName));
+        super(grp, r, createName(cntr.incrementAndGet(), threadName, gridName));
+
+        this.gridName = gridName;
+    }
+
+    /**
+     * @param gridName Name of grid this thread is created for.
+     * @param threadGrp Thread group.
+     * @param threadName Name of thread.
+     */
+    protected IgniteThread(String gridName, ThreadGroup threadGrp, String threadName) {
+        super(threadGrp, threadName);
+
+        this.gridName = gridName;
+    }
+
+    /** {@inheritDoc} */
+    public String getGridName() {
+        return gridName;
     }
 
     /**
@@ -82,24 +101,10 @@ public class IgniteThread extends Thread {
      * @param gridName Grid name.
      * @return New thread name.
      */
-    private static String createName(long num, String threadName, String gridName) {
+    protected static String createName(long num, String threadName, String gridName) {
         return threadName + "-#" + num + '%' + gridName + '%';
     }
 
-    /**
-     * @param procMsg Flag indicating whether thread is currently processing message.
-     */
-    public void processingMessage(boolean procMsg) {
-        this.procMsg = procMsg;
-    }
-
-    /**
-     * @return Flag indicating whether thread is currently processing message.
-     */
-    public boolean processingMessage() {
-        return procMsg;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgniteThread.class, this, "name", getName());

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java
index 0af422f..940e1a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java
+++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java
@@ -166,7 +166,8 @@ public class IgniteThreadPoolExecutor extends ThreadPoolExecutor {
     /**
      * Creates a new service with the given initial parameters.
      *
-     * @param gridName Name of the grid
+     * @param threadNamePrefix Will be added at the beginning of all created threads.
+     * @param gridName Must be the name of the grid.
      * @param corePoolSize The number of threads to keep in the pool, even if they are idle.
      * @param maxPoolSize The maximum number of threads to allow in the pool.
      * @param keepAliveTime When the number of threads is greater than the core, this is the maximum time
@@ -175,6 +176,7 @@ public class IgniteThreadPoolExecutor extends ThreadPoolExecutor {
      *      runnable tasks submitted by the {@link #execute(Runnable)} method.
      */
     public IgniteThreadPoolExecutor(
+        String threadNamePrefix,
         String gridName,
         int corePoolSize,
         int maxPoolSize,
@@ -186,38 +188,7 @@ public class IgniteThreadPoolExecutor extends ThreadPoolExecutor {
             keepAliveTime,
             TimeUnit.MILLISECONDS,
             workQ,
-            new IgniteThreadFactory(gridName)
-        );
-    }
-
-    /**
-     * Creates a new service with the given initial parameters.
-     *
-     * @param gridName Name of the grid.
-     * @param corePoolSize The number of threads to keep in the pool, even if they are idle.
-     * @param maxPoolSize The maximum number of threads to allow in the pool.
-     * @param keepAliveTime When the number of threads is greater than the core, this is the maximum time
-     *      that excess idle threads will wait for new tasks before terminating.
-     * @param workQ The queue to use for holding tasks before they are executed. This queue will hold only the
-     *      runnable tasks submitted by the {@link #execute(Runnable)} method.
-     * @param hnd Optional handler to use when execution is blocked because the thread bounds and queue
-     *      capacities are reached. If {@code null} then {@code AbortPolicy}
-     *      handler is used by default.
-     */
-    public IgniteThreadPoolExecutor(
-        String gridName,
-        int corePoolSize,
-        int maxPoolSize,
-        long keepAliveTime,
-        BlockingQueue<Runnable> workQ,
-        RejectedExecutionHandler hnd) {
-        this(
-            corePoolSize,
-            maxPoolSize,
-            keepAliveTime,
-            workQ,
-            new IgniteThreadFactory(gridName),
-            hnd
+            new IgniteThreadFactory(gridName, threadNamePrefix)
         );
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
index 49be8dd..a06e6c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
@@ -3874,4 +3874,4 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
             rawValArr = rawReader.readDecimalArray();
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
index 5e43b2a..95ca9b5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
@@ -19,10 +19,10 @@ package org.apache.ignite.internal.processors.cache;
 
 import com.google.common.collect.ImmutableSet;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
-import java.util.HashSet;
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 5af90e9..41e9016 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -44,8 +46,6 @@ import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.EntryProcessorResult;
 import javax.cache.processor.MutableEntry;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
 import junit.framework.AssertionFailedError;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
index ef96d9f..cb16aeb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
@@ -43,12 +43,10 @@ import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
 import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.typedef.CA;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
-import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
index 2408176..0e858ed 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
@@ -24,8 +24,11 @@ import java.io.ObjectOutput;
 import java.io.Serializable;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RunnableFuture;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
@@ -69,6 +72,7 @@ import org.apache.ignite.p2p.GridP2PTestJob;
 import org.apache.ignite.p2p.GridP2PTestTask;
 import org.apache.ignite.testframework.GridTestClassLoader;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -84,6 +88,9 @@ public abstract class GridMarshallerAbstractTest extends GridCommonAbstractTest
     /** */
     private static Marshaller marsh;
 
+    /** */
+    private static String gridName;
+
     /** Closure job. */
     protected IgniteInClosure<String> c1 = new IgniteInClosure<String>() {
         @Override public void apply(String s) {
@@ -144,6 +151,7 @@ public abstract class GridMarshallerAbstractTest extends GridCommonAbstractTest
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         marsh = grid().configuration().getMarshaller();
+        gridName = grid().configuration().getGridName();
     }
 
     /**
@@ -831,8 +839,28 @@ public abstract class GridMarshallerAbstractTest extends GridCommonAbstractTest
      * @throws IgniteCheckedException Thrown if any exception occurs while unmarshalling.
      */
     @SuppressWarnings({"RedundantTypeArguments"})
-    protected static <T> T unmarshal(byte[] buf) throws IgniteCheckedException {
-        return marsh.<T>unmarshal(buf, Thread.currentThread().getContextClassLoader());
+    protected static <T> T unmarshal(final byte[] buf) throws IgniteCheckedException {
+        RunnableFuture<T> f = new FutureTask<>(new Callable<T>() {
+            @Override public T call() throws IgniteCheckedException {
+                return marsh.<T>unmarshal(buf, Thread.currentThread().getContextClassLoader());
+            }
+        });
+
+        // Any deserialization has to be executed under a thread, that contains the grid name.
+        new IgniteThread(gridName, "unmarshal-thread", f).start();
+
+        try {
+            return f.get();
+        }
+        catch (Exception e) {
+            if (e.getCause() instanceof IgniteCheckedException) {
+                throw (IgniteCheckedException)e.getCause();
+            }
+
+            fail(e.getCause().getMessage());
+        }
+
+        return null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index eaf63d7..2d91895 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -60,10 +60,10 @@ import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.portable.BinaryEnumCache;
 import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
 import org.apache.ignite.internal.util.GridClassLoaderCache;
-import org.apache.ignite.internal.portable.BinaryEnumCache;
 import org.apache.ignite.internal.util.GridTestClockTimer;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.F;
@@ -89,6 +89,7 @@ import org.apache.ignite.testframework.junits.logger.GridTestLog4jLogger;
 import org.apache.ignite.testframework.junits.multijvm.IgniteCacheProcessProxy;
 import org.apache.ignite.testframework.junits.multijvm.IgniteNodeRunner;
 import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy;
+import org.apache.ignite.thread.IgniteThread;
 import org.apache.log4j.ConsoleAppender;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -150,7 +151,7 @@ public abstract class GridAbstractTest extends TestCase {
     private static long ts = System.currentTimeMillis();
 
     /** Starting grid name. */
-    protected static ThreadLocal<String> startingGrid = new ThreadLocal<>();
+    protected final static ThreadLocal<String> startingGrid = new ThreadLocal<>();
 
     /**
      *
@@ -1628,7 +1629,7 @@ public abstract class GridAbstractTest extends TestCase {
     @Override protected void runTest() throws Throwable {
         final AtomicReference<Throwable> ex = new AtomicReference<>();
 
-        Thread runner = new Thread("test-runner") {
+        Thread runner = new IgniteThread(getTestGridName(), "test-runner", new Runnable() {
             @Override public void run() {
                 try {
                     runTestInternal();
@@ -1639,7 +1640,7 @@ public abstract class GridAbstractTest extends TestCase {
                     ex.set(hnd != null ? hnd.apply(e) : e);
                 }
             }
-        };
+        });
 
         runner.start();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
index 970002d..097c75f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
@@ -153,7 +153,7 @@ public class IgniteTestResources {
      * @param prestart Prestart flag.
      */
     public void startThreads(boolean prestart) {
-        execSvc = new IgniteThreadPoolExecutor(nodeId.toString(), 40, 40, Long.MAX_VALUE,
+        execSvc = new IgniteThreadPoolExecutor(nodeId.toString(), null, 40, 40, Long.MAX_VALUE,
             new LinkedBlockingQueue<Runnable>());
 
         // Improve concurrency for testing.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 7e45470..88d0834 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -29,17 +29,17 @@ import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionSelfTest;
 import org.apache.ignite.cache.store.GridCacheBalancingStoreSelfTest;
 import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest;
 import org.apache.ignite.cache.store.StoreResourceInjectionSelfTest;
+import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreMultitreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
-import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerSelfTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;
 import org.apache.ignite.internal.processors.cache.CacheFutureExceptionSelfTest;
-import org.apache.ignite.internal.processors.cache.CachePutEventListenerErrorSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheNamesSelfTest;
+import org.apache.ignite.internal.processors.cache.CachePutEventListenerErrorSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityApiSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityMapperSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityRoutingSelfTest;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
index 574ddeb..efe4d2a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 import java.util.HashSet;
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityRoutingSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccSelfTest;
@@ -27,7 +28,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictio
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredSelfTest;
 import org.apache.ignite.internal.processors.cache.expiry.IgniteCacheAtomicLocalExpiryPolicyTest;
-import org.apache.ignite.internal.processors.cache.expiry.IgniteCacheExpiryPolicyTestSuite;
 import org.apache.ignite.internal.processors.cache.portable.GridPortableCacheEntryMemorySizeSelfTest;
 import org.apache.ignite.internal.processors.cache.portable.datastreaming.DataStreamProcessorPortableSelfTest;
 import org.apache.ignite.internal.processors.cache.portable.datastreaming.GridDataStreamerImplSelfTest;
@@ -44,7 +44,6 @@ import org.apache.ignite.internal.processors.cache.portable.distributed.dht.Grid
 import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortablesNearPartitionedByteArrayValuesSelfTest;
 import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest;
-import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.testframework.config.GridTestProperties;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
index 411e363..6f8ca2d 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import java.io.IOException;
 import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import javax.cache.CacheException;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SqlQuery;
@@ -75,6 +76,10 @@ public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnm
                 private void readObject(ObjectInputStream is) throws IOException {
                     throw new IOException();
                 }
+
+                private void writeObject(ObjectOutputStream os) throws IOException {
+                    throw new IOException();
+                }
             })).getAll();
 
             assertTrue("Request unmarshalling failed, but error response was not sent.", portableMarshaller());

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/osgi/src/test/java/org/apache/ignite/osgi/AbstractIgniteKarafTest.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/test/java/org/apache/ignite/osgi/AbstractIgniteKarafTest.java b/modules/osgi/src/test/java/org/apache/ignite/osgi/AbstractIgniteKarafTest.java
index 786b543..394de68 100644
--- a/modules/osgi/src/test/java/org/apache/ignite/osgi/AbstractIgniteKarafTest.java
+++ b/modules/osgi/src/test/java/org/apache/ignite/osgi/AbstractIgniteKarafTest.java
@@ -23,9 +23,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import javax.inject.Inject;
-
 import org.apache.karaf.features.FeaturesService;
-
 import org.junit.runner.RunWith;
 import org.ops4j.pax.exam.Option;
 import org.ops4j.pax.exam.junit.PaxExam;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
index 8a073fd..bc7dc4b 100644
--- a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
@@ -44,6 +44,7 @@ import org.apache.ignite.internal.util.lang.GridTuple;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.lifecycle.LifecycleEventType;
 import org.apache.ignite.resources.IgniteInstanceResource;
@@ -857,6 +858,34 @@ public class GridFactorySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testCurrentIgnite() throws Exception {
+        final String LEFT = "LEFT";
+        final String RIGHT = "RIGHT";
+        try {
+            Ignite iLEFT = startGrid(LEFT);
+            Ignite iRIGHT = startGrid(RIGHT);
+            waitForDiscovery(iLEFT, iRIGHT);
+
+            iLEFT.compute(iLEFT.cluster().forRemotes()).run(new IgniteRunnable() {
+                @Override public void run() {
+                    assert Ignition.localIgnite().name().equals(RIGHT);
+                }
+            });
+
+            iRIGHT.compute(iRIGHT.cluster().forRemotes()).run(new IgniteRunnable() {
+                @Override public void run() {
+                    assert Ignition.localIgnite().name().equals(LEFT);
+                }
+            });
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
      * Test task.
      */
     private static class TestTask extends ComputeTaskSplitAdapter<Void, Void> {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0adee3ae/modules/spring/src/test/java/org/apache/ignite/internal/GridSpringBeanSerializationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/internal/GridSpringBeanSerializationSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/internal/GridSpringBeanSerializationSelfTest.java
index 5c82242..cbe35bd 100644
--- a/modules/spring/src/test/java/org/apache/ignite/internal/GridSpringBeanSerializationSelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/internal/GridSpringBeanSerializationSelfTest.java
@@ -71,6 +71,8 @@ public class GridSpringBeanSerializationSelfTest extends GridCommonAbstractTest
 
         cfg.setConnectorConfiguration(null);
 
+        cfg.setGridName(getTestGridName());
+
         return cfg;
     }
 


[47/50] [abbrv] ignite git commit: Fix OSGi test.

Posted by sb...@apache.org.
Fix OSGi test.

(@ProbeBuilder applies to all test classes when run from a suite).


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

Branch: refs/heads/ignite-1.5.1
Commit: bdbe4068b3851f7a412935acb3e36860cd08efa9
Parents: 3baf4d1
Author: Raul Kripalani <ra...@apache.org>
Authored: Tue Dec 8 21:04:36 2015 +0100
Committer: Raul Kripalani <ra...@apache.org>
Committed: Tue Dec 8 21:04:36 2015 +0100

----------------------------------------------------------------------
 .../test/java/org/apache/ignite/osgi/IgniteOsgiServiceTest.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bdbe4068/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiServiceTest.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiServiceTest.java b/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiServiceTest.java
index 9a2e92d..4f875ed 100644
--- a/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiServiceTest.java
+++ b/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiServiceTest.java
@@ -93,7 +93,7 @@ public class IgniteOsgiServiceTest extends AbstractIgniteKarafTest {
      */
     @ProbeBuilder
     public TestProbeBuilder probeConfiguration(TestProbeBuilder probe) {
-        probe.setHeader(Constants.IMPORT_PACKAGE, "*,org.apache.ignite.osgi.activators");
+        probe.setHeader(Constants.IMPORT_PACKAGE, "*,org.apache.ignite.osgi.activators;resolution:=\"optional\"");
 
         return probe;
     }


[24/50] [abbrv] ignite git commit: ignite-1.5 Added tx mdoe configuration for IgnitePutTxBenchmark.

Posted by sb...@apache.org.
ignite-1.5 Added tx mdoe configuration for IgnitePutTxBenchmark.


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

Branch: refs/heads/ignite-1.5.1
Commit: 233231de53e0d2c7d91244f7d65a13ecd515d8ce
Parents: 8033bc4
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 7 10:40:19 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 7 10:40:19 2015 +0300

----------------------------------------------------------------------
 .../yardstick/cache/IgnitePutTxBenchmark.java   | 26 +++++++++++++++++---
 1 file changed, 22 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/233231de/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
index 9c3389f..15b7cd6 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
@@ -18,8 +18,11 @@
 package org.apache.ignite.yardstick.cache;
 
 import java.util.Map;
+import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.yardstick.IgniteBenchmarkUtils;
 import org.apache.ignite.yardstick.cache.model.SampleValue;
 import org.yardstickframework.BenchmarkConfiguration;
 
@@ -27,20 +30,35 @@ import org.yardstickframework.BenchmarkConfiguration;
  * Ignite benchmark that performs transactional put operations.
  */
 public class IgnitePutTxBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private IgniteTransactions txs;
+
+    /** */
+    private Callable<Void> clo;
+
     /** {@inheritDoc} */
     @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
         super.setUp(cfg);
 
         if (!IgniteSystemProperties.getBoolean("SKIP_MAP_CHECK"))
             ignite().compute().broadcast(new WaitMapExchangeFinishCallable());
+
+        txs = ignite().transactions();
+
+        clo = new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                int key = nextRandom(args.range());
+
+                cache.put(key, new SampleValue(key));
+
+                return null;
+            }
+        };
     }
 
     /** {@inheritDoc} */
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        int key = nextRandom(args.range());
-
-        // Implicit transaction is used.
-        cache.put(key, new SampleValue(key));
+        IgniteBenchmarkUtils.doInTransaction(txs, args.txConcurrency(), args.txIsolation(), clo);
 
         return true;
     }


[10/50] [abbrv] ignite git commit: IGNITE-1270 Implement basic support for OSGi.

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/pom.xml
----------------------------------------------------------------------
diff --git a/modules/osgi/pom.xml b/modules/osgi/pom.xml
new file mode 100644
index 0000000..d6d46bf
--- /dev/null
+++ b/modules/osgi/pom.xml
@@ -0,0 +1,171 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!--
+    POM file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
+    <artifactId>ignite-osgi</artifactId>
+    <version>1.5.0-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.osgi</groupId>
+            <artifactId>org.osgi.core</artifactId>
+            <version>${osgi.core.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.ops4j.pax.exam</groupId>
+            <artifactId>pax-exam</artifactId>
+            <version>4.6.0</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.ops4j.pax.exam</groupId>
+            <artifactId>pax-exam-junit4</artifactId>
+            <version>4.6.0</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.ops4j.pax.exam</groupId>
+            <artifactId>pax-exam-container-karaf</artifactId>
+            <version>4.6.0</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.karaf</groupId>
+            <artifactId>apache-karaf</artifactId>
+            <version>${karaf.version}</version>
+            <type>tar.gz</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>javax.inject</groupId>
+            <artifactId>javax.inject</artifactId>
+            <version>1</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.ops4j.pax.url</groupId>
+            <artifactId>pax-url-aether</artifactId>
+            <version>2.4.3</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.12</version>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. 
+                 This bundle is a fragment attached to the ignite-core bundle, as it contains and exports classes in 
+                 the org.apache.ignite.internal.processors.query.h2.opt in the same manner as ignite-geospatial, thus 
+                 leading to a split package situation in OSGi. It also contains an internal processor.
+            -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <configuration>
+                    <instructions>
+                        <Fragment-Host>org.apache.ignite.ignite-core</Fragment-Host>
+                    </instructions>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.servicemix.tooling</groupId>
+                <artifactId>depends-maven-plugin</artifactId>
+                <version>1.3.1</version>
+                <executions>
+                    <execution>
+                        <id>generate-depends-file</id>
+                        <goals>
+                            <goal>generate-depends-file</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <forkCount>1</forkCount>
+                    <systemProperties>
+                        <property>
+                            <name>karafVersion</name>
+                            <value>${karaf.version}</value>
+                        </property>
+                        <property>
+                            <name>projectVersion</name>
+                            <value>${project.version}</value>
+                        </property>
+                        <property>
+                            <name>camelVersion</name>
+                            <value>${camel.version}</value>
+                        </property>
+                    </systemProperties>
+                </configuration>
+            </plugin>
+
+        </plugins>
+    </build>
+    
+</project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/main/java/org/apache/ignite/osgi/IgniteAbstractOsgiContextActivator.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/main/java/org/apache/ignite/osgi/IgniteAbstractOsgiContextActivator.java b/modules/osgi/src/main/java/org/apache/ignite/osgi/IgniteAbstractOsgiContextActivator.java
new file mode 100644
index 0000000..ac76f6e
--- /dev/null
+++ b/modules/osgi/src/main/java/org/apache/ignite/osgi/IgniteAbstractOsgiContextActivator.java
@@ -0,0 +1,238 @@
+/*
+ * 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.osgi;
+
+import java.util.Dictionary;
+import java.util.Hashtable;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.osgi.classloaders.BundleDelegatingClassLoader;
+import org.apache.ignite.osgi.classloaders.ContainerSweepClassLoader;
+import org.apache.ignite.osgi.classloaders.OsgiClassLoadingStrategyType;
+import org.jetbrains.annotations.Nullable;
+import org.osgi.framework.BundleActivator;
+import org.osgi.framework.BundleContext;
+
+/**
+ * This {@link BundleActivator} starts Apache Ignite inside the OSGi container when the bundle is started.
+ * <p>
+ * Create an implementation of this class and set the {@code Bundle-Activator} OSGi Manifest header to the FQN of
+ * your class.
+ * <p>
+ * You must provide the {@link IgniteConfiguration} to start by implementing the {@link #igniteConfiguration()}
+ * abstract method. The return value of this method cannot be {@code null}. For example, if your implementation is
+ * called {@code org.myorg.osgi.IgniteActivator}, your bundle must provide the following header:
+ * <pre>
+ * Bundle-Activator: org.myorg.osgi.IgniteActivator
+ * </pre>
+ * You may use the
+ * <a href="https://felix.apache.org/documentation/subprojects/apache-felix-maven-bundle-plugin-bnd.html">Maven
+ * Bundle Plugin</a> to generate your bundle (or bundle manifest), including the required header.
+ * <p>
+ * This activator also exports the Ignite instance as an OSGi service, with the property {@code ignite.name} set
+ * to the value of {@link Ignite#name()}, if and only if the name is not null.
+ * <p>
+ * Currently, Ignite only allows a single instance per container. We may remove this limitation if enough demand
+ * builds up in the community.
+ *
+ * @see <a href="http://wiki.osgi.org/wiki/Bundle-Activator">Bundle-Activator OSGi Manifest header</a>
+ *
+ */
+public abstract class IgniteAbstractOsgiContextActivator implements BundleActivator {
+    /** OSGI service property name. */
+    public static final String OSGI_SERVICE_PROP_IGNITE_NAME = "ignite.name";
+
+    /** The instance of Ignite started by this Activator. */
+    protected Ignite ignite;
+
+    /** Our bundle context. */
+    private BundleContext bundleCtx;
+
+    /** Ignite logger. */
+    private IgniteLogger log;
+
+    /**
+     * Method invoked by OSGi to start the bundle. It starts the specified Ignite configuration.
+     *
+     * @param ctx Bundle context.
+     * @throws Exception
+     */
+    @Override public final void start(BundleContext ctx) throws Exception {
+        // Ensure that no other instances are running.
+        if (IgniteOsgiUtils.gridCount() > 0) {
+            throw new IgniteException("Failed to start Ignite instance (another instance is already running " +
+                "and ignite-osgi is currently limited to a single instance per container).");
+        }
+
+        bundleCtx = ctx;
+
+        // Start the Ignite configuration specified by the user.
+        IgniteConfiguration cfg = igniteConfiguration();
+
+        A.notNull(cfg, "Ignite configuration");
+
+        // Override the classloader with the classloading strategy chosen by the user.
+        ClassLoader clsLdr;
+
+        if (classLoadingStrategy() == OsgiClassLoadingStrategyType.BUNDLE_DELEGATING)
+            clsLdr = new BundleDelegatingClassLoader(bundleCtx.getBundle(), Ignite.class.getClassLoader());
+        else
+            clsLdr = new ContainerSweepClassLoader(bundleCtx.getBundle(), Ignite.class.getClassLoader());
+
+        cfg.setClassLoader(clsLdr);
+
+        onBeforeStart(ctx);
+
+        // Start Ignite.
+        try {
+            ignite = Ignition.start(cfg);
+        }
+        catch (Throwable t) {
+            U.error(log, "Failed to start Ignite via OSGi Activator [errMsg=" + t.getMessage() + ']', t);
+
+            onAfterStart(ctx, t);
+
+            return;
+        }
+
+        log = ignite.log();
+
+        log.info("Started Ignite from OSGi Activator [name=" + ignite.name() + ']');
+
+        // Add into Ignite's OSGi registry.
+        IgniteOsgiUtils.classloaders().put(ignite, clsLdr);
+
+        // Export Ignite as a service.
+        exportOsgiService(ignite);
+
+        onAfterStart(ctx, null);
+    }
+
+    /**
+     * Stops Ignite when the bundle is stopping.
+     *
+     * @param ctx Bundle context.
+     * @throws Exception If failed.
+     */
+    @Override public final void stop(BundleContext ctx) throws Exception {
+        onBeforeStop(ctx);
+
+        try {
+            ignite.close();
+        }
+        catch (Throwable t) {
+            U.error(log, "Failed to stop Ignite via OSGi Activator [errMsg=" + t.getMessage() + ']', t);
+
+            onAfterStop(ctx, t);
+
+            return;
+        }
+
+        if (log.isInfoEnabled())
+            log.info("Stopped Ignite from OSGi Activator [name=" + ignite.name() + ']');
+
+        IgniteOsgiUtils.classloaders().remove(ignite);
+
+        onAfterStop(ctx, null);
+    }
+
+    /**
+     * This method is called before Ignite initialises.
+     * <p>
+     * The default implementation is empty. Override it to introduce custom logic.
+     *
+     * @param ctx The {@link BundleContext}.
+     */
+    protected void onBeforeStart(BundleContext ctx) {
+        // No-op.
+    }
+
+    /**
+     * This method is called after Ignite initialises, only if initialization succeeded.
+     * <p>
+     * The default implementation is empty. Override it to introduce custom logic.
+     *
+     * @param ctx The {@link BundleContext}.
+     * @param t Throwable in case an error occurred when starting. {@code null} otherwise.
+     */
+    protected void onAfterStart(BundleContext ctx, @Nullable Throwable t) {
+        // No-op.
+    }
+
+    /**
+     * This method is called before Ignite stops.
+     * <p>
+     * The default implementation is empty. Override it to introduce custom logic.
+     *
+     * @param ctx The {@link BundleContext}.
+     */
+    protected void onBeforeStop(BundleContext ctx) {
+        // No-op.
+    }
+
+    /**
+     * This method is called after Ignite stops, only if the operation succeeded.
+     * <p>
+     * The default implementation is empty. Override it to introduce custom logic.
+     *
+     * @param ctx The {@link BundleContext}.
+     * @param t Throwable in case an error occurred when stopping. {@code null} otherwise.
+     */
+    protected void onAfterStop(BundleContext ctx, @Nullable Throwable t) {
+        // No-op.
+    }
+
+    /**
+     * Override this method to provide the Ignite configuration this bundle will start.
+     *
+     * @return The Ignite configuration.
+     */
+    public abstract IgniteConfiguration igniteConfiguration();
+
+    /**
+     * Override this method to indicate which classloading strategy to use.
+     *
+     * @return The strategy.
+     */
+    public OsgiClassLoadingStrategyType classLoadingStrategy() {
+        return OsgiClassLoadingStrategyType.BUNDLE_DELEGATING;
+    }
+
+    /**
+     * Exports the Ignite instance onto the OSGi Service Registry.
+     *
+     * @param ignite Ignite.
+     */
+    private void exportOsgiService(Ignite ignite) {
+        Dictionary<String, String> dict = new Hashtable<>();
+
+        // Only add the service property if the grid name != null.
+        if (ignite.name() != null)
+            dict.put(OSGI_SERVICE_PROP_IGNITE_NAME, ignite.name());
+
+        bundleCtx.registerService(Ignite.class, ignite, dict);
+
+        if (log.isInfoEnabled())
+            log.info("Exported OSGi service for Ignite with properties: " + dict);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/main/java/org/apache/ignite/osgi/IgniteOsgiUtils.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/main/java/org/apache/ignite/osgi/IgniteOsgiUtils.java b/modules/osgi/src/main/java/org/apache/ignite/osgi/IgniteOsgiUtils.java
new file mode 100644
index 0000000..deda3e8
--- /dev/null
+++ b/modules/osgi/src/main/java/org/apache/ignite/osgi/IgniteOsgiUtils.java
@@ -0,0 +1,69 @@
+/*
+ * 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.osgi;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.Ignite;
+import org.jsr166.ConcurrentHashMap8;
+import org.osgi.framework.Bundle;
+import org.osgi.framework.FrameworkUtil;
+
+/**
+ * Helper class for OSGi.
+ */
+public class IgniteOsgiUtils {
+    /** Whether we are running in an OSGi container. */
+    private static boolean osgi = FrameworkUtil.getBundle(IgniteOsgiUtils.class) != null;
+
+    /** Maps Ignite instances to the ClassLoaders of the bundles they were started from. */
+    private static final ConcurrentMap<Ignite, ClassLoader> CLASSLOADERS = new ConcurrentHashMap8<>();
+
+    /**
+     * Private constructor.
+     */
+    private IgniteOsgiUtils() { }
+
+    /**
+     * Returns whether we are running in an OSGi environment.
+     *
+     * @return {@code true/false}.
+     */
+    public static boolean isOsgi() {
+        return osgi;
+    }
+
+    /**
+     * Returns a {@link Map} of {@link Ignite} instances and the classloaders of the {@link Bundle}s they were
+     * started from.
+     *
+     * @return The {@link Map}.
+     */
+    protected static Map<Ignite, ClassLoader> classloaders() {
+        return CLASSLOADERS;
+    }
+
+    /**
+     * Returns the number of grids currently running in this OSGi container.
+     *
+     * @return The grid count.
+     */
+    public static int gridCount() {
+        return CLASSLOADERS.size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/BundleDelegatingClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/BundleDelegatingClassLoader.java b/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/BundleDelegatingClassLoader.java
new file mode 100644
index 0000000..07c0682
--- /dev/null
+++ b/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/BundleDelegatingClassLoader.java
@@ -0,0 +1,147 @@
+/*
+ * 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.osgi.classloaders;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Enumeration;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.osgi.framework.Bundle;
+
+/**
+ * A {@link ClassLoader} implementation delegating to a given OSGi bundle, and to the specified {@link ClassLoader}
+ * as a fallback.
+ */
+public class BundleDelegatingClassLoader extends ClassLoader {
+    /** The bundle which loaded Ignite. */
+    protected final Bundle bundle;
+
+    /** The fallback classloader, expected to be the ignite-core classloader. */
+    @GridToStringExclude
+    protected final ClassLoader clsLdr;
+
+    /**
+     * Constructor.
+     *
+     * @param bundle The bundle
+     */
+    public BundleDelegatingClassLoader(Bundle bundle) {
+        this(bundle, null);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param bundle The bundle.
+     * @param classLoader Fallback classloader.
+     */
+    public BundleDelegatingClassLoader(Bundle bundle, ClassLoader classLoader) {
+        this.bundle = bundle;
+        this.clsLdr = classLoader;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Class<?> findClass(String name) throws ClassNotFoundException {
+        return bundle.loadClass(name);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected URL findResource(String name) {
+        URL resource = bundle.getResource(name);
+
+        if (resource == null && clsLdr != null)
+            resource = clsLdr.getResource(name);
+
+        return resource;
+    }
+
+    /**
+     * Finds a given resource from within the {@link #bundle}.
+     *
+     * @param name The resource name.
+     * @return URLs of resources.
+     * @throws IOException
+     */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    protected Enumeration findResources(String name) throws IOException {
+        return bundle.getResources(name);
+    }
+
+    /**
+     * Loads a class trying the {@link #bundle} first, falling back to the ClassLoader {@link #clsLdr}.
+     *
+     * @param name Class name.
+     * @param resolve {@code true} to resolve the class.
+     * @return The Class.
+     * @throws ClassNotFoundException
+     */
+    @Override protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
+        Class<?> cls;
+
+        try {
+            cls = findClass(name);
+        }
+        catch (ClassNotFoundException e) {
+            if (clsLdr == null)
+                throw classNotFoundException(name);
+
+            try {
+                cls = clsLdr.loadClass(name);
+            }
+            catch (ClassNotFoundException e2) {
+                throw classNotFoundException(name);
+            }
+
+        }
+
+        if (resolve)
+            resolveClass(cls);
+
+        return cls;
+    }
+
+    /**
+     * Returns the {@link Bundle} to which this ClassLoader is associated.
+     *
+     * @return The Bundle.
+     */
+    public Bundle getBundle() {
+        return bundle;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(BundleDelegatingClassLoader.class, this);
+    }
+
+    /**
+     * Builds a {@link ClassNotFoundException}.
+     *
+     * @param clsName Class name.
+     * @return The exception.
+     */
+    protected ClassNotFoundException classNotFoundException(String clsName) {
+        String s = "Failed to resolve class [name=" + clsName +
+            ", bundleId=" + bundle.getBundleId() +
+            ", symbolicName=" + bundle.getSymbolicName() +
+            ", fallbackClsLdr=" + clsLdr + ']';
+
+        return new ClassNotFoundException(s);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/ContainerSweepClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/ContainerSweepClassLoader.java b/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/ContainerSweepClassLoader.java
new file mode 100644
index 0000000..e2e773a
--- /dev/null
+++ b/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/ContainerSweepClassLoader.java
@@ -0,0 +1,134 @@
+/*
+ * 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.osgi.classloaders;
+
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.jsr166.ConcurrentHashMap8;
+import org.osgi.framework.Bundle;
+import org.osgi.framework.Constants;
+
+/**
+ * A {@link ClassLoader} implementation that first attempts to load the class from the associated {@link Bundle}. As
+ * a fallback, it sweeps the entire OSGi container to find the requested class, returning the first hit.
+ * <p>
+ * It keeps a cache of resolved classes and unresolvable classes, in order to optimize subsequent lookups.
+ */
+public class ContainerSweepClassLoader extends BundleDelegatingClassLoader {
+    /** Classes resolved previously. */
+    private final ConcurrentMap<String, Bundle> resolved = new ConcurrentHashMap8<>();
+
+    /** Unresolvable classes. */
+    private final Set<String> nonResolvable = new GridConcurrentHashSet<>();
+
+    /**
+     * Constructor with a {@link Bundle} only.
+     *
+     * @param bundle The bundle.
+     */
+    public ContainerSweepClassLoader(Bundle bundle) {
+        super(bundle);
+    }
+
+    /**
+     * Constructor with a {@link Bundle} and another {@link ClassLoader} to check.
+     *
+     * @param bundle The bundle.
+     * @param classLoader The other classloader to check.
+     */
+    public ContainerSweepClassLoader(Bundle bundle, ClassLoader classLoader) {
+        super(bundle, classLoader);
+    }
+
+    /**
+     * Runs the same logic to find the class as {@link BundleDelegatingClassLoader}, but if not found, sweeps the
+     * OSGi container to locate the first {@link Bundle} that can load the class, and uses it to do so.
+     *
+     * @param name The classname.
+     * @param resolve Whether to resolve the class or not.
+     * @return The loaded class.
+     * @throws ClassNotFoundException
+     */
+    @Override protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
+        // If we know it's not resolvable, throw the exception immediately.
+        if (nonResolvable.contains(name))
+            throw classNotFoundException(name);
+
+        Class<?> cls;
+
+        // First, delegate to super, and return the class if found.
+        try {
+            cls = super.loadClass(name, resolve);
+            return cls;
+        }
+        catch (ClassNotFoundException e) {
+            // Continue.
+        }
+
+        // Else, check the cache.
+        if (resolved.containsKey(name))
+            return resolved.get(name).loadClass(name);
+
+        // If still unresolved, sweep the container.
+        cls = sweepContainer(name);
+
+        // If still unresolved, throw the exception.
+        if (cls == null)
+            throw classNotFoundException(name);
+
+        return cls;
+    }
+
+    /**
+     * Sweeps the OSGi container to find the first {@link Bundle} that can load the class.
+     *
+     * @param name The classname.
+     * @return The loaded class.
+     */
+    protected Class<?> sweepContainer(String name) {
+        Class<?> cls = null;
+
+        Bundle[] bundles = bundle.getBundleContext().getBundles();
+
+        int bundleIdx = 0;
+
+        for (; bundleIdx < bundles.length; bundleIdx++) {
+            Bundle b = bundles[bundleIdx];
+
+            // Skip bundles that haven't reached RESOLVED state; skip fragments.
+            if (b.getState() <= Bundle.RESOLVED || b.getHeaders().get(Constants.FRAGMENT_HOST) != null)
+                continue;
+
+            try {
+                cls = b.loadClass(name);
+                break;
+            }
+            catch (ClassNotFoundException e) {
+                // No-op.
+            }
+        }
+
+        if (cls == null)
+            nonResolvable.add(name);
+        else
+            resolved.put(name, bundles[bundleIdx]);
+
+        return cls;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/OsgiClassLoadingStrategyType.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/OsgiClassLoadingStrategyType.java b/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/OsgiClassLoadingStrategyType.java
new file mode 100644
index 0000000..9afde42
--- /dev/null
+++ b/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/OsgiClassLoadingStrategyType.java
@@ -0,0 +1,29 @@
+/*
+ * 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.osgi.classloaders;
+
+/**
+ * Enum for the user to indicate which type of {@link ClassLoader} Ignite should use.
+ */
+public enum OsgiClassLoadingStrategyType {
+    /** Use this value for {@link BundleDelegatingClassLoader}. */
+    BUNDLE_DELEGATING,
+
+    /** Use this value for {@link ContainerSweepClassLoader}. */
+    CONTAINER_SWEEP
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/test/java/org/apache/ignite/osgi/AbstractIgniteKarafTest.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/test/java/org/apache/ignite/osgi/AbstractIgniteKarafTest.java b/modules/osgi/src/test/java/org/apache/ignite/osgi/AbstractIgniteKarafTest.java
new file mode 100644
index 0000000..786b543
--- /dev/null
+++ b/modules/osgi/src/test/java/org/apache/ignite/osgi/AbstractIgniteKarafTest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.osgi;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import javax.inject.Inject;
+
+import org.apache.karaf.features.FeaturesService;
+
+import org.junit.runner.RunWith;
+import org.ops4j.pax.exam.Option;
+import org.ops4j.pax.exam.junit.PaxExam;
+import org.ops4j.pax.exam.karaf.options.LogLevelOption;
+import org.ops4j.pax.exam.spi.reactors.ExamReactorStrategy;
+import org.ops4j.pax.exam.spi.reactors.PerMethod;
+import org.osgi.framework.BundleContext;
+
+import static org.ops4j.pax.exam.CoreOptions.junitBundles;
+import static org.ops4j.pax.exam.CoreOptions.maven;
+import static org.ops4j.pax.exam.CoreOptions.mavenBundle;
+import static org.ops4j.pax.exam.CoreOptions.options;
+import static org.ops4j.pax.exam.CoreOptions.systemProperty;
+import static org.ops4j.pax.exam.karaf.options.KarafDistributionOption.editConfigurationFileExtend;
+import static org.ops4j.pax.exam.karaf.options.KarafDistributionOption.features;
+import static org.ops4j.pax.exam.karaf.options.KarafDistributionOption.karafDistributionConfiguration;
+import static org.ops4j.pax.exam.karaf.options.KarafDistributionOption.logLevel;
+
+/**
+ * Abstract test class that sets up an Apache Karaf container with Ignite installed.
+ */
+@RunWith(PaxExam.class)
+@ExamReactorStrategy(PerMethod.class)
+public abstract class AbstractIgniteKarafTest {
+    /** Features we do not expect to be installed. */
+    protected static final Set<String> IGNORED_FEATURES = new HashSet<>(
+        Arrays.asList("ignite-log4j", "ignite-scalar-2.10"));
+
+    /** Regex matching ignite features. */
+    protected static final String IGNITE_FEATURES_NAME_REGEX = "ignite.*";
+
+    /** Project version. */
+    protected static final String PROJECT_VERSION = System.getProperty("projectVersion");
+
+    /** Pax Exam will inject the Bundle Context here. */
+    @Inject
+    protected BundleContext bundleCtx;
+
+    /** Pax Exam will inject the Karaf Features Service. */
+    @Inject
+    protected FeaturesService featuresSvc;
+
+    /**
+     * Base configuration for a Karaf container running the specified Ignite features.
+     *
+     * @return The configuration.
+     */
+    public Option[] baseConfig() {
+        return options(
+
+            // Specify which version of Karaf to use.
+            karafDistributionConfiguration()
+                .frameworkUrl(maven().groupId("org.apache.karaf").artifactId("apache-karaf").type("tar.gz")
+                    .versionAsInProject())
+                .karafVersion(System.getProperty("karafVersion"))
+                .useDeployFolder(false)
+                .unpackDirectory(new File("target/paxexam/unpack")),
+
+            // Add JUnit bundles.
+            junitBundles(),
+
+            // Add the additional JRE exports that Ignite requires.
+            editConfigurationFileExtend("etc/jre.properties", "jre-1.7", "sun.nio.ch"),
+            editConfigurationFileExtend("etc/jre.properties", "jre-1.8", "sun.nio.ch"),
+
+            // Make log level INFO.
+            logLevel(LogLevelOption.LogLevel.INFO),
+
+            // Add our features repository.
+            features(mavenBundle()
+                    .groupId("org.apache.ignite").artifactId("ignite-osgi-karaf")
+                    .version(System.getProperty("projectVersion")).type("xml/features"),
+                featuresToInstall().toArray(new String[0])),
+
+            // Propagate the projectVersion system property.
+            systemProperty("projectVersion").value(System.getProperty("projectVersion"))
+        );
+    }
+
+    protected abstract List<String> featuresToInstall();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteKarafFeaturesInstallationTest.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteKarafFeaturesInstallationTest.java b/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteKarafFeaturesInstallationTest.java
new file mode 100644
index 0000000..c0eb06b
--- /dev/null
+++ b/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteKarafFeaturesInstallationTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.osgi;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.karaf.features.Feature;
+import org.junit.Test;
+import org.ops4j.pax.exam.Configuration;
+import org.ops4j.pax.exam.CoreOptions;
+import org.ops4j.pax.exam.Option;
+import org.ops4j.pax.exam.karaf.options.KarafDistributionOption;
+import org.osgi.framework.Bundle;
+import org.osgi.framework.Constants;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Pax Exam test class to check if all features could be resolved and installed.
+ */
+public class IgniteKarafFeaturesInstallationTest extends AbstractIgniteKarafTest {
+    /** Number of features expected to exist. */
+    private static final int EXPECTED_FEATURES = 25;
+
+    private static final String CAMEL_REPO_URI = "mvn:org.apache.camel.karaf/apache-camel/" +
+        System.getProperty("camelVersion") + "/xml/features";
+
+    /**
+     * Container configuration.
+     *
+     * @return The configuration.
+     */
+    @Configuration
+    public Option[] config() {
+        List<Option> options = new ArrayList<>(Arrays.asList(baseConfig()));
+
+        options.add(KarafDistributionOption.features(CAMEL_REPO_URI));
+
+        return CoreOptions.options(options.toArray(new Option[0]));
+    }
+
+    /**
+     * @throws Exception
+     */
+    @Test
+    public void testAllBundlesActiveAndFeaturesInstalled() throws Exception {
+        // Asssert all bundles except fragments are ACTIVE.
+        for (Bundle b : bundleCtx.getBundles()) {
+            System.out.println(String.format("Checking state of bundle [symbolicName=%s, state=%s]",
+                b.getSymbolicName(), b.getState()));
+
+            if (b.getHeaders().get(Constants.FRAGMENT_HOST) == null)
+                assertTrue(b.getState() == Bundle.ACTIVE);
+        }
+
+        // Check that according to the FeaturesService, all Ignite features except ignite-log4j are installed.
+        Feature[] features = featuresSvc.getFeatures(IGNITE_FEATURES_NAME_REGEX);
+
+        assertNotNull(features);
+        assertEquals(EXPECTED_FEATURES, features.length);
+
+        for (Feature f : features) {
+            if (IGNORED_FEATURES.contains(f.getName()))
+                continue;
+
+            boolean installed = featuresSvc.isInstalled(f);
+
+            System.out.println(String.format("Checking if feature is installed [featureName=%s, installed=%s]",
+                f.getName(), installed));
+
+            assertTrue(installed);
+            assertEquals(PROJECT_VERSION.replaceAll("-", "."), f.getVersion());
+        }
+    }
+
+    /**
+     * @return Features list.
+     */
+    @Override protected List<String> featuresToInstall() {
+        return Arrays.asList("ignite-all");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiServiceTest.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiServiceTest.java b/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiServiceTest.java
new file mode 100644
index 0000000..9a2e92d
--- /dev/null
+++ b/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiServiceTest.java
@@ -0,0 +1,131 @@
+/*
+ * 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.osgi;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import javax.inject.Inject;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.osgi.activators.BasicIgniteTestActivator;
+import org.apache.ignite.osgi.activators.TestOsgiFlags;
+import org.apache.ignite.osgi.activators.TestOsgiFlagsImpl;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.ops4j.pax.exam.Configuration;
+import org.ops4j.pax.exam.CoreOptions;
+import org.ops4j.pax.exam.Option;
+import org.ops4j.pax.exam.ProbeBuilder;
+import org.ops4j.pax.exam.TestProbeBuilder;
+import org.ops4j.pax.exam.junit.PaxExam;
+import org.ops4j.pax.exam.spi.reactors.ExamReactorStrategy;
+import org.ops4j.pax.exam.spi.reactors.PerMethod;
+import org.ops4j.pax.exam.util.Filter;
+import org.osgi.framework.BundleContext;
+import org.osgi.framework.Constants;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.ops4j.pax.exam.CoreOptions.streamBundle;
+import static org.ops4j.pax.tinybundles.core.TinyBundles.bundle;
+import static org.ops4j.pax.tinybundles.core.TinyBundles.withBnd;
+
+/**
+ * Pax Exam test class to check whether the Ignite service is exposed properly and whether lifecycle callbacks
+ * are invoked.
+ */
+@RunWith(PaxExam.class)
+@ExamReactorStrategy(PerMethod.class)
+public class IgniteOsgiServiceTest extends AbstractIgniteKarafTest {
+    /** Injects the Ignite OSGi service. */
+    @Inject @Filter("(ignite.name=testGrid)")
+    private Ignite ignite;
+
+    @Inject
+    private BundleContext bundleCtx;
+
+    /**
+     * @return Config.
+     */
+    @Configuration
+    public Option[] bundleDelegatingConfig() {
+        List<Option> options = new ArrayList<>(Arrays.asList(baseConfig()));
+
+        // Add bundles we require.
+        options.add(
+            streamBundle(bundle()
+                .add(BasicIgniteTestActivator.class)
+                .add(TestOsgiFlags.class)
+                .add(TestOsgiFlagsImpl.class)
+                .set(Constants.BUNDLE_SYMBOLICNAME, BasicIgniteTestActivator.class.getSimpleName())
+                .set(Constants.BUNDLE_ACTIVATOR, BasicIgniteTestActivator.class.getName())
+                .set(Constants.EXPORT_PACKAGE, "org.apache.ignite.osgi.activators")
+                .set(Constants.DYNAMICIMPORT_PACKAGE, "*")
+                .build(withBnd())));
+
+        // Uncomment this if you'd like to debug inside the container.
+        // options.add(KarafDistributionOption.debugConfiguration());
+
+        return CoreOptions.options(options.toArray(new Option[0]));
+    }
+
+    /**
+     * Builds the probe.
+     *
+     * @param probe The probe builder.
+     * @return The probe builder.
+     */
+    @ProbeBuilder
+    public TestProbeBuilder probeConfiguration(TestProbeBuilder probe) {
+        probe.setHeader(Constants.IMPORT_PACKAGE, "*,org.apache.ignite.osgi.activators");
+
+        return probe;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testServiceExposedAndCallbacksInvoked() throws Exception {
+        assertNotNull(ignite);
+        assertEquals("testGrid", ignite.name());
+
+        TestOsgiFlags flags = (TestOsgiFlags) bundleCtx.getService(
+            bundleCtx.getAllServiceReferences(TestOsgiFlags.class.getName(), null)[0]);
+
+        assertNotNull(flags);
+        assertEquals(Boolean.TRUE, flags.getOnBeforeStartInvoked());
+        assertEquals(Boolean.TRUE, flags.getOnAfterStartInvoked());
+
+        // The bundle is still not stopped, therefore these callbacks cannot be tested.
+        assertNull(flags.getOnBeforeStopInvoked());
+        assertNull(flags.getOnAfterStopInvoked());
+
+        // No exceptions.
+        assertNull(flags.getOnAfterStartThrowable());
+        assertNull(flags.getOnAfterStopThrowable());
+    }
+
+    /**
+     * @return Features.
+     */
+    @Override protected List<String> featuresToInstall() {
+        return Arrays.asList("ignite-core");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiTestSuite.java b/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiTestSuite.java
new file mode 100644
index 0000000..0a3d69c
--- /dev/null
+++ b/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteOsgiTestSuite.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.osgi;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+/**
+ * Test suite for OSGi-related test cases.
+ * <p>
+ * NOTE: Have to use JUnit 4 annotations because Pax Exam is built on JUnit 4.
+ */
+@RunWith(Suite.class)
+@Suite.SuiteClasses({IgniteOsgiServiceTest.class, IgniteKarafFeaturesInstallationTest.class})
+public class IgniteOsgiTestSuite {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/BasicIgniteTestActivator.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/BasicIgniteTestActivator.java b/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/BasicIgniteTestActivator.java
new file mode 100644
index 0000000..c414092
--- /dev/null
+++ b/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/BasicIgniteTestActivator.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.osgi.activators;
+
+import java.util.Hashtable;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.osgi.IgniteAbstractOsgiContextActivator;
+import org.apache.ignite.osgi.classloaders.OsgiClassLoadingStrategyType;
+import org.jetbrains.annotations.Nullable;
+import org.osgi.framework.BundleContext;
+
+/**
+ * Basic Ignite Activator for testing.
+ */
+public class BasicIgniteTestActivator extends IgniteAbstractOsgiContextActivator {
+    /** Flags to report our state to a watcher. */
+    private TestOsgiFlagsImpl flags = new TestOsgiFlagsImpl();
+
+    /**
+     * @return Ignite config.
+     */
+    @Override public IgniteConfiguration igniteConfiguration() {
+        IgniteConfiguration config = new IgniteConfiguration();
+
+        config.setGridName("testGrid");
+
+        return config;
+    }
+
+    /**
+     * @return Strategy.
+     */
+    @Override public OsgiClassLoadingStrategyType classLoadingStrategy() {
+        return OsgiClassLoadingStrategyType.BUNDLE_DELEGATING;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onBeforeStart(BundleContext ctx) {
+        flags.onBeforeStartInvoked = Boolean.TRUE;
+
+        // Export the flags as an OSGi service.
+        ctx.registerService(TestOsgiFlags.class, flags, new Hashtable<String, Object>());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onAfterStart(BundleContext ctx, @Nullable Throwable t) {
+        flags.onAfterStartInvoked = Boolean.TRUE;
+        flags.onAfterStartThrowable = t;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onBeforeStop(BundleContext ctx) {
+        flags.onBeforeStopInvoked = Boolean.TRUE;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onAfterStop(BundleContext ctx, @Nullable Throwable t) {
+        flags.onAfterStopInvoked = Boolean.TRUE;
+        flags.onAfterStopThrowable = t;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/TestOsgiFlags.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/TestOsgiFlags.java b/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/TestOsgiFlags.java
new file mode 100644
index 0000000..09a2d29
--- /dev/null
+++ b/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/TestOsgiFlags.java
@@ -0,0 +1,53 @@
+/*
+ * 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.osgi.activators;
+
+/**
+ * Interface to export the flags in OSGi.
+ */
+public interface TestOsgiFlags {
+    /**
+     * @return The flag.
+     */
+    Boolean getOnBeforeStartInvoked();
+
+    /**
+     * @return The flag.
+     */
+    Boolean getOnAfterStartInvoked();
+
+    /**
+     * @return The flag.
+     */
+    Throwable getOnAfterStartThrowable();
+
+    /**
+     * @return The flag.
+     */
+    Boolean getOnBeforeStopInvoked();
+
+    /**
+     * @return The flag.
+     */
+    Boolean getOnAfterStopInvoked();
+
+    /**
+     * @return The flag.
+     */
+    Throwable getOnAfterStopThrowable();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/TestOsgiFlagsImpl.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/TestOsgiFlagsImpl.java b/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/TestOsgiFlagsImpl.java
new file mode 100644
index 0000000..ccec2df
--- /dev/null
+++ b/modules/osgi/src/test/java/org/apache/ignite/osgi/activators/TestOsgiFlagsImpl.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.osgi.activators;
+
+/**
+ * Data transfer object representing flags we want to watch from the OSGi tests.
+ */
+public class TestOsgiFlagsImpl implements TestOsgiFlags {
+    /** onBeforeStartInvoked flag. */
+    public Boolean onBeforeStartInvoked;
+
+    /** onAfterStartInvoked flag. */
+    public Boolean onAfterStartInvoked;
+
+    /** onAfterStartThrowable flag. */
+    public Throwable onAfterStartThrowable;
+
+    /** onBeforeStartInvoked flag. */
+    public Boolean onBeforeStopInvoked;
+
+    /** onAfterStopInvoked flag. */
+    public Boolean onAfterStopInvoked;
+
+    /** onAfterStopThrowable flag. */
+    public Throwable onAfterStopThrowable;
+
+    /**
+     * @return The flag.
+     */
+    @Override public Boolean getOnBeforeStartInvoked() {
+        return onBeforeStartInvoked;
+    }
+
+    /**
+     * @return The flag.
+     */
+    @Override public Boolean getOnAfterStartInvoked() {
+        return onAfterStartInvoked;
+    }
+
+    /**
+     * @return The flag.
+     */
+    @Override public Throwable getOnAfterStartThrowable() {
+        return onAfterStartThrowable;
+    }
+
+    /**
+     * @return The flag.
+     */
+    @Override public Boolean getOnBeforeStopInvoked() {
+        return onBeforeStopInvoked;
+    }
+
+    /**
+     * @return The flag.
+     */
+    @Override public Boolean getOnAfterStopInvoked() {
+        return onAfterStopInvoked;
+    }
+
+    /**
+     * @return The flag.
+     */
+    @Override public Throwable getOnAfterStopThrowable() {
+        return onAfterStopThrowable;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 3dc18d3..0780144 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -34,6 +34,13 @@
     <version>1.5.0-b1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
+    <properties>
+        <osgi.export.package>
+            org.apache.ignite.internal.processors.rest.protocols.http.jetty,
+            {local-packages}
+        </osgi.export.package>
+    </properties>
+
     <dependencies>
         <dependency>
             <groupId>org.apache.ignite</groupId>
@@ -44,13 +51,13 @@
         <dependency>
             <groupId>org.apache.tomcat</groupId>
             <artifactId>tomcat-servlet-api</artifactId>
-            <version>8.0.23</version>
+            <version>${tomcat.version}</version>
         </dependency>
 
         <dependency>
             <groupId>commons-lang</groupId>
             <artifactId>commons-lang</artifactId>
-            <version>2.6</version>
+            <version>${commons.lang.version}</version>
         </dependency>
 
         <dependency>
@@ -92,38 +99,38 @@
         <dependency>
             <groupId>net.sf.json-lib</groupId>
             <artifactId>json-lib</artifactId>
-            <version>2.4</version>
+            <version>${jsonlib.version}</version>
             <classifier>jdk15</classifier>
         </dependency>
 
         <dependency>
             <groupId>net.sf.ezmorph</groupId>
             <artifactId>ezmorph</artifactId>
-            <version>1.0.6</version>
+            <version>${ezmorph.version}</version>
         </dependency>
 
         <dependency>
             <groupId>commons-collections</groupId>
             <artifactId>commons-collections</artifactId>
-            <version>3.2.1</version>
+            <version>${commons.collections.version}</version>
         </dependency>
 
         <dependency>
             <groupId>commons-beanutils</groupId>
             <artifactId>commons-beanutils</artifactId>
-            <version>1.8.3</version>
+            <version>${commons.beanutils.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
-            <version>1.7.7</version>
+            <version>${slf4j.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-log4j12</artifactId>
-            <version>1.7.7</version>
+            <version>${slf4j.version}</version>
         </dependency>
 
         <dependency>
@@ -131,4 +138,15 @@
             <artifactId>log4j</artifactId>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 83e515b..9a958eb 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -44,7 +44,7 @@
         <dependency>
             <groupId>org.scala-lang</groupId>
             <artifactId>scala-library</artifactId>
-            <version>2.10.4</version>
+            <version>${scala210.library.version}</version>
         </dependency>
 
         <dependency>
@@ -109,6 +109,12 @@
                 <groupId>net.alchim31.maven</groupId>
                 <artifactId>scala-maven-plugin</artifactId>
             </plugin>
+
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
         </plugins>
 
         <!-- TODO IGNITE-956 FIX scaladocs plugins-->
@@ -194,5 +200,6 @@
                 <!--</executions>-->
             <!--</plugin>-->
         <!--</plugins>-->
+
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index e78ce73..301a103 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -44,7 +44,7 @@
         <dependency>
             <groupId>org.scala-lang</groupId>
             <artifactId>scala-library</artifactId>
-            <version>2.11.7</version>
+            <version>${scala211.library.version}</version>
         </dependency>
 
         <dependency>
@@ -184,6 +184,13 @@
                     </execution>
                 </executions>
             </plugin>
+
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 6e687a6..cb53713 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -34,6 +34,13 @@
     <version>1.5.0-b1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
+    <properties>
+        <osgi.export.package>
+            org.apache.ignite.internal.processors.schedule,
+            {local-packages}
+        </osgi.export.package>
+    </properties>
+
     <dependencies>
         <dependency>
             <groupId>org.apache.ignite</groupId>
@@ -44,7 +51,7 @@
         <dependency>
             <groupId>it.sauronsoftware.cron4j</groupId>
             <artifactId>cron4j</artifactId>
-            <version>2.2.5</version>
+            <version>${cron4j.version}</version>
         </dependency>
 
         <dependency>
@@ -75,4 +82,20 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this fragment. It is a fragment because it contains internal processors 
+                 that would be looked up by ignite-core. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <configuration>
+                    <instructions>
+                        <Fragment-Host>org.apache.ignite.ignite-core</Fragment-Host>
+                    </instructions>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 2fae337..36f6b13 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -44,7 +44,7 @@
         <dependency>
             <groupId>com.h2database</groupId>
             <artifactId>h2</artifactId>
-            <version>1.3.175</version>
+            <version>${h2.version}</version>
             <scope>test</scope>
         </dependency>
     </dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 130ea19..1449549 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -44,7 +44,17 @@
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
-            <version>1.6.4</version>
+            <version>${slf4j.version}</version>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 627cc49..0b24898 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -52,7 +52,7 @@
         <dependency>
             <groupId>org.scala-lang</groupId>
             <artifactId>scala-library</artifactId>
-            <version>2.10.4</version>
+            <version>${scala210.library.version}</version>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 2e90632..1808a0a 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -106,7 +106,7 @@
         <dependency>
             <groupId>com.h2database</groupId>
             <artifactId>h2</artifactId>
-            <version>1.3.175</version>
+            <version>${h2.version}</version>
             <scope>test</scope>
         </dependency>
 
@@ -118,11 +118,10 @@
             <scope>test</scope>
         </dependency>
 
-
         <dependency>
             <groupId>com.h2database</groupId>
             <artifactId>h2</artifactId>
-            <version>1.3.175</version>
+            <version>${h2.version}</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
@@ -136,5 +135,30 @@
                 </excludes>
             </testResource>
         </testResources>
+
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. 
+                 This bundle is a fragment attached to the ignite-core bundle, as it contains and exports classes in the org.apache.ignite
+                 leading to a split package situation in OSGi.
+            -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <configuration>
+                    <instructions>
+                        <Fragment-Host>org.apache.ignite.ignite-core</Fragment-Host>
+                        <Require-Bundle>
+                            org.apache.servicemix.bundles.spring-beans,
+                            org.apache.servicemix.bundles.spring-context,
+                            org.apache.servicemix.bundles.spring-context-support,
+                            org.apache.servicemix.bundles.spring-core,
+                            org.apache.servicemix.bundles.spring-expression,
+                            org.apache.servicemix.bundles.spring-jdbc,
+                            org.apache.servicemix.bundles.spring-tx
+                        </Require-Bundle>
+                    </instructions>
+                </configuration>
+            </plugin>
+        </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 1079c00..e1b0fe5 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -44,7 +44,7 @@
         <dependency>
             <groupId>com.jcraft</groupId>
             <artifactId>jsch</artifactId>
-            <version>0.1.53</version>
+            <version>${jsch.version}</version>
         </dependency>
 
         <dependency>
@@ -69,4 +69,21 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <!-- This is a fragment because it's an internal processor module. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <configuration>
+                    <instructions>
+                        <Fragment-Host>org.apache.ignite.ignite-core</Fragment-Host>
+                    </instructions>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/twitter/pom.xml
----------------------------------------------------------------------
diff --git a/modules/twitter/pom.xml b/modules/twitter/pom.xml
index fc924f6..1649043 100644
--- a/modules/twitter/pom.xml
+++ b/modules/twitter/pom.xml
@@ -65,7 +65,7 @@
         <dependency>
             <groupId>com.twitter</groupId>
             <artifactId>hbc-twitter4j</artifactId>
-            <version>2.2.0</version>
+            <version>${twitter.hbc.version}</version>
         </dependency>
 
         <dependency>
@@ -118,4 +118,14 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 5f2436e..37c77ba 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -80,19 +80,19 @@
         <dependency>
             <groupId>net.sf.jtidy</groupId>
             <artifactId>jtidy</artifactId>
-            <version>r938</version>
+            <version>${jtidy.version}</version>
         </dependency>
 
         <dependency>
             <groupId>commons-codec</groupId>
             <artifactId>commons-codec</artifactId>
-            <version>1.6</version>
+            <version>${commons.codec.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.tomcat</groupId>
             <artifactId>tomcat-servlet-api</artifactId>
-            <version>8.0.23</version>
+            <version>${tomcat.version}</version>
             <scope>test</scope>
         </dependency>
 
@@ -145,4 +145,15 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/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 8d1bd74..87ce699 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -80,13 +80,13 @@
         <dependency>
             <groupId>org.scala-lang</groupId>
             <artifactId>scala-library</artifactId>
-            <version>2.10.4</version>
+            <version>${scala210.library.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.scala-lang</groupId>
             <artifactId>jline</artifactId>
-            <version>2.10.4</version>
+            <version>${scala210.jline.version}</version>
         </dependency>
         <!-- Third party dependencies -->
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index 1ed21ae..d4fc6cf 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -80,7 +80,7 @@
         <dependency>
             <groupId>org.scala-lang</groupId>
             <artifactId>scala-library</artifactId>
-            <version>2.11.7</version>
+            <version>${scala211.library.version}</version>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index 5cdda0c..7754a4b 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -53,13 +53,13 @@
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
-            <version>1.7.7</version>
+            <version>${slf4j.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-log4j12</artifactId>
-            <version>1.7.7</version>
+            <version>${slf4j.version}</version>
         </dependency>
         <!-- Third party dependencies -->
     </dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index c97e77b..d25ce61 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -44,7 +44,7 @@
         <dependency>
             <groupId>org.apache.tomcat</groupId>
             <artifactId>tomcat-servlet-api</artifactId>
-            <version>8.0.23</version>
+            <version>${tomcat.version}</version>
         </dependency>
 
         <dependency>
@@ -89,4 +89,15 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+    
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml
index 8aa5730..8ade247 100644
--- a/modules/zookeeper/pom.xml
+++ b/modules/zookeeper/pom.xml
@@ -34,10 +34,6 @@
     <version>1.5.0-b1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
-    <properties>
-        <curator.version>2.9.1</curator.version>
-    </properties>
-
     <dependencies>
         <dependency>
             <groupId>org.apache.ignite</groupId>
@@ -87,4 +83,14 @@
         </dependency>
     </dependencies>
 
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
 </project>


[40/50] [abbrv] ignite git commit: IGNITE-1694 : .NET: Use DllImport instead of delegates in P/Invoke.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/568f910f/modules/platforms/dotnet/Apache.Ignite.FxCop
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.FxCop b/modules/platforms/dotnet/Apache.Ignite.FxCop
index 5c7f7f1..0df73c5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.FxCop
+++ b/modules/platforms/dotnet/Apache.Ignite.FxCop
@@ -107,11 +107,329 @@
    </RuleFile>
    <RuleFile Name="$(FxCopDir)\Rules\PerformanceRules.dll" Enabled="True" AllRulesEnabled="True" />
    <RuleFile Name="$(FxCopDir)\Rules\PortabilityRules.dll" Enabled="True" AllRulesEnabled="True" />
-   <RuleFile Name="$(FxCopDir)\Rules\SecurityRules.dll" Enabled="True" AllRulesEnabled="True" />
+   <RuleFile Name="$(FxCopDir)\Rules\SecurityRules.dll" Enabled="True" AllRulesEnabled="False">
+    <Rule Name="AptcaMethodsShouldOnlyCallAptcaMethods" Enabled="True" />
+    <Rule Name="AptcaTypesShouldOnlyExtendAptcaBaseTypes" Enabled="True" />
+    <Rule Name="ArrayFieldsShouldNotBeReadOnly" Enabled="True" />
+    <Rule Name="CallGCKeepAliveWhenUsingNativeResources" Enabled="True" />
+    <Rule Name="CatchNonClsCompliantExceptionsInGeneralHandlers" Enabled="True" />
+    <Rule Name="DoNotDeclareReadOnlyMutableReferenceTypes" Enabled="True" />
+    <Rule Name="DoNotIndirectlyExposeMethodsWithLinkDemands" Enabled="True" />
+    <Rule Name="MethodSecurityShouldBeASupersetOfType" Enabled="True" />
+    <Rule Name="OverrideLinkDemandsShouldBeIdenticalToBase" Enabled="True" />
+    <Rule Name="PointersShouldNotBeVisible" Enabled="True" />
+    <Rule Name="ReviewDeclarativeSecurityOnValueTypes" Enabled="True" />
+    <Rule Name="ReviewDenyAndPermitOnlyUsage" Enabled="True" />
+    <Rule Name="ReviewImperativeSecurity" Enabled="True" />
+    <Rule Name="ReviewVisibleEventHandlers" Enabled="True" />
+    <Rule Name="SealMethodsThatSatisfyPrivateInterfaces" Enabled="True" />
+    <Rule Name="SecureAsserts" Enabled="True" />
+    <Rule Name="SecuredTypesShouldNotExposeFields" Enabled="True" />
+    <Rule Name="SecureSerializationConstructors" Enabled="True" />
+    <Rule Name="SecurityTransparentAssembliesShouldNotContainSecurityCriticalCode" Enabled="True" />
+    <Rule Name="SecurityTransparentCodeShouldNotAssert" Enabled="True" />
+    <Rule Name="SecurityTransparentCodeShouldNotReferenceNonpublicSecurityCriticalCode" Enabled="True" />
+    <Rule Name="StaticConstructorsShouldBePrivate" Enabled="True" />
+    <Rule Name="TypeLinkDemandsRequireInheritanceDemands" Enabled="True" />
+    <Rule Name="WrapVulnerableFinallyClausesInOuterTry" Enabled="True" />
+   </RuleFile>
    <RuleFile Name="$(FxCopDir)\Rules\UsageRules.dll" Enabled="True" AllRulesEnabled="True" />
   </RuleFiles>
   <Groups />
   <Settings />
  </Rules>
- <FxCopReport Version="1.36" />
+ <FxCopReport Version="1.36">
+  <Targets>
+   <Target Name="$(ProjectDir)/Apache.Ignite.Core/bin/x64/Debug/Apache.Ignite.Core.dll">
+    <Modules>
+     <Module Name="apache.ignite.core.dll">
+      <Namespaces>
+       <Namespace Name="Apache.Ignite.Core.Binary">
+        <Types>
+         <Type Name="IBinaryReader">
+          <Members>
+           <Member Name="#GetRawReader()">
+            <Messages>
+             <Message TypeName="UsePropertiesWhereAppropriate" Category="Microsoft.Design" CheckId="CA1024" Created="2015-11-11 15:43:38Z">
+              <Issue Certainty="50">
+               <Item>'IBinaryReader.GetRawReader()'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+         <Type Name="IBinaryWriter">
+          <Members>
+           <Member Name="#GetRawWriter()">
+            <Messages>
+             <Message TypeName="UsePropertiesWhereAppropriate" Category="Microsoft.Design" CheckId="CA1024" Created="2015-11-11 15:43:38Z">
+              <Issue Certainty="50">
+               <Item>'IBinaryWriter.GetRawWriter()'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+         <Type Name="IIgniteBinary">
+          <Members>
+           <Member Name="#GetBinaryTypes()">
+            <Messages>
+             <Message TypeName="UsePropertiesWhereAppropriate" Category="Microsoft.Design" CheckId="CA1024" Created="2015-11-11 15:43:38Z">
+              <Issue Certainty="50">
+               <Item>'IIgniteBinary.GetBinaryTypes()'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+        </Types>
+       </Namespace>
+       <Namespace Name="Apache.Ignite.Core.Cache">
+        <Types>
+         <Type Name="ICache`2">
+          <Members>
+           <Member Name="#TryLocalPeek(!0,!1&amp;,Apache.Ignite.Core.Cache.CachePeekMode[])">
+            <Messages>
+             <Message Id="1#" TypeName="AvoidOutParameters" Category="Microsoft.Design" CheckId="CA1021" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'value'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+        </Types>
+       </Namespace>
+       <Namespace Name="Apache.Ignite.Core.Compute">
+        <Types>
+         <Type Name="ComputeJobAdapter`1">
+          <Members>
+           <Member Name="#GetArgument`1(System.Int32)">
+            <Messages>
+             <Message TypeName="DoNotRaiseReservedExceptionTypes" Category="Microsoft.Usage" CheckId="CA2201" Created="2015-11-11 15:43:38Z">
+              <Issue Name="Reserved">
+               <Item>'ComputeJobAdapter&lt;T&gt;.GetArgument&lt;TArg&gt;(int)'</Item>
+               <Item>'IndexOutOfRangeException'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+        </Types>
+       </Namespace>
+       <Namespace Name="Apache.Ignite.Core.Events">
+        <Types>
+         <Type Name="IEvents">
+          <Members>
+           <Member Name="#GetEnabledEvents()">
+            <Messages>
+             <Message TypeName="UsePropertiesWhereAppropriate" Category="Microsoft.Design" CheckId="CA1024" Created="2015-11-11 15:43:38Z">
+              <Issue Certainty="50">
+               <Item>'IEvents.GetEnabledEvents()'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+        </Types>
+       </Namespace>
+       <Namespace Name="Apache.Ignite.Core.Impl">
+        <Types>
+         <Type Name="IgniteUtils">
+          <Members>
+           <Member Name="#.cctor()">
+            <Messages>
+             <Message TypeName="InitializeReferenceTypeStaticFieldsInline" Category="Microsoft.Performance" CheckId="CA1810" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'IgniteUtils'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+        </Types>
+       </Namespace>
+       <Namespace Name="Apache.Ignite.Core.Impl.Binary">
+        <Types>
+         <Type Name="BinaryReader">
+          <Members>
+           <Member Name="#Deserialize`1()">
+            <Messages>
+             <Message Id="System.String.Format(System.String,System.Object)" TypeName="SpecifyIFormatProvider" Category="Microsoft.Globalization" CheckId="CA1305" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'string.Format(string, object)'</Item>
+               <Item>'BinaryReader.Deserialize&lt;T&gt;()'</Item>
+               <Item>'string.Format(IFormatProvider, string, params object[])'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+           <Member Name="#IsNotNullHeader(System.Byte)">
+            <Messages>
+             <Message Id="System.String.Format(System.String,System.Object,System.Object)" TypeName="SpecifyIFormatProvider" Category="Microsoft.Globalization" CheckId="CA1305" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'string.Format(string, object, object)'</Item>
+               <Item>'BinaryReader.IsNotNullHeader(byte)'</Item>
+               <Item>'string.Format(IFormatProvider, string, params object[])'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+         <Type Name="Marshaller">
+          <Members>
+           <Member Name="#AddType(System.Type,System.Int32,System.String,System.Boolean,System.Boolean,Apache.Ignite.Core.Binary.IBinaryNameMapper,Apache.Ignite.Core.Binary.IBinaryIdMapper,Apache.Ignite.Core.Binary.IBinarySerializer,System.String)">
+            <Messages>
+             <Message Id="System.String.Format(System.String,System.Object,System.Object,System.Object)" TypeName="SpecifyIFormatProvider" Category="Microsoft.Globalization" CheckId="CA1305" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'string.Format(string, object, object, object)'</Item>
+               <Item>'Marshaller.AddType(Type, int, string, bool, bool, IBinaryNameMapper, IBinaryIdMapper, IBinarySerializer, string)'</Item>
+               <Item>'string.Format(IFormatProvider, string, params object[])'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+        </Types>
+       </Namespace>
+       <Namespace Name="Apache.Ignite.Core.Impl.Common">
+        <Types>
+         <Type Name="Classpath">
+          <Members>
+           <Member Name="#AppendHomeClasspath(System.String,System.Boolean,System.Text.StringBuilder)">
+            <Messages>
+             <Message Id="System.String.EndsWith(System.String)" TypeName="SpecifyStringComparison" Category="Microsoft.Globalization" CheckId="CA1307" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'Classpath.AppendHomeClasspath(string, bool, StringBuilder)'</Item>
+               <Item>'string.EndsWith(string)'</Item>
+               <Item>'string.EndsWith(string, StringComparison)'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+           <Member Name="#CreateClasspath(System.String,Apache.Ignite.Core.IgniteConfiguration,System.Boolean)">
+            <Messages>
+             <Message Id="System.String.EndsWith(System.String)" TypeName="SpecifyStringComparison" Category="Microsoft.Globalization" CheckId="CA1307" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'Classpath.CreateClasspath(string, IgniteConfiguration, bool)'</Item>
+               <Item>'string.EndsWith(string)'</Item>
+               <Item>'string.EndsWith(string, StringComparison)'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+         <Type Name="Future`1">
+          <Members>
+           <Member Name="#OnNullResult()">
+            <Messages>
+             <Message TypeName="DoNotCatchGeneralExceptionTypes" Category="Microsoft.Design" CheckId="CA1031" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'Future&lt;T&gt;.OnNullResult()'</Item>
+               <Item>'Exception'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+         <Type Name="IgniteHome">
+          <Members>
+           <Member Name="#Resolve(Apache.Ignite.Core.IgniteConfiguration)">
+            <Messages>
+             <Message Id="System.String.Format(System.String,System.Object)" TypeName="SpecifyIFormatProvider" Category="Microsoft.Globalization" CheckId="CA1305" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'string.Format(string, object)'</Item>
+               <Item>'IgniteHome.Resolve(IgniteConfiguration)'</Item>
+               <Item>'string.Format(IFormatProvider, string, params object[])'</Item>
+              </Issue>
+             </Message>
+             <Message Id="System.String.Format(System.String,System.Object,System.Object)" TypeName="SpecifyIFormatProvider" Category="Microsoft.Globalization" CheckId="CA1305" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'string.Format(string, object, object)'</Item>
+               <Item>'IgniteHome.Resolve(IgniteConfiguration)'</Item>
+               <Item>'string.Format(IFormatProvider, string, params object[])'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+        </Types>
+       </Namespace>
+       <Namespace Name="Apache.Ignite.Core.Impl.Memory">
+        <Types>
+         <Type Name="IPlatformMemory">
+          <Members>
+           <Member Name="#GetStream()">
+            <Messages>
+             <Message TypeName="UsePropertiesWhereAppropriate" Category="Microsoft.Design" CheckId="CA1024" Created="2015-11-11 15:43:38Z">
+              <Issue Certainty="50">
+               <Item>'IPlatformMemory.GetStream()'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+        </Types>
+       </Namespace>
+       <Namespace Name="Apache.Ignite.Core.Impl.Unmanaged">
+        <Types>
+         <Type Name="UnmanagedUtils">
+          <Members>
+           <Member Name="#.cctor()">
+            <Messages>
+             <Message Id="System.String.Format(System.String,System.Object,System.Object)" TypeName="SpecifyIFormatProvider" Category="Microsoft.Globalization" CheckId="CA1305" Created="2015-11-11 15:43:38Z">
+              <Issue>
+               <Item>'string.Format(string, object, object)'</Item>
+               <Item>'UnmanagedUtils.UnmanagedUtils()'</Item>
+               <Item>'string.Format(IFormatProvider, string, params object[])'</Item>
+              </Issue>
+             </Message>
+            </Messages>
+           </Member>
+          </Members>
+         </Type>
+        </Types>
+       </Namespace>
+      </Namespaces>
+     </Module>
+    </Modules>
+   </Target>
+  </Targets>
+  <Rules>
+   <Rule TypeName="AvoidOutParameters" Category="Microsoft.Design" CheckId="CA1021">
+    <Resolution Name="Default">Consider a design that does not require that {0} be an out parameter.</Resolution>
+   </Rule>
+   <Rule TypeName="DoNotCatchGeneralExceptionTypes" Category="Microsoft.Design" CheckId="CA1031">
+    <Resolution Name="Default">Modify {0} to catch a more specific exception than {1} or rethrow the exception.</Resolution>
+   </Rule>
+   <Rule TypeName="DoNotRaiseReservedExceptionTypes" Category="Microsoft.Usage" CheckId="CA2201">
+    <Resolution Name="Reserved">{0} creates an exception of type {1}, an exception type that is reserved by the runtime and should never be raised by managed code. If this exception instance might be thrown, use a different exception type.</Resolution>
+   </Rule>
+   <Rule TypeName="InitializeReferenceTypeStaticFieldsInline" Category="Microsoft.Performance" CheckId="CA1810">
+    <Resolution Name="Default">Initialize all static fields in {0} when those fields are declared and remove the explicit static constructor.</Resolution>
+   </Rule>
+   <Rule TypeName="SpecifyIFormatProvider" Category="Microsoft.Globalization" CheckId="CA1305">
+    <Resolution Name="Default">Because the behavior of {0} could vary based on the current user's locale settings, replace this call in {1} with a call to {2}. If the result of {2} will be displayed to the user, specify 'CultureInfo.CurrentCulture' as the 'IFormatProvider' parameter. Otherwise, if the result will be stored and accessed by software, such as when it is persisted to disk or to a database, specify 'CultureInfo.InvariantCulture'.</Resolution>
+   </Rule>
+   <Rule TypeName="SpecifyStringComparison" Category="Microsoft.Globalization" CheckId="CA1307">
+    <Resolution Name="Default">{0} makes a call to {1} that does not explicitly provide a StringComparison. This should be replaced with a call to {2}.</Resolution>
+   </Rule>
+   <Rule TypeName="UsePropertiesWhereAppropriate" Category="Microsoft.Design" CheckId="CA1024">
+    <Resolution Name="Default">Change {0} to a property if appropriate.</Resolution>
+   </Rule>
+  </Rules>
+ </FxCopReport>
 </FxCopProject>


[33/50] [abbrv] ignite git commit: ignite-1692 Changed test initialization logic to remove race confition that caused ClusterTopologyServerNotFoundException

Posted by sb...@apache.org.
ignite-1692 Changed test initialization logic to remove race confition that caused ClusterTopologyServerNotFoundException


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

Branch: refs/heads/ignite-1.5.1
Commit: 5cd0039a535b4c2ea7434d4b085c9e18f28c930d
Parents: d579183
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 8 13:12:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 8 13:12:24 2015 +0300

----------------------------------------------------------------------
 .../DataStreamProcessorSelfTest.java            | 25 ++++++++++++--------
 1 file changed, 15 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5cd0039a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
index 65dd690..9fedc35 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
@@ -206,14 +206,17 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
     @SuppressWarnings("ErrorNotRethrown")
     private void checkDataStreamer() throws Exception {
         try {
-            Ignite g1 = startGrid(1);
-
             useCache = true;
 
-            Ignite g2 = startGrid(2);
+            Ignite igniteWithCache = startGrid(2);
+
             startGrid(3);
 
-            final IgniteDataStreamer<Integer, Integer> ldr = g1.dataStreamer(null);
+            useCache = false;
+
+            Ignite igniteWithoutCache = startGrid(1);
+
+            final IgniteDataStreamer<Integer, Integer> ldr = igniteWithoutCache.dataStreamer(null);
 
             ldr.receiver(DataStreamerCacheUpdaters.<Integer, Integer>batchedSorted());
 
@@ -255,7 +258,7 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
 
             assertEquals(total, s2 + s3);
 
-            final IgniteDataStreamer<Integer, Integer> rmvLdr = g2.dataStreamer(null);
+            final IgniteDataStreamer<Integer, Integer> rmvLdr = igniteWithCache.dataStreamer(null);
 
             rmvLdr.receiver(DataStreamerCacheUpdaters.<Integer, Integer>batchedSorted());
 
@@ -443,15 +446,17 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
             // Start all required nodes.
             int idx = 1;
 
-            for (int i = 0; i < nodesCntNoCache; i++)
-                startGrid(idx++);
-
             useCache = true;
 
             for (int i = 0; i < nodesCntCache; i++)
                 startGrid(idx++);
 
-            Ignite g1 = grid(1);
+            useCache = false;
+
+            for (int i = 0; i < nodesCntNoCache; i++)
+                startGrid(idx++);
+
+            Ignite g1 = grid(idx - 1);
 
             // Get and configure loader.
             final IgniteDataStreamer<Integer, Integer> ldr = g1.dataStreamer(null);
@@ -1019,4 +1024,4 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
             }
         }
     }
-}
\ No newline at end of file
+}


[22/50] [abbrv] ignite git commit: IGNITE-2041 Fixed typo.

Posted by sb...@apache.org.
IGNITE-2041 Fixed typo.


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

Branch: refs/heads/ignite-1.5.1
Commit: 3d585d5a5d3ccc5c3071d7547a19cf055bd9ce83
Parents: 23ead50
Author: AKuznetsov <ak...@gridgain.com>
Authored: Sat Dec 5 16:57:53 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Sat Dec 5 16:57:53 2015 +0700

----------------------------------------------------------------------
 .../binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java    | 2 +-
 .../ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3d585d5a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java
index 9df9f79..63d947c 100644
--- a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java
@@ -138,7 +138,7 @@ public class CacheBinaryAutoStoreExample {
                 cache.clear();
 
                 System.out.println(">>> ------------------------------------------");
-                System.out.println(">>> Load data to cache from DB with cusom SQL...");
+                System.out.println(">>> Load data to cache from DB with custom SQL...");
 
                 // Load cache on all data nodes with custom SQL statement.
                 cache.loadCache(null, "java.lang.Long", "select * from PERSON where id <= 3");

http://git-wip-us.apache.org/repos/asf/ignite/blob/3d585d5a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java
index a262c38..5498e57 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java
@@ -143,7 +143,7 @@ public class CacheAutoStoreExample {
                 cache.clear();
 
                 System.out.println(">>> ------------------------------------------");
-                System.out.println(">>> Load data to cache from DB with cusom SQL...");
+                System.out.println(">>> Load data to cache from DB with custom SQL...");
 
                 // Load cache on all data nodes with custom SQL statement.
                 cache.loadCache(null, "java.lang.Long", "select * from PERSON where id <= 3");


[43/50] [abbrv] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

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


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

Branch: refs/heads/ignite-1.5.1
Commit: 0e81a3973cccfe6d912e073cd9f44db5a5c46d79
Parents: 568f910 3a34003
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Dec 8 16:36:12 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Dec 8 16:36:12 2015 +0300

----------------------------------------------------------------------
 .../config/benchmark-failover.properties        |  2 +-
 ...IgniteTransactionalWriteInvokeBenchmark.java | 42 +++++++++++++++-----
 2 files changed, 33 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[39/50] [abbrv] ignite git commit: Fixed race in IgniteTransactionalWriteInvokeBenchmark test.

Posted by sb...@apache.org.
Fixed race in IgniteTransactionalWriteInvokeBenchmark test.


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

Branch: refs/heads/ignite-1.5.1
Commit: 3a340034056f5da0ca00ed88e128b59cc28381d2
Parents: efe632b
Author: ashutak <as...@gridgain.com>
Authored: Tue Dec 8 16:02:36 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Dec 8 16:02:36 2015 +0300

----------------------------------------------------------------------
 .../config/benchmark-failover.properties        |  2 +-
 ...IgniteTransactionalWriteInvokeBenchmark.java | 42 +++++++++++++++-----
 2 files changed, 33 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3a340034/modules/yardstick/config/benchmark-failover.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-failover.properties b/modules/yardstick/config/benchmark-failover.properties
index 7ed464c..352e2d4 100644
--- a/modules/yardstick/config/benchmark-failover.properties
+++ b/modules/yardstick/config/benchmark-failover.properties
@@ -60,7 +60,7 @@ RESTART_SERVERS=true
 SERVER_HOSTS=localhost,localhost
 
 # Comma-separated list of the hosts to run BenchmarkDrivers on.
-DRIVER_HOSTS=localhost
+DRIVER_HOSTS=localhost,localhost
 
 # Remote username.
 # REMOTE_USER=

http://git-wip-us.apache.org/repos/asf/ignite/blob/3a340034/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
index 35befad..67044a1 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
@@ -25,9 +25,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
 import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCountDownLatch;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.affinity.Affinity;
@@ -58,29 +60,49 @@ public class IgniteTransactionalWriteInvokeBenchmark extends IgniteFailoverAbstr
     /** */
     private static final Long INITIAL_VALUE = 1L;
 
+    /** */
+    public static final int TIMEOUT_SEC = 20 * 60;
+
     /** {@inheritDoc} */
     @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
         super.setUp(cfg);
 
         assert args.keysCount() > 0 : "Count of keys: " + args.keysCount();
 
-        println(cfg, "Populating data...");
+        IgniteCountDownLatch latch = ignite().countDownLatch("DATA-POPULATED-LATCH-" + cacheName(), 1, true, true);
+
+        if (cfg.memberId() == 0) {
+            println(cfg, "Populating data for cache: " + cacheName());
 
-        long start = System.nanoTime();
+            long start = System.nanoTime();
 
-        try (IgniteDataStreamer<String, Long> dataLdr = ignite().dataStreamer(cacheName())) {
-            for (int k = 0; k < args.range() && !Thread.currentThread().isInterrupted(); k++) {
-                dataLdr.addData("key-" + k + "-master", INITIAL_VALUE);
+            try (IgniteDataStreamer<String, Long> dataLdr = ignite().dataStreamer(cacheName())) {
+                for (int k = 0; k < args.range() && !Thread.currentThread().isInterrupted(); k++) {
+                    dataLdr.addData("key-" + k + "-master", INITIAL_VALUE);
 
-                for (int i = 0; i < args.keysCount(); i++)
-                    dataLdr.addData("key-" + k + "-" + i, INITIAL_VALUE);
+                    for (int i = 0; i < args.keysCount(); i++)
+                        dataLdr.addData("key-" + k + "-" + i, INITIAL_VALUE);
 
-                if (k % 100000 == 0)
-                    println(cfg, "Populated accounts: " + k);
+                    if (k % 100000 == 0)
+                        println(cfg, "Populated accounts: " + k);
+                }
             }
+
+            println(cfg, "Finished populating data in " + ((System.nanoTime() - start) / 1_000_000)
+                + " ms. for cache: " + cacheName());
+
+            latch.countDown();
         }
+        else {
+            println(cfg, "Waiting for populating data in cache by driver with id 0: " + cacheName());
+
+            boolean success = latch.await(TIMEOUT_SEC, TimeUnit.SECONDS);
 
-        println(cfg, "Finished populating data in " + ((System.nanoTime() - start) / 1_000_000) + " ms.");
+            if (!success)
+                throw new IllegalStateException("Failed to wait that data populating finish.");
+
+            println(cfg, "Finished waiting for populating data in cache by driver with id 0: " + cacheName());
+        }
     }
 
     /** {@inheritDoc} */


[42/50] [abbrv] ignite git commit: IGNITE-1694 : .NET: Use DllImport instead of delegates in P/Invoke.

Posted by sb...@apache.org.
IGNITE-1694 : .NET: Use DllImport instead of delegates in P/Invoke.


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

Branch: refs/heads/ignite-1.5.1
Commit: 568f910f11472e6ab533c3f1cef6f197456e006f
Parents: efe632b
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Tue Dec 8 16:35:51 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Dec 8 16:35:51 2015 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |  10 -
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    | 358 +++++++++
 .../Impl/Unmanaged/UnmanagedUtils.cs            | 788 +++----------------
 modules/platforms/dotnet/Apache.Ignite.FxCop    | 322 +++++++-
 5 files changed, 798 insertions(+), 681 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/568f910f/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 2c7d787..831130c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -297,6 +297,7 @@
     <Compile Include="Impl\Transactions\TransactionImpl.cs" />
     <Compile Include="Impl\Transactions\TransactionMetricsImpl.cs" />
     <Compile Include="Impl\Transactions\TransactionsImpl.cs" />
+    <Compile Include="Impl\Unmanaged\IgniteJniNativeMethods.cs" />
     <Compile Include="Impl\Unmanaged\IUnmanagedTarget.cs" />
     <Compile Include="Impl\Unmanaged\UnmanagedCallbackHandlers.cs" />
     <Compile Include="Impl\Unmanaged\UnmanagedCallbacks.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/568f910f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
index 6e25e7e..eb3ba97 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
@@ -33,15 +33,5 @@ namespace Apache.Ignite.Core.Impl
         [DllImport("kernel32.dll", SetLastError = true, CharSet = CharSet.Ansi, BestFitMapping = false, 
             ThrowOnUnmappableChar = true)]
         internal static extern IntPtr LoadLibrary(string path);
-
-        /// <summary>
-        /// Get procedure address with WinAPI.
-        /// </summary>
-        /// <param name="ptr">DLL pointer.</param>
-        /// <param name="name">Procedure name.</param>
-        /// <returns>Procedure address.</returns>
-        [DllImport("kernel32.dll", SetLastError = true, CharSet = CharSet.Ansi, BestFitMapping = false, 
-            ThrowOnUnmappableChar = true)]
-        internal static extern IntPtr GetProcAddress(IntPtr ptr, string name);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/568f910f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
new file mode 100644
index 0000000..860e703
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
@@ -0,0 +1,358 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Unmanaged
+{
+    using System.Runtime.InteropServices;
+    using System.Security;
+
+    /// <summary>
+    /// Ignite JNI native methods.
+    /// </summary>
+    [SuppressUnmanagedCodeSecurity]
+    internal unsafe static class IgniteJniNativeMethods
+    {
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteReallocate")]
+        public static extern int Reallocate(long memPtr, int cap);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteIgnitionStart")]
+        public static extern void* IgnitionStart(void* ctx, sbyte* cfgPath, sbyte* gridName, int factoryId, 
+            long dataPtr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteIgnitionStop")]
+        [return: MarshalAs(UnmanagedType.U1)]
+        public static extern bool IgnitionStop(void* ctx, sbyte* gridName, [MarshalAs(UnmanagedType.U1)] bool cancel);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteIgnitionStopAll")]
+        public static extern void IgnitionStopAll(void* ctx, [MarshalAs(UnmanagedType.U1)] bool cancel);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorReleaseStart")]
+        public static extern void ProcessorReleaseStart(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorProjection")]
+        public static extern void* ProcessorProjection(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorCache")]
+        public static extern void* ProcessorCache(void* ctx, void* obj, sbyte* name);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorCreateCache")]
+        public static extern void* ProcessorCreateCache(void* ctx, void* obj, sbyte* name);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorGetOrCreateCache")]
+        public static extern void* ProcessorGetOrCreateCache(void* ctx, void* obj, sbyte* name);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorAffinity")]
+        public static extern void* ProcessorAffinity(void* ctx, void* obj, sbyte* name);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorDataStreamer")]
+        public static extern void* ProcessorDataStreamer(void* ctx, void* obj, sbyte* name, 
+            [MarshalAs(UnmanagedType.U1)] bool keepBinary);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorTransactions")]
+        public static extern void* ProcessorTransactions(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorCompute")]
+        public static extern void* ProcessorCompute(void* ctx, void* obj, void* prj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorMessage")]
+        public static extern void* ProcessorMessage(void* ctx, void* obj, void* prj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorEvents")]
+        public static extern void* ProcessorEvents(void* ctx, void* obj, void* prj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorServices")]
+        public static extern void* ProcessorServices(void* ctx, void* obj, void* prj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorExtensions")]
+        public static extern void* ProcessorExtensions(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorAtomicLong")]
+        public static extern void* ProcessorAtomicLong(void* ctx, void* obj, sbyte* name, long initVal,
+            [MarshalAs(UnmanagedType.U1)] bool create);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutLong")]
+        public static extern long TargetInStreamOutLong(void* ctx, void* target, int opType, long memPtr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutStream")]
+        public static extern void TargetInStreamOutStream(void* ctx, void* target, int opType, long inMemPtr,
+            long outMemPtr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutObject")]
+        public static extern void* TargetInStreanOutObject(void* ctx, void* target, int opType, long memPtr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInObjectStreamOutStream")]
+        public static extern void TargetInObjectStreamOutStream(void* ctx, void* target, int opType,
+            void* arg, long inMemPtr, long outMemPtr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetOutLong")]
+        public static extern long TargetOutLong(void* ctx, void* target, int opType);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetOutStream")]
+        public static extern void TargetOutStream(void* ctx, void* target, int opType, long memPtr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetOutObject")]
+        public static extern void* TargetOutObject(void* ctx, void* target, int opType);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetListenFuture")]
+        public static extern void TargetListenFut(void* ctx, void* target, long futId, int typ);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetListenFutureForOperation")]
+        public static extern void TargetListenFutForOp(void* ctx, void* target, long futId, int typ, int opId);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAffinityPartitions")]
+        public static extern int AffinityParts(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheWithSkipStore")]
+        public static extern void* CacheWithSkipStore(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheWithNoRetries")]
+        public static extern void* CacheWithNoRetries(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheWithExpiryPolicy")]
+        public static extern void* CacheWithExpiryPolicy(void* ctx, void* obj, long create, long update, long access);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheWithAsync")]
+        public static extern void* CacheWithAsync(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheWithKeepPortable")]
+        public static extern void* CacheWithKeepBinary(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheClear")]
+        public static extern void CacheClear(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheRemoveAll")]
+        public static extern void CacheRemoveAll(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheOutOpQueryCursor")]
+        public static extern void* CacheOutOpQueryCursor(void* ctx, void* obj, int type, long memPtr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheOutOpContinuousQuery")]
+        public static extern void* CacheOutOpContinuousQuery(void* ctx, void* obj, int type, long memPtr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheIterator")]
+        public static extern void* CacheIterator(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheLocalIterator")]
+        public static extern void* CacheLocalIterator(void* ctx, void* obj, int peekModes);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheEnterLock")]
+        public static extern void CacheEnterLock(void* ctx, void* obj, long id);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheExitLock")]
+        public static extern void CacheExitLock(void* ctx, void* obj, long id);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheTryEnterLock")]
+        [return: MarshalAs(UnmanagedType.U1)]
+        public static extern bool CacheTryEnterLock(void* ctx, void* obj, long id, long timeout);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheCloseLock")]
+        public static extern void CacheCloseLock(void* ctx, void* obj, long id);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheRebalance")]
+        public static extern void CacheRebalance(void* ctx, void* obj, long futId);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheSize")]
+        public static extern int CacheSize(void* ctx, void* obj, int peekModes, [MarshalAs(UnmanagedType.U1)] bool loc);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheStoreCallbackInvoke")]
+        public static extern void CacheStoreCallbackInvoke(void* ctx, void* obj, long memPtr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteComputeWithNoFailover")]
+        public static extern void ComputeWithNoFailover(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteComputeWithTimeout")]
+        public static extern void ComputeWithTimeout(void* ctx, void* target, long timeout);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteComputeExecuteNative")]
+        public static extern void ComputeExecuteNative(void* ctx, void* target, long taskPtr, long topVer);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteContinuousQueryClose")]
+        public static extern void ContinuousQryClose(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteContinuousQueryGetInitialQueryCursor")]
+        public static extern void* ContinuousQryGetInitialQueryCursor(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerListenTopology")]
+        public static extern void DataStreamerListenTop(void* ctx, void* obj, long ptr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerAllowOverwriteGet")]
+        [return: MarshalAs(UnmanagedType.U1)]
+        public static extern bool DataStreamerAllowOverwriteGet(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerAllowOverwriteSet")]
+        public static extern void DataStreamerAllowOverwriteSet(void* ctx, void* obj, 
+            [MarshalAs(UnmanagedType.U1)] bool val);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerSkipStoreGet")]
+        [return: MarshalAs(UnmanagedType.U1)]
+        public static extern bool DataStreamerSkipStoreGet(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerSkipStoreSet")]
+        public static extern void DataStreamerSkipStoreSet(void* ctx, void* obj, 
+            [MarshalAs(UnmanagedType.U1)] bool val);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerPerNodeBufferSizeGet")]
+        public static extern int DataStreamerPerNodeBufferSizeGet(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerPerNodeBufferSizeSet")]
+        public static extern void DataStreamerPerNodeBufferSizeSet(void* ctx, void* obj, int val);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerPerNodeParallelOperationsGet")]
+        public static extern int DataStreamerPerNodeParallelOpsGet(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerPerNodeParallelOperationsSet")]
+        public static extern void DataStreamerPerNodeParallelOpsSet(void* ctx, void* obj, int val);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteMessagingWithAsync")]
+        public static extern void* MessagingWithAsync(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForOthers")]
+        public static extern void* ProjectionForOthers(void* ctx, void* obj, void* prj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForRemotes")]
+        public static extern void* ProjectionForRemotes(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForDaemons")]
+        public static extern void* ProjectionForDaemons(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForRandom")]
+        public static extern void* ProjectionForRandom(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForOldest")]
+        public static extern void* ProjectionForOldest(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForYoungest")]
+        public static extern void* ProjectionForYoungest(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionResetMetrics")]
+        public static extern void ProjectionResetMetrics(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionOutOpRet")]
+        public static extern void* ProjectionOutOpRet(void* ctx, void* obj, int type, long memPtr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteQueryCursorIterator")]
+        public static extern void QryCursorIterator(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteQueryCursorClose")]
+        public static extern void QryCursorClose(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAcquire")]
+        public static extern void* Acquire(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteRelease")]
+        public static extern void Release(void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsStart")]
+        public static extern long TxStart(void* ctx, void* target, int concurrency, int isolation, long timeout,
+            int txSize);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsCommit")]
+        public static extern int TxCommit(void* ctx, void* target, long id);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsCommitAsync")]
+        public static extern void TxCommitAsync(void* ctx, void* target, long id, long futId);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsRollback")]
+        public static extern int TxRollback(void* ctx, void* target, long id);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsRollbackAsync")]
+        public static extern void TxRollbackAsync(void* ctx, void* target, long id, long futId);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsClose")]
+        public static extern int TxClose(void* ctx, void* target, long id);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsState")]
+        public static extern int TxState(void* ctx, void* target, long id);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsSetRollbackOnly")]
+        [return: MarshalAs(UnmanagedType.U1)]
+        public static extern bool TxSetRollbackOnly(void* ctx, void* target, long id);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsResetMetrics")]
+        public static extern void TxResetMetrics(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteThrowToJava")]
+        public static extern void ThrowToJava(void* ctx, char* msg);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteHandlersSize")]
+        public static extern int HandlersSize();
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCreateContext")]
+        public static extern void* CreateContext(void* opts, int optsLen, void* cbs);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDeleteContext")]
+        public static extern void DeleteContext(void* ptr);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDestroyJvm")]
+        public static extern void DestroyJvm(void* ctx);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteEventsWithAsync")]
+        public static extern void* EventsWithAsync(void* ctx, void* obj);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteEventsStopLocalListen")]
+        [return: MarshalAs(UnmanagedType.U1)]
+        public static extern bool EventsStopLocalListen(void* ctx, void* obj, long hnd);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteEventsLocalListen")]
+        public static extern void EventsLocalListen(void* ctx, void* obj, long hnd, int type);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteEventsIsEnabled")]
+        [return: MarshalAs(UnmanagedType.U1)]
+        public static extern bool EventsIsEnabled(void* ctx, void* obj, int type);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteServicesWithAsync")]
+        public static extern void* ServicesWithAsync(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteServicesWithServerKeepPortable")]
+        public static extern void* ServicesWithServerKeepBinary(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteServicesCancel")]
+        public static extern long ServicesCancel(void* ctx, void* target, char* name);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteServicesCancelAll")]
+        public static extern long ServicesCancelAll(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteServicesGetServiceProxy")]
+        public static extern void* ServicesGetServiceProxy(void* ctx, void* target, char* name,
+            [MarshalAs(UnmanagedType.U1)] bool sticky);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongGet")]
+        public static extern long AtomicLongGet(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongIncrementAndGet")]
+        public static extern long AtomicLongIncrementAndGet(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongAddAndGet")]
+        public static extern long AtomicLongAddAndGet(void* ctx, void* target, long value);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongDecrementAndGet")]
+        public static extern long AtomicLongDecrementAndGet(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongGetAndSet")]
+        public static extern long AtomicLongGetAndSet(void* ctx, void* target, long value);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongCompareAndSetAndGet")]
+        public static extern long AtomicLongCompareAndSetAndGet(void* ctx, void* target, long expVal, long newVal);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongIsClosed")]
+        [return: MarshalAs(UnmanagedType.U1)]
+        public static extern bool AtomicLongIsClosed(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongClose")]
+        public static extern void AtomicLongClose(void* ctx, void* target);
+    }
+}
\ No newline at end of file


[48/50] [abbrv] ignite git commit: ignite-2030 Fixed issue with JAVA_HOME containing parenthesis

Posted by sb...@apache.org.
ignite-2030  Fixed issue with JAVA_HOME containing parenthesis


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

Branch: refs/heads/ignite-1.5.1
Commit: 69047243bb1de980647711a59573bf44e69761eb
Parents: bdbe406
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 9 09:27:23 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 9 09:27:23 2015 +0300

----------------------------------------------------------------------
 bin/ignite.bat            | 5 +++--
 bin/include/parseargs.bat | 2 +-
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/69047243/bin/ignite.bat
----------------------------------------------------------------------
diff --git a/bin/ignite.bat b/bin/ignite.bat
index 43cc58b..97c2e45 100644
--- a/bin/ignite.bat
+++ b/bin/ignite.bat
@@ -20,6 +20,7 @@
 ::
 
 @echo off
+Setlocal EnableDelayedExpansion
 
 if "%OS%" == "Windows_NT"  setlocal
 
@@ -119,7 +120,7 @@ if %ERRORLEVEL% neq 0 (
 :: Process 'restart'.
 ::
 set RANDOM_NUMBER_COMMAND="%JAVA_HOME%\bin\java.exe" -cp %CP% org.apache.ignite.startup.cmdline.CommandLineRandomNumberGenerator
-for /f "usebackq tokens=*" %%i in (`"%RANDOM_NUMBER_COMMAND%"`) do set RANDOM_NUMBER=%%i
+for /f "usebackq tokens=*" %%i in (`%RANDOM_NUMBER_COMMAND%`) do set RANDOM_NUMBER=%%i
 
 set RESTART_SUCCESS_FILE="%IGNITE_HOME%\work\ignite_success_%RANDOM_NUMBER%"
 set RESTART_SUCCESS_OPT=-DIGNITE_SUCCESS_FILE=%RESTART_SUCCESS_FILE%
@@ -132,7 +133,7 @@ set RESTART_SUCCESS_OPT=-DIGNITE_SUCCESS_FILE=%RESTART_SUCCESS_FILE%
 :: This is executed if -nojmx is not specified
 ::
 if not "%NO_JMX%" == "1" (
-    for /F "tokens=*" %%A in ('""%JAVA_HOME%\bin\java" -cp %CP% org.apache.ignite.internal.util.portscanner.GridJmxPortFinder"') do (
+    for /F "tokens=*" %%A in ('""!JAVA_HOME!\bin\java" -cp %CP% org.apache.ignite.internal.util.portscanner.GridJmxPortFinder"') do (
         set JMX_PORT=%%A
     )
 )

http://git-wip-us.apache.org/repos/asf/ignite/blob/69047243/bin/include/parseargs.bat
----------------------------------------------------------------------
diff --git a/bin/include/parseargs.bat b/bin/include/parseargs.bat
index c4b35d2..86e5766 100644
--- a/bin/include/parseargs.bat
+++ b/bin/include/parseargs.bat
@@ -40,7 +40,7 @@
 :: in other scripts to parse common command lines parameters.
 
 set convertArgsCmd="%JAVA_HOME%\bin\java.exe" -cp %CP% org.apache.ignite.startup.cmdline.CommandLineTransformer %*
-for /f "usebackq tokens=*" %%i in (`"%convertArgsCmd%"`) do set reformattedArgs=%%i
+for /f "usebackq tokens=*" %%i in (`%convertArgsCmd%`) do set reformattedArgs=%%i
 
 for %%i in (%reformattedArgs%) do (
     if "%%i" == "CommandLineTransformerError" exit /b 1


[31/50] [abbrv] ignite git commit: 1.5.0-b2-SNAPSHOT

Posted by sb...@apache.org.
1.5.0-b2-SNAPSHOT


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

Branch: refs/heads/ignite-1.5.1
Commit: 3b26859bd64b7ddbc9d0b8094663887784b4f2cf
Parents: c077522
Author: Ignite Teamcity <ig...@apache.org>
Authored: Tue Dec 8 11:32:26 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Tue Dec 8 11:32:26 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                               | 2 +-
 examples/schema-import/pom.xml                                 | 2 +-
 modules/aop/pom.xml                                            | 2 +-
 modules/apache-license-gen/pom.xml                             | 2 +-
 modules/aws/pom.xml                                            | 2 +-
 modules/camel/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/flume/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/jms11/pom.xml                                          | 2 +-
 modules/jta/pom.xml                                            | 2 +-
 modules/kafka/pom.xml                                          | 2 +-
 modules/log4j/pom.xml                                          | 2 +-
 modules/log4j2/pom.xml                                         | 2 +-
 modules/mesos/pom.xml                                          | 2 +-
 modules/mqtt/pom.xml                                           | 2 +-
 modules/osgi-karaf/pom.xml                                     | 5 ++---
 modules/osgi-paxlogging/pom.xml                                | 6 ++----
 modules/osgi/pom.xml                                           | 5 ++---
 modules/platforms/cpp/common/configure.ac                      | 2 +-
 modules/platforms/cpp/core-test/configure.ac                   | 2 +-
 modules/platforms/cpp/core/configure.ac                        | 2 +-
 modules/platforms/cpp/examples/configure.ac                    | 2 +-
 modules/platforms/cpp/ignite/configure.ac                      | 2 +-
 .../dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs | 4 ++--
 .../Properties/AssemblyInfo.cs                                 | 4 ++--
 .../dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs | 4 ++--
 .../dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs       | 4 ++--
 .../platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs  | 4 ++--
 .../examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs | 4 ++--
 .../Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs       | 4 ++--
 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/twitter/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 +-
 modules/zookeeper/pom.xml                                      | 2 +-
 pom.xml                                                        | 6 +++---
 64 files changed, 76 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 33c4f51..41a5b6f 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/examples/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/examples/schema-import/pom.xml b/examples/schema-import/pom.xml
index d315c63..40f20a3 100644
--- a/examples/schema-import/pom.xml
+++ b/examples/schema-import/pom.xml
@@ -35,7 +35,7 @@
     </properties>
 
     <artifactId>ignite-schema-import-demo</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index 8598863..68a6c4b 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index 6d3c985..fac37b1 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -31,7 +31,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 68bb4c8..b3a4286 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/camel/pom.xml
----------------------------------------------------------------------
diff --git a/modules/camel/pom.xml b/modules/camel/pom.xml
index e1c20f2..7b73f1d 100644
--- a/modules/camel/pom.xml
+++ b/modules/camel/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-camel</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 5ab61d4..b0f411f 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 783fa68..f81db04 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 2a58e3f..c9d0be7 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 88e1c25..c3d34ef 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <repositories>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/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 edf7d3c..073c9ab 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.5.0-b1-SNAPSHOT
+ignite.version=1.5.0-b2-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index de8eed5..26d7d06 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.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/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 972f61a..3eea056 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.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <modelVersion>4.0.0</modelVersion>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index 1176c85..118a045 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.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/flume/pom.xml
----------------------------------------------------------------------
diff --git a/modules/flume/pom.xml b/modules/flume/pom.xml
index 8598248..871ca1b 100644
--- a/modules/flume/pom.xml
+++ b/modules/flume/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-flume</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index edab865..f9d79f4 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index 229ecef..69eb9a0 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index df39996..00b18e7 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index 766ca97..01cd464 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index e7a1c3b..d108842 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 8678caa..7c95748 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/jms11/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jms11/pom.xml b/modules/jms11/pom.xml
index 2e2ffe6..030b546 100644
--- a/modules/jms11/pom.xml
+++ b/modules/jms11/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jms11</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index 9dabe28..e296940 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index 443d73c..97df51c 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-kafka</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index e27732e..e22d50c 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/log4j2/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j2/pom.xml b/modules/log4j2/pom.xml
index e195ce7..21fb4b3 100644
--- a/modules/log4j2/pom.xml
+++ b/modules/log4j2/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j2</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index ee6be82..1901ec6 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/mqtt/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mqtt/pom.xml b/modules/mqtt/pom.xml
index c0b17e6..24c98b9 100644
--- a/modules/mqtt/pom.xml
+++ b/modules/mqtt/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mqtt</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/osgi-karaf/pom.xml
----------------------------------------------------------------------
diff --git a/modules/osgi-karaf/pom.xml b/modules/osgi-karaf/pom.xml
index e1f53e2..68e0c12 100644
--- a/modules/osgi-karaf/pom.xml
+++ b/modules/osgi-karaf/pom.xml
@@ -20,8 +20,7 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -32,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-osgi-karaf</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/osgi-paxlogging/pom.xml
----------------------------------------------------------------------
diff --git a/modules/osgi-paxlogging/pom.xml b/modules/osgi-paxlogging/pom.xml
index 15a5e0c..7695349 100644
--- a/modules/osgi-paxlogging/pom.xml
+++ b/modules/osgi-paxlogging/pom.xml
@@ -20,9 +20,7 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -33,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-osgi-paxlogging</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <packaging>jar</packaging>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/osgi/pom.xml
----------------------------------------------------------------------
diff --git a/modules/osgi/pom.xml b/modules/osgi/pom.xml
index 47c7928..ade8fd6 100644
--- a/modules/osgi/pom.xml
+++ b/modules/osgi/pom.xml
@@ -20,8 +20,7 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -32,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-osgi</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/cpp/common/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/configure.ac b/modules/platforms/cpp/common/configure.ac
index 8f53ea3..9dfb0a3 100644
--- a/modules/platforms/cpp/common/configure.ac
+++ b/modules/platforms/cpp/common/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite JNI bridge for C++], [1.5.0-EA], [dev@ignite.apache.org], [ignite-common], [ignite.apache.org])
+AC_INIT([Apache Ignite JNI bridge for C++], [1.5.0.2], [dev@ignite.apache.org], [ignite-common], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/cpp/core-test/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/configure.ac b/modules/platforms/cpp/core-test/configure.ac
index 14baf55..2a3e125 100644
--- a/modules/platforms/cpp/core-test/configure.ac
+++ b/modules/platforms/cpp/core-test/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite C++ Test], [1.5.0-EA], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
+AC_INIT([Apache Ignite C++ Test], [1.5.0.2], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/cpp/core/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/configure.ac b/modules/platforms/cpp/core/configure.ac
index f10eeb9..368fcac 100644
--- a/modules/platforms/cpp/core/configure.ac
+++ b/modules/platforms/cpp/core/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite C++], [1.5.0-EA], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
+AC_INIT([Apache Ignite C++], [1.5.0.2], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/cpp/examples/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/configure.ac b/modules/platforms/cpp/examples/configure.ac
index 2d30028..a75e5f3 100644
--- a/modules/platforms/cpp/examples/configure.ac
+++ b/modules/platforms/cpp/examples/configure.ac
@@ -2,7 +2,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Ingnite C++ examples],[1.5.0-EA],[dec@ignite.apache.org],[ignite-examples],[ignite.apache.org])
+AC_INIT([Ingnite C++ examples],[1.5.0.2],[dec@ignite.apache.org],[ignite-examples],[ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/cpp/ignite/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/ignite/configure.ac b/modules/platforms/cpp/ignite/configure.ac
index 41d944c..3a85d9d 100644
--- a/modules/platforms/cpp/ignite/configure.ac
+++ b/modules/platforms/cpp/ignite/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite C++ Runner], [1.5.0-EA], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
+AC_INIT([Apache Ignite C++ Runner], [1.5.0.2], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
index db3e260..3318641 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
@@ -31,5 +31,5 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("8fae8395-7e91-411a-a78f-44d6d3fed0fc")]
 
-[assembly: AssemblyVersion("1.5.0.0")]
-[assembly: AssemblyFileVersion("1.5.0.0")]
+[assembly: AssemblyVersion("1.5.0.2")]
+[assembly: AssemblyFileVersion("1.5.0.2")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
index 22776d3..88b7a23 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
@@ -45,5 +45,5 @@ using System.Runtime.InteropServices;
 // You can specify all the values or you can default the Build and Revision Numbers
 // by using the '*' as shown below:
 // [assembly: AssemblyVersion("1.0.*")]
-[assembly: AssemblyVersion("1.5.0.0")]
-[assembly: AssemblyFileVersion("1.5.0.0")]
+[assembly: AssemblyVersion("1.5.0.2")]
+[assembly: AssemblyFileVersion("1.5.0.2")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
index 979ccb8..9d32327 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
@@ -31,5 +31,5 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("de8dd5cc-7c7f-4a09-80d5-7086d9416a7b")]
 
-[assembly: AssemblyVersion("1.5.0.0")]
-[assembly: AssemblyFileVersion("1.5.0.0")]
+[assembly: AssemblyVersion("1.5.0.2")]
+[assembly: AssemblyFileVersion("1.5.0.2")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
index 0fbbdc1..4fb0458 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
@@ -33,8 +33,8 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("97db45a8-f922-456a-a819-7b3c6e5e03ba")]
 
-[assembly: AssemblyVersion("1.5.0.0")]
-[assembly: AssemblyFileVersion("1.5.0.0")]
+[assembly: AssemblyVersion("1.5.0.2")]
+[assembly: AssemblyFileVersion("1.5.0.2")]
 
 [assembly: CLSCompliant(true)]
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
index bee163c..bd2d2ad 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
@@ -31,5 +31,5 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("0f9702ec-da7d-4ce5-b4b7-73310c885355")]
 
-[assembly: AssemblyVersion("1.5.0.0")]
-[assembly: AssemblyFileVersion("1.5.0.0")]
+[assembly: AssemblyVersion("1.5.0.2")]
+[assembly: AssemblyFileVersion("1.5.0.2")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
index bc75631..082dde2 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
@@ -31,5 +31,5 @@
 
 [assembly: Guid("41a0cb95-3435-4c78-b867-900b28e2c9ee")]
 
-[assembly: AssemblyVersion("1.5.0.0")]
-[assembly: AssemblyFileVersion("1.5.0.0")]
+[assembly: AssemblyVersion("1.5.0.2")]
+[assembly: AssemblyFileVersion("1.5.0.2")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
index ab0af87..cc08601 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
@@ -31,5 +31,5 @@ using System.Reflection;
 
 [assembly: Guid("ce65ec7c-d3cf-41ad-8f45-f90d5af68d77")]
 
-[assembly: AssemblyVersion("1.5.0.0")]
-[assembly: AssemblyFileVersion("1.5.0.0")]
+[assembly: AssemblyVersion("1.5.0.2")]
+[assembly: AssemblyFileVersion("1.5.0.2")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 0780144..6d5166a 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.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 9a958eb..c75880d 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.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 301a103..cdabc4b 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index cb53713..1435121 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 36f6b13..3fe9ad3 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.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 1449549..27b4cda 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 0b24898..faeb45f 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.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index cb56d2c..780397c 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 1808a0a..f1b9bc1 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index e1b0fe5..70091e7 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 86adea3..c259cce 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/twitter/pom.xml
----------------------------------------------------------------------
diff --git a/modules/twitter/pom.xml b/modules/twitter/pom.xml
index 1649043..60e415f 100644
--- a/modules/twitter/pom.xml
+++ b/modules/twitter/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-twitter</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 37c77ba..cf51bfa 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/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 87ce699..a4e50cb 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.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index d4fc6cf..f686238 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.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index 7754a4b..4d63db7 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.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index d25ce61..401a1cc 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 185dea0..8c3e4c1 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index 8f9babe..a726321 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yarn</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/modules/zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml
index 8ade247..6a5bc70 100644
--- a/modules/zookeeper/pom.xml
+++ b/modules/zookeeper/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-zookeeper</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b26859b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 993eab2..97d5f6f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.5.0-b1-SNAPSHOT</version>
+    <version>1.5.0-b2-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>
@@ -781,7 +781,7 @@
 
                                                 var pos = p.search("-");
 
-                                                if (pos > 0)
+                                                if (pos &gt; 0)
                                                 {
                                                     var suffix = p.substring(pos);
 
@@ -795,7 +795,7 @@
                                                     if (patch !== null)
                                                         ver += parseInt(patch[1]) * 100;
 
-                                                    if (suffix.search("final") > 0)
+                                                    if (suffix.search("final") &gt; 0)
                                                         ver += 10000;
 
                                                     var resVer = p.substring(0, pos) +"." + ver;


[36/50] [abbrv] ignite git commit: Ignite-1.5 - Added missing serialVersionUID to fix the build.

Posted by sb...@apache.org.
Ignite-1.5 - Added missing serialVersionUID to fix the build.


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

Branch: refs/heads/ignite-1.5.1
Commit: 67ebd02c9a58ef2d835e55e4aa6efdcec6d53b8c
Parents: c292748
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Dec 8 15:09:10 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Dec 8 15:09:10 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/util/GridMessageCollection.java    | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/67ebd02c/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
index 09a5e96..6044bbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
@@ -34,6 +34,9 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
  */
 public final class GridMessageCollection<M extends Message> implements Message {
     /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
     @GridDirectCollection(Message.class)
     private Collection<M> msgs;
 


[29/50] [abbrv] ignite git commit: IGNITE-2050 - Fixed duplicate query types in configuration. - Fixes #291.

Posted by sb...@apache.org.
IGNITE-2050 - Fixed duplicate query types in configuration. - Fixes #291.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-1.5.1
Commit: ebddcb187231c08bcf53a7c567f10064393a8e60
Parents: 9a14d64
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Dec 7 20:11:48 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Dec 7 20:11:48 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  34 ++++-
 ...cheDuplicateEntityConfigurationSelfTest.java | 126 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +
 3 files changed, 159 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ebddcb18/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 034d20d..af3530a 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -1872,7 +1872,20 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
             TypeDescriptor desc = processKeyAndValueClasses(keyCls, valCls);
 
-            qryEntities.add(convert(desc));
+            QueryEntity converted = convert(desc);
+
+            boolean dup = false;
+
+            for (QueryEntity entity : qryEntities) {
+                if (F.eq(entity.getValueType(), converted.getValueType())) {
+                    dup = true;
+
+                    break;
+                }
+            }
+
+            if (!dup)
+                qryEntities.add(converted);
         }
 
         return this;
@@ -1968,10 +1981,21 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     public CacheConfiguration<K, V> setQueryEntities(Collection<QueryEntity> qryEntities) {
         if (this.qryEntities == null)
             this.qryEntities = new ArrayList<>(qryEntities);
-        else if (indexedTypes != null)
-            this.qryEntities.addAll(qryEntities);
-        else
-            throw new CacheException("Query entities can be set only once.");
+
+        for (QueryEntity entity : qryEntities) {
+            boolean found = false;
+
+            for (QueryEntity existing : this.qryEntities) {
+                if (F.eq(entity.getValueType(), existing.getValueType())) {
+                    found = true;
+
+                    break;
+                }
+            }
+
+            if (!found)
+                this.qryEntities.add(entity);
+        }
 
         return this;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebddcb18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDuplicateEntityConfigurationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDuplicateEntityConfigurationSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDuplicateEntityConfigurationSelfTest.java
new file mode 100644
index 0000000..8311213
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDuplicateEntityConfigurationSelfTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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 java.util.Arrays;
+import java.util.LinkedHashMap;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteCacheDuplicateEntityConfigurationSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(disco);
+
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassDuplicatesQueryEntity() throws Exception {
+        String cacheName = "duplicate";
+
+        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
+
+        ccfg.setIndexedTypes(Integer.class, Person.class);
+
+        QueryEntity entity = new QueryEntity();
+
+        entity.setKeyType(Integer.class.getName());
+        entity.setValueType(Person.class.getName());
+
+        LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+
+        fields.put("name", String.class.getName());
+
+        entity.setFields(fields);
+
+        ccfg.setQueryEntities(Arrays.asList(entity));
+
+        try {
+            ignite(0).getOrCreateCache(ccfg);
+        }
+        finally {
+            ignite(0).destroyCache(cacheName);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassDuplicatesQueryReverse() throws Exception {
+        String cacheName = "duplicate";
+
+        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
+
+        QueryEntity entity = new QueryEntity();
+
+        entity.setKeyType(Integer.class.getName());
+        entity.setValueType(Person.class.getName());
+
+        LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+
+        fields.put("name", String.class.getName());
+
+        entity.setFields(fields);
+
+        ccfg.setQueryEntities(Arrays.asList(entity));
+
+        ccfg.setIndexedTypes(Integer.class, Person.class);
+
+        try {
+            ignite(0).getOrCreateCache(ccfg);
+        }
+        finally {
+            ignite(0).destroyCache(cacheName);
+        }
+    }
+
+    private static class Person {
+        @QuerySqlField
+        private String name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebddcb18/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 8311414..fc88c75 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSel
 import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest;
@@ -111,6 +112,9 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         // Parsing
         suite.addTestSuite(GridQueryParsingTest.class);
 
+        // Config.
+        suite.addTestSuite(IgniteCacheDuplicateEntityConfigurationSelfTest.class);
+
         // Queries tests.
         suite.addTestSuite(IgniteSqlSplitterSelfTest.class);
         suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class);


[05/50] [abbrv] ignite git commit: ignite-2042 Added special queue/set key classes to make collocation work with BinaryMarshaller. Also fixed issue with 'invoke' result with binary marshaller.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
index c44455f..519d3c1 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
@@ -65,6 +65,7 @@ import org.apache.ignite.internal.processors.cache.datastructures.partitioned.Gr
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedSetSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedAtomicLongApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedCountDownLatchSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedQueueNoBackupsTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedSetNoBackupsSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedAtomicReferenceApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedAtomicStampedApiSelfTest;
@@ -165,6 +166,8 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgniteClientDataStructuresTest.class));
         suite.addTest(new TestSuite(IgniteClientDiscoveryDataStructuresTest.class));
 
+        suite.addTest(new TestSuite(IgnitePartitionedQueueNoBackupsTest.class));
+
         return suite;
     }
 }
\ No newline at end of file


[34/50] [abbrv] ignite git commit: ignite-1.5 - MessageCollection + marshalling issue test

Posted by sb...@apache.org.
ignite-1.5 - MessageCollection + marshalling issue test


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

Branch: refs/heads/ignite-1.5.1
Commit: 322a85a359e0fc2c56f5c3aa38fc48a92e553289
Parents: c077522
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 8 13:17:36 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 8 13:17:36 2015 +0300

----------------------------------------------------------------------
 .../ignite/codegen/MessageCodeGenerator.java    |   6 +-
 .../communication/GridIoMessageFactory.java     |   8 +-
 .../internal/util/GridMessageCollection.java    | 157 +++++++++++++++++++
 .../internal/util/UUIDCollectionMessage.java    |  42 ++++-
 .../testsuites/IgniteUtilSelfTestSuite.java     |   2 +
 .../ignite/util/GridMessageCollectionTest.java  | 105 +++++++++++++
 6 files changed, 314 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/322a85a3/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
index 8733bb3..587ad06 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
@@ -43,9 +43,6 @@ import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.IgniteCodeGeneratingFail;
-import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEntry;
-import org.apache.ignite.internal.util.UUIDCollectionMessage;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -170,8 +167,9 @@ public class MessageCodeGenerator {
 
 //        gen.generateAll(true);
 
-        gen.generateAndWrite(GridNearAtomicUpdateRequest.class);
+//        gen.generateAndWrite(GridNearAtomicUpdateRequest.class);
 
+//        gen.generateAndWrite(GridMessageCollection.class);
 //        gen.generateAndWrite(DataStreamerEntry.class);
 
 //        gen.generateAndWrite(GridDistributedLockRequest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/322a85a3/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index c9b8e27..b0ef3f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -125,6 +125,7 @@ import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRe
 import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultResponse;
 import org.apache.ignite.internal.util.GridByteArrayList;
 import org.apache.ignite.internal.util.GridLongList;
+import org.apache.ignite.internal.util.GridMessageCollection;
 import org.apache.ignite.internal.util.UUIDCollectionMessage;
 import org.apache.ignite.lang.IgniteOutClosure;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -720,7 +721,12 @@ public class GridIoMessageFactory implements MessageFactory {
 
                 break;
 
-            // [-3..119] - this
+            case 124:
+                msg = new GridMessageCollection<>();
+
+                break;
+
+            // [-3..119] [124] - this
             // [120..123] - DR
             // [-4..-22] - SQL
             default:

http://git-wip-us.apache.org/repos/asf/ignite/blob/322a85a3/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
new file mode 100644
index 0000000..09a5e96
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ * Collection of messages.
+ */
+public final class GridMessageCollection<M extends Message> implements Message {
+    /** */
+    @GridDirectCollection(Message.class)
+    private Collection<M> msgs;
+
+    /**
+     *
+     */
+    public GridMessageCollection() {
+        // No-op.
+    }
+
+    /**
+     * @param msgs Collection of messages.
+     */
+    public GridMessageCollection(Collection<M> msgs) {
+        this.msgs = msgs;
+    }
+
+    /**
+     * @param msgs Messages.
+     * @return Message list.
+     */
+    public static <X extends Message> GridMessageCollection<X> of(X... msgs) {
+        if (msgs == null || msgs.length == 0)
+            return null;
+
+        List<X> list = msgs.length == 1 ? Collections.singletonList(msgs[0]) : Arrays.asList(msgs);
+
+        return new GridMessageCollection<>(list);
+    }
+
+    /**
+     * @return Messages.
+     */
+    public Collection<M> messages() {
+        return msgs;
+    }
+
+    /**
+     * @param msgs Messages.
+     */
+    public void messages(Collection<M> msgs) {
+        this.msgs = msgs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeCollection("msgs", msgs, MessageCollectionItemType.MSG))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                msgs = reader.readCollection("msgs", MessageCollectionItemType.MSG);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(GridMessageCollection.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return 124;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        GridMessageCollection<?> that = (GridMessageCollection<?>)o;
+
+        return msgs == that.msgs || (msgs != null && msgs.equals(that.msgs));
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return msgs != null ? msgs.hashCode() : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridMessageCollection.class, this, "msgsSize", msgs == null ? null : msgs.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/322a85a3/modules/core/src/main/java/org/apache/ignite/internal/util/UUIDCollectionMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/UUIDCollectionMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/UUIDCollectionMessage.java
index 25e3376..33bce53 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/UUIDCollectionMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/UUIDCollectionMessage.java
@@ -18,16 +18,20 @@
 package org.apache.ignite.internal.util;
 
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 
 /**
- *
+ * Collection of UUIDs.
  */
 public class UUIDCollectionMessage implements Message {
     /** */
@@ -52,6 +56,19 @@ public class UUIDCollectionMessage implements Message {
     }
 
     /**
+     * @param uuids UUIDs.
+     * @return Message.
+     */
+    public static UUIDCollectionMessage of(UUID... uuids) {
+        if (uuids == null || uuids.length == 0)
+            return null;
+
+        List<UUID> list = uuids.length == 1 ? Collections.singletonList(uuids[0]) : Arrays.asList(uuids);
+
+        return new UUIDCollectionMessage(list);
+    }
+
+    /**
      * @return The collection of UUIDs that was wrapped.
      */
     public Collection<UUID> uuids() {
@@ -111,4 +128,27 @@ public class UUIDCollectionMessage implements Message {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        UUIDCollectionMessage that = (UUIDCollectionMessage)o;
+
+        return uuids == that.uuids || (uuids != null && uuids.equals(that.uuids));
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return uuids != null ? uuids.hashCode() : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(UUIDCollectionMessage.class, this, "uuidsSize", uuids == null ? null : uuids.size());
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/322a85a3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
index ac3241d..bd7bb96 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
@@ -37,6 +37,7 @@ import org.apache.ignite.spi.discovery.ClusterMetricsSnapshotSerializeSelfTest;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.thread.GridThreadPoolExecutorServiceSelfTest;
 import org.apache.ignite.util.GridLongListSelfTest;
+import org.apache.ignite.util.GridMessageCollectionTest;
 import org.apache.ignite.util.GridQueueSelfTest;
 import org.apache.ignite.util.GridSpinReadWriteLockSelfTest;
 import org.apache.ignite.util.GridStringBuilderFactorySelfTest;
@@ -73,6 +74,7 @@ public class IgniteUtilSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridLongListSelfTest.class);
         suite.addTestSuite(GridCacheUtilsSelfTest.class);
         suite.addTestSuite(IgniteExceptionRegistrySelfTest.class);
+        suite.addTestSuite(GridMessageCollectionTest.class);
 
         // Metrics.
         suite.addTestSuite(ClusterMetricsSnapshotSerializeSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/322a85a3/modules/core/src/test/java/org/apache/ignite/util/GridMessageCollectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridMessageCollectionTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridMessageCollectionTest.java
new file mode 100644
index 0000000..e910a8a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridMessageCollectionTest.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.util;
+
+import java.nio.ByteBuffer;
+import junit.framework.TestCase;
+import org.apache.ignite.internal.direct.DirectMessageReader;
+import org.apache.ignite.internal.direct.DirectMessageWriter;
+import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
+import org.apache.ignite.internal.util.UUIDCollectionMessage;
+import org.apache.ignite.plugin.extensions.communication.Message;
+
+import static java.util.UUID.randomUUID;
+import static org.apache.ignite.internal.util.GridMessageCollection.of;
+
+/**
+ *
+ */
+public class GridMessageCollectionTest extends TestCase {
+    /** */
+    private byte proto;
+
+    /**
+     *
+     */
+    public void testMarshal() {
+        UUIDCollectionMessage um0 = UUIDCollectionMessage.of();
+        UUIDCollectionMessage um1 = UUIDCollectionMessage.of(randomUUID());
+        UUIDCollectionMessage um2 = UUIDCollectionMessage.of(randomUUID(), randomUUID());
+        UUIDCollectionMessage um3 = UUIDCollectionMessage.of(randomUUID(), randomUUID(), randomUUID());
+
+        assertNull(um0);
+        assertEquals(3, um3.uuids().size());
+
+        proto = 2;
+        doTestMarshal(um0, um1, um2, um3);
+
+        proto = 1;
+        doTestMarshal(um0, um1, um2, um3);
+    }
+
+    /**
+     * @param um0 Null.
+     * @param um1 One uuid list.
+     * @param um2 Two uuid list.
+     * @param um3 Three uuid list.
+     */
+    private void doTestMarshal(
+        UUIDCollectionMessage um0,
+        UUIDCollectionMessage um1,
+        UUIDCollectionMessage um2,
+        UUIDCollectionMessage um3
+    ) {
+        doTestMarshal(um1);
+        doTestMarshal(um2);
+        doTestMarshal(um3);
+
+        doTestMarshal(of(um0));
+        doTestMarshal(of(um1));
+        doTestMarshal(of(um2));
+        doTestMarshal(of(um3));
+
+        doTestMarshal(of(um2, um3));
+        doTestMarshal(of(um1, um0, um3));
+
+        doTestMarshal(of(of(um3), of(um2)));
+        doTestMarshal(of(of(of(of(of(um0))), um1, of(um3))));
+    }
+
+    /**
+     * @param m Message.
+     */
+    private void doTestMarshal(Message m) {
+        ByteBuffer buf = ByteBuffer.allocate(8 * 1024);
+
+        DirectMessageWriter w = new DirectMessageWriter(proto);
+
+        m.writeTo(buf, w);
+
+        buf.flip();
+
+        DirectMessageReader r = new DirectMessageReader(new GridIoMessageFactory(null), proto);
+
+        r.setBuffer(buf);
+
+        Message mx = r.readMessage(null);
+
+        assertEquals(m, mx);
+    }
+}


[35/50] [abbrv] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

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


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

Branch: refs/heads/ignite-1.5.1
Commit: c292748013acfcbf6b3752183a34bb26de16c1f4
Parents: 322a85a 5cd0039
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 8 13:18:30 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 8 13:18:30 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                |  2 +-
 examples/schema-import/pom.xml                  |  2 +-
 modules/aop/pom.xml                             |  2 +-
 modules/apache-license-gen/pom.xml              |  2 +-
 modules/aws/pom.xml                             |  2 +-
 modules/camel/pom.xml                           |  2 +-
 modules/clients/pom.xml                         |  2 +-
 modules/cloud/pom.xml                           |  2 +-
 modules/codegen/pom.xml                         |  2 +-
 modules/core/pom.xml                            |  2 +-
 .../apache/ignite/internal/GridComponent.java   |  3 +-
 .../ignite/internal/GridPluginComponent.java    |  4 +-
 .../apache/ignite/internal/IgniteKernal.java    | 18 +++-
 .../internal/managers/GridManagerAdapter.java   |  5 +-
 .../deployment/GridDeploymentManager.java       |  5 +-
 .../processors/GridProcessorAdapter.java        |  5 +-
 .../processors/cache/GridCacheContext.java      |  6 +-
 .../processors/cache/GridCacheProcessor.java    | 26 +++++-
 .../datastructures/DataStructuresProcessor.java |  4 +-
 .../core/src/main/resources/ignite.properties   |  2 +-
 .../IgniteClientReconnectAbstractTest.java      | 95 +++++++++++++++++---
 .../IgniteClientReconnectAtomicsTest.java       | 57 ++++++++++++
 .../IgniteClientReconnectCacheTest.java         |  5 +-
 .../IgniteClientReconnectCollectionsTest.java   | 51 +++++++++++
 .../DataStreamProcessorSelfTest.java            | 25 +++---
 modules/extdata/p2p/pom.xml                     |  2 +-
 .../extdata/uri/modules/uri-dependency/pom.xml  |  2 +-
 modules/extdata/uri/pom.xml                     |  2 +-
 modules/flume/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/jms11/pom.xml                           |  2 +-
 modules/jta/pom.xml                             |  2 +-
 modules/kafka/pom.xml                           |  2 +-
 modules/log4j/pom.xml                           |  2 +-
 modules/log4j2/pom.xml                          |  2 +-
 modules/mesos/pom.xml                           |  2 +-
 modules/mqtt/pom.xml                            |  2 +-
 modules/osgi-karaf/pom.xml                      |  5 +-
 modules/osgi-paxlogging/pom.xml                 |  6 +-
 modules/osgi/pom.xml                            |  5 +-
 modules/platforms/cpp/common/configure.ac       |  2 +-
 modules/platforms/cpp/core-test/configure.ac    |  2 +-
 modules/platforms/cpp/core/configure.ac         |  2 +-
 modules/platforms/cpp/examples/configure.ac     |  2 +-
 modules/platforms/cpp/ignite/configure.ac       |  2 +-
 .../Properties/AssemblyInfo.cs                  |  4 +-
 .../Properties/AssemblyInfo.cs                  |  4 +-
 .../Properties/AssemblyInfo.cs                  |  4 +-
 .../Properties/AssemblyInfo.cs                  |  4 +-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |  4 +-
 .../Properties/AssemblyInfo.cs                  |  4 +-
 .../Properties/AssemblyInfo.cs                  |  4 +-
 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/twitter/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 +-
 modules/zookeeper/pom.xml                       |  2 +-
 pom.xml                                         |  6 +-
 78 files changed, 345 insertions(+), 120 deletions(-)
----------------------------------------------------------------------



[50/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1.5' into ignite-1.5.1

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1.5' into ignite-1.5.1

Conflicts:
	benchmarks/cache-comparison/pom.xml
	benchmarks/filesystem/hadoop1/pom.xml
	benchmarks/filesystem/hadoop2/pom.xml
	benchmarks/mongo/pom.xml
	benchmarks/risk-analytics/pom.xml
	benchmarks/serialization/pom.xml
	benchmarks/yardstick/src/main/dotnet/gridgain-benchmarks/Properties/AssemblyInfo.cs
	examples/pom.xml
	modules/clients/pom.xml
	modules/codegen/pom.xml
	modules/compatibility/pom.xml
	modules/core/pom.xml
	modules/core/src/main/resources/gridgain.properties
	modules/diagnostic/pom.xml
	modules/dr-demo/pom.xml
	modules/license-gen/pom.xml
	modules/mongo-sniffer/pom.xml
	modules/mongo-visor/pom.xml
	modules/mongo/pom.xml
	modules/platforms/cpp/core-test/configure.ac
	modules/platforms/cpp/core/configure.ac
	modules/platforms/cpp/examples/configure.ac
	modules/platforms/dotnet/examples/GridGainExamples/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/examples/GridGainExamplesDll/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/gridgain-examples-test/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/gridgain-exe/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/gridgain-test-compatibility/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/gridgain-test-dll/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/gridgain-test-runner/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/gridgain-test/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/gridgain/Properties/AssemblyInfo.cs
	modules/tools/pom.xml
	modules/visor-console/pom.xml
	modules/visor-demo/pom.xml
	modules/visor-tester-plugin/pom.xml
	modules/visor-tester/pom.xml
	modules/visor/pom.xml
	modules/yardstick/dotnet/Properties/AssemblyInfo.cs
	modules/yardstick/pom.xml
	parent/pom.xml
	pilots/chronotrack/pom.xml
	pilots/ctb/pom.xml
	pilots/dsi/pom.xml
	pilots/ionic/pom.xml
	pilots/sb/pom.xml
	pilots/sony/pom.xml
	pilots/wellsfargo/pom.xml
	pilots/worldpay/pom.xml
	pom.xml

Merge remote-tracking branch 'remotes/origin/ignite-1.5' into ignite-1.5.1

Conflicts:
	examples/pom.xml
	examples/schema-import/pom.xml
	modules/aop/pom.xml
	modules/apache-license-gen/pom.xml
	modules/aws/pom.xml
	modules/camel/pom.xml
	modules/clients/pom.xml
	modules/cloud/pom.xml
	modules/codegen/pom.xml
	modules/core/pom.xml
	modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldAccessor.java
	modules/core/src/main/resources/ignite.properties
	modules/extdata/p2p/pom.xml
	modules/extdata/uri/modules/uri-dependency/pom.xml
	modules/extdata/uri/pom.xml
	modules/flume/pom.xml
	modules/gce/pom.xml
	modules/geospatial/pom.xml
	modules/hadoop/pom.xml
	modules/hibernate/pom.xml
	modules/indexing/pom.xml
	modules/jcl/pom.xml
	modules/jms11/pom.xml
	modules/jta/pom.xml
	modules/kafka/pom.xml
	modules/log4j/pom.xml
	modules/log4j2/pom.xml
	modules/mesos/pom.xml
	modules/mqtt/pom.xml
	modules/platforms/cpp/common/configure.ac
	modules/platforms/cpp/core-test/configure.ac
	modules/platforms/cpp/core/configure.ac
	modules/platforms/cpp/examples/configure.ac
	modules/platforms/cpp/ignite/configure.ac
	modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
	modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
	modules/rest-http/pom.xml
	modules/scalar-2.10/pom.xml
	modules/scalar/pom.xml
	modules/schedule/pom.xml
	modules/schema-import/pom.xml
	modules/slf4j/pom.xml
	modules/spark-2.10/pom.xml
	modules/spark/pom.xml
	modules/spring/pom.xml
	modules/ssh/pom.xml
	modules/tools/pom.xml
	modules/twitter/pom.xml
	modules/urideploy/pom.xml
	modules/visor-console-2.10/pom.xml
	modules/visor-console/pom.xml
	modules/visor-plugins/pom.xml
	modules/web/pom.xml
	modules/yardstick/pom.xml
	modules/yarn/pom.xml
	modules/zookeeper/pom.xml
	pom.xml


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

Branch: refs/heads/ignite-1.5.1
Commit: ebb4aa63b0866699513d75f51efdfbb34980cf83
Parents: 250aa4f 0adee3a
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Dec 9 10:35:43 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Dec 9 10:35:43 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |   1 +
 bin/ignite.bat                                  |   5 +-
 bin/include/parseargs.bat                       |   2 +-
 examples/config/example-default.xml             |   7 +
 examples/pom.xml                                |   4 +-
 examples/schema-import/pom.xml                  |   4 +-
 .../store/auto/CacheBinaryAutoStoreExample.java | 158 ++++
 .../datagrid/store/auto/package-info.java       |  22 +
 .../store/auto/CacheAutoStoreExample.java       |  93 ++-
 .../auto/CacheAutoStoreLoadDataExample.java     |  85 --
 .../datagrid/store/auto/CacheConfig.java        |  81 --
 .../datagrid/store/auto/DbH2ServerStartup.java  |  79 --
 .../messaging/MessagingPingPongExample.java     |   4 -
 .../ignite/examples/util/DbH2ServerStartup.java |  79 ++
 .../ignite/examples/util/package-info.java      |  22 +
 .../examples/MessagingExamplesSelfTest.java     |   2 -
 modules/aop/pom.xml                             |  14 +-
 modules/apache-license-gen/pom.xml              |   2 +-
 modules/aws/pom.xml                             |  20 +-
 modules/camel/pom.xml                           |  13 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |   6 +-
 modules/core/pom.xml                            |  30 +-
 .../main/java/org/apache/ignite/Ignition.java   |  16 +
 .../store/jdbc/CacheAbstractJdbcStore.java      |  17 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   9 +
 .../configuration/CacheConfiguration.java       |  35 +-
 .../configuration/IgniteConfiguration.java      |   6 +-
 .../apache/ignite/internal/GridComponent.java   |   3 +-
 .../ignite/internal/GridPluginComponent.java    |   4 +-
 .../apache/ignite/internal/IgniteKernal.java    |  20 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  39 +-
 .../ignite/internal/MarshallerContextImpl.java  |  30 +-
 .../internal/managers/GridManagerAdapter.java   |   5 +-
 .../managers/communication/GridIoManager.java   |   3 +-
 .../communication/GridIoMessageFactory.java     |   8 +-
 .../deployment/GridDeploymentManager.java       |   5 +-
 .../internal/portable/BinaryFieldAccessor.java  |   9 +-
 .../internal/portable/BinaryReaderExImpl.java   |  21 +-
 .../internal/portable/BinaryWriterExImpl.java   |  24 +-
 .../internal/portable/PortableContext.java      |  46 +-
 .../processors/GridProcessorAdapter.java        |   5 +-
 .../CacheDefaultBinaryAffinityKeyMapper.java    |   2 +-
 .../processors/cache/GridCacheAdapter.java      |  65 +-
 .../cache/GridCacheConcurrentMap.java           | 734 +++++------------
 .../processors/cache/GridCacheContext.java      |  36 +-
 .../processors/cache/GridCacheIoManager.java    |   2 +-
 .../processors/cache/GridCacheMapEntry.java     |  44 +-
 .../cache/GridCacheMapEntryFactory.java         |  13 +-
 .../GridCachePartitionExchangeManager.java      |  17 +-
 .../processors/cache/GridCachePreloader.java    |  11 +-
 .../processors/cache/GridCacheProcessor.java    |  28 +-
 .../CacheDataStructuresManager.java             |  31 +-
 .../distributed/GridDistributedCacheEntry.java  |  14 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |   2 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  13 +-
 .../distributed/dht/GridDhtCacheEntry.java      |  14 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |  15 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  51 +-
 .../distributed/dht/GridNoStorageCacheMap.java  |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  43 +-
 .../dht/atomic/GridDhtAtomicCacheEntry.java     |  16 +-
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |  15 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  13 +-
 .../colocated/GridDhtColocatedCacheEntry.java   |  14 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |  15 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |   4 +-
 .../dht/preloader/GridDhtPartitionDemander.java |  41 +-
 .../GridDhtPartitionsExchangeFuture.java        |   9 +-
 .../dht/preloader/GridDhtPreloader.java         |   4 +-
 .../preloader/GridDhtPreloaderAssignments.java  |  19 +-
 .../distributed/near/GridNearCacheAdapter.java  |   8 +-
 .../distributed/near/GridNearCacheEntry.java    |  16 +-
 .../near/GridNearOffHeapCacheEntry.java         |  15 +-
 .../processors/cache/local/GridLocalCache.java  |   8 +-
 .../cache/local/GridLocalCacheEntry.java        |  15 +-
 .../local/atomic/GridLocalAtomicCache.java      |   8 +-
 .../CacheObjectBinaryProcessorImpl.java         |  49 +-
 .../cache/query/GridCacheQueryManager.java      |  58 +-
 .../cache/transactions/IgniteTxAdapter.java     |   5 +
 .../cache/transactions/IgniteTxEntry.java       |   8 +-
 .../transactions/IgniteTxLocalAdapter.java      |  11 +-
 .../datastructures/CollocatedQueueItemKey.java  |  75 ++
 .../datastructures/CollocatedSetItemKey.java    |  87 ++
 .../datastructures/DataStructuresProcessor.java |  11 +-
 .../GridAtomicCacheQueueImpl.java               |   8 +-
 .../datastructures/GridCacheQueueAdapter.java   |  30 +-
 .../datastructures/GridCacheQueueItemKey.java   |   9 +-
 .../datastructures/GridCacheSetImpl.java        |  37 +-
 .../datastructures/GridCacheSetItemKey.java     |  21 +-
 .../GridTransactionalCacheQueueImpl.java        |   2 +-
 .../processors/datastructures/QueueItemKey.java |  27 +
 .../processors/datastructures/SetItemKey.java   |  36 +
 .../handlers/query/QueryCommandHandler.java     |   8 +-
 .../internal/util/GridMessageCollection.java    | 160 ++++
 .../ignite/internal/util/IgniteUtils.java       |  18 +-
 .../internal/util/UUIDCollectionMessage.java    |  42 +-
 .../util/nio/GridNioBackPressureControl.java    |  14 +-
 .../optimized/OptimizedMarshallerUtils.java     |  16 +-
 .../org/apache/ignite/spi/IgniteSpiThread.java  |  10 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   6 +-
 .../org/apache/ignite/thread/IgniteThread.java  |  43 +-
 .../ignite/thread/IgniteThreadPoolExecutor.java |  37 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../IgniteClientReconnectAbstractTest.java      |  95 ++-
 .../IgniteClientReconnectAtomicsTest.java       |  57 ++
 .../IgniteClientReconnectCacheTest.java         |   5 +-
 .../IgniteClientReconnectCollectionsTest.java   |  51 ++
 .../portable/BinaryMarshallerSelfTest.java      |   2 +-
 ...cheAbstractFullApiMultithreadedSelfTest.java |   2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   6 +-
 .../cache/GridCacheAlwaysEvictionPolicy.java    |   2 +-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |   6 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |  11 -
 .../GridCacheConditionalDeploymentSelfTest.java |  20 +-
 .../cache/GridCacheDeploymentSelfTest.java      |  11 +-
 ...CacheMultinodeUpdateNearEnabledSelfTest.java |   5 +
 .../IgniteCacheEntryProcessorCallTest.java      | 497 ++++++++++++
 .../cache/IgniteCacheInvokeAbstractTest.java    | 369 ++++++---
 ...eAbstractDataStructuresFailoverSelfTest.java |   9 +-
 .../GridCacheQueueApiSelfAbstractTest.java      |  18 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |   6 +-
 .../GridCachePartitionedQueueApiSelfTest.java   |   5 +
 ...dCachePartitionedQueueEntryMoveSelfTest.java |   2 +-
 .../IgnitePartitionedQueueNoBackupsTest.java    |  92 +++
 .../GridCacheReplicatedQueueApiSelfTest.java    |   5 +
 .../CacheLockReleaseNodeLeaveTest.java          | 161 ++++
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |  37 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   2 +
 .../dht/GridNearCacheTxNodeFailureSelfTest.java |   4 -
 ...idCachePartitionedHitsAndMissesSelfTest.java |   4 +-
 ...cingDelayedPartitionMapExchangeSelfTest.java |   9 +-
 .../GridCacheRebalancingAsyncSelfTest.java      |   3 +-
 .../GridCacheRebalancingSyncCheckDataTest.java  |  98 +++
 .../GridCacheRebalancingSyncSelfTest.java       |  55 +-
 ...eRebalancingUnmarshallingFailedSelfTest.java |   6 +-
 .../GridCacheWriteBehindStoreAbstractTest.java  |   2 +-
 .../DataStreamProcessorSelfTest.java            |  25 +-
 .../internal/util/nio/GridNioSelfTest.java      |  69 +-
 .../loadtests/hashmap/GridHashMapLoadTest.java  |   4 +-
 .../marshaller/GridMarshallerAbstractTest.java  |  32 +-
 .../config/GridTestProperties.java              |   6 +
 .../testframework/junits/GridAbstractTest.java  |  31 +-
 .../junits/IgniteTestResources.java             |   2 +-
 .../IgniteBinaryObjectsCacheTestSuite3.java     |   2 +
 .../IgniteCacheDataStructuresSelfTestSuite.java |   3 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   6 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 .../testsuites/IgniteCacheTestSuite3.java       |   2 +
 .../IgnitePortableCacheTestSuite.java           |   3 +-
 .../testsuites/IgniteUtilSelfTestSuite.java     |   2 +
 .../ignite/util/GridMessageCollectionTest.java  | 105 +++
 modules/extdata/p2p/pom.xml                     |   2 +-
 .../extdata/uri/modules/uri-dependency/pom.xml  |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/flume/pom.xml                           |  18 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |  22 +-
 modules/hadoop/pom.xml                          |   4 +-
 modules/hibernate/pom.xml                       |  12 +-
 modules/indexing/pom.xml                        |  24 +-
 ...cheDuplicateEntityConfigurationSelfTest.java | 126 +++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   5 +
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +
 modules/jcl/pom.xml                             |  13 +-
 modules/jms11/pom.xml                           |  22 +-
 modules/jta/pom.xml                             |  19 +-
 modules/kafka/pom.xml                           |  18 +-
 modules/log4j/pom.xml                           |  12 +-
 modules/log4j2/pom.xml                          |  13 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/mqtt/pom.xml                            |  26 +-
 modules/osgi-karaf/README.txt                   |  18 +
 modules/osgi-karaf/licenses/apache-2.0.txt      | 202 +++++
 modules/osgi-karaf/pom.xml                      |  83 ++
 .../osgi-karaf/src/main/resources/features.xml  | 327 ++++++++
 modules/osgi-paxlogging/README.txt              |  12 +
 modules/osgi-paxlogging/licenses/apache-2.0.txt | 202 +++++
 modules/osgi-paxlogging/pom.xml                 |  67 ++
 modules/osgi/README.txt                         |  65 ++
 modules/osgi/licenses/apache-2.0.txt            | 202 +++++
 modules/osgi/pom.xml                            | 170 ++++
 .../IgniteAbstractOsgiContextActivator.java     | 238 ++++++
 .../org/apache/ignite/osgi/IgniteOsgiUtils.java |  69 ++
 .../BundleDelegatingClassLoader.java            | 147 ++++
 .../classloaders/ContainerSweepClassLoader.java | 134 ++++
 .../OsgiClassLoadingStrategyType.java           |  29 +
 .../ignite/osgi/AbstractIgniteKarafTest.java    | 107 +++
 .../IgniteKarafFeaturesInstallationTest.java    | 100 +++
 .../ignite/osgi/IgniteOsgiServiceTest.java      | 131 +++
 .../apache/ignite/osgi/IgniteOsgiTestSuite.java |  32 +
 .../activators/BasicIgniteTestActivator.java    |  76 ++
 .../ignite/osgi/activators/TestOsgiFlags.java   |  53 ++
 .../osgi/activators/TestOsgiFlagsImpl.java      |  83 ++
 modules/platforms/cpp/common/configure.ac       |   2 +-
 modules/platforms/cpp/core-test/configure.ac    |   2 +-
 modules/platforms/cpp/core/configure.ac         |   2 +-
 modules/platforms/cpp/examples/configure.ac     |   2 +-
 modules/platforms/cpp/ignite/configure.ac       |   2 +-
 .../Properties/AssemblyInfo.cs                  |   6 +-
 .../Properties/AssemblyInfo.cs                  |   6 +-
 .../Binary/BinarySelfTest.cs                    |  67 ++
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |   3 +-
 .../Properties/AssemblyInfo.cs                  |   6 +-
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Impl/Binary/BinaryReflectiveActions.cs      |   4 +-
 .../Impl/Common/DelegateConverter.cs            |  15 +-
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |  10 -
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    | 358 +++++++++
 .../Impl/Unmanaged/UnmanagedUtils.cs            | 788 +++----------------
 .../Properties/AssemblyInfo.cs                  |   6 +-
 modules/platforms/dotnet/Apache.Ignite.FxCop    | 322 +++++++-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   6 +-
 .../Properties/AssemblyInfo.cs                  |   6 +-
 .../Properties/AssemblyInfo.cs                  |   6 +-
 modules/rest-http/pom.xml                       |  36 +-
 modules/scalar-2.10/pom.xml                     |  11 +-
 modules/scalar/pom.xml                          |  11 +-
 modules/schedule/pom.xml                        |  27 +-
 modules/schema-import/pom.xml                   |   4 +-
 .../ignite/schema/ui/SchemaImportApp.java       |  14 +
 modules/slf4j/pom.xml                           |  14 +-
 modules/spark-2.10/pom.xml                      |   4 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |  32 +-
 .../ignite/internal/GridFactorySelfTest.java    |  29 +
 .../GridSpringBeanSerializationSelfTest.java    |   2 +
 modules/ssh/pom.xml                             |  21 +-
 modules/tools/pom.xml                           |   2 +-
 modules/twitter/pom.xml                         |  14 +-
 modules/urideploy/pom.xml                       |  19 +-
 modules/visor-console-2.10/pom.xml              |   6 +-
 modules/visor-console/pom.xml                   |   4 +-
 modules/visor-plugins/pom.xml                   |   6 +-
 modules/web/pom.xml                             |  15 +-
 .../config/benchmark-failover.properties        |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 .../yardstick/cache/IgnitePutTxBenchmark.java   |  26 +-
 ...IgniteTransactionalInvokeRetryBenchmark.java |   4 +-
 ...IgniteTransactionalWriteInvokeBenchmark.java |  34 +-
 modules/yarn/pom.xml                            |   2 +-
 .../apache/ignite/yarn/utils/package-info.java  |  22 +
 modules/zookeeper/pom.xml                       |  16 +-
 parent/pom.xml                                  | 140 +++-
 pom.xml                                         |  46 +-
 247 files changed, 7738 insertions(+), 2528 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb4aa63/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
index 5f5767a,3318641..314804e
--- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--/*
++/*
   * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb4aa63/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
index 165fd20,88b7a23..569f230
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--/*
++/*
   * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb4aa63/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
index 81148dc,9d32327..c194f29
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--/*
++/*
   * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb4aa63/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
index d82d096,4fb0458..0f6b32b
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--/*
++/*
   * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb4aa63/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
index 6b39342,bd2d2ad..8f9f843
--- a/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--/*
++/*
   * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb4aa63/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
index 78f0c9e,082dde2..7e07d8c
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--/*
++/*
   * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb4aa63/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
index 5492d2f,cc08601..43788f5
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--/*
++/*
   * 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.


[45/50] [abbrv] ignite git commit: ignite-1.5 Cache map refactoring. This closes #279.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapSelfTest.java
index 1f494c0..5bb6794 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapSelfTest.java
@@ -220,8 +220,6 @@ public class GridCacheConcurrentMapSelfTest extends GridCommonAbstractTest {
         Thread.sleep(1000);
 
         jcache().get(rand.nextInt(cnt));
-
-        assertEquals(0, local().map.iteratorMapSize());
     }
 
     /**
@@ -315,8 +313,6 @@ public class GridCacheConcurrentMapSelfTest extends GridCommonAbstractTest {
         Thread.sleep(1000);
 
         jcache().get(rand.nextInt(cnt));
-
-        assertEquals(0, local().map.iteratorMapSize());
     }
 
     /**
@@ -352,14 +348,7 @@ public class GridCacheConcurrentMapSelfTest extends GridCommonAbstractTest {
                 System.gc();
 
                 c.get(100);
-
-                if (local().map.iteratorMapSize() == 0)
-                    break;
-                else
-                    U.sleep(500);
             }
-
-            assertEquals(0, local().map.iteratorMapSize());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
index a41f6c8..b459dde 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
@@ -81,7 +81,7 @@ public class GridHashMapLoadTest extends GridCommonAbstractTest {
             Integer val = i++;
 
             map.put(key, new GridCacheMapEntry(ctx, ctx.toCacheKeyObject(key),
-                key.hashCode(), ctx.toCacheObject(val), null, 1) {
+                key.hashCode(), ctx.toCacheObject(val)) {
                 @Override public boolean tmLock(IgniteInternalTx tx,
                     long timeout,
                     @Nullable GridCacheVersion serOrder,
@@ -103,4 +103,4 @@ public class GridHashMapLoadTest extends GridCommonAbstractTest {
                 info("Inserted objects: " + i / 2);
         }
     }
-}
\ No newline at end of file
+}


[18/50] [abbrv] ignite git commit: Javadoc hotfix

Posted by sb...@apache.org.
Javadoc hotfix


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

Branch: refs/heads/ignite-1.5.1
Commit: d5fa3c1b69cfa336e70087dded9ddfa099000235
Parents: e5de23e
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Dec 4 13:24:59 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Dec 4 13:24:59 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/preloader/GridDhtPartitionDemander.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d5fa3c1b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index ced0d10..998f7a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -359,7 +359,7 @@ public class GridDhtPartitionDemander {
      * @param fut Future.
      * @param assigns Assignments.
      * @throws IgniteCheckedException If failed.
-     * @return
+     * @return Partitions were requested.
      */
     private boolean requestPartitions(
         RebalanceFuture fut,


[12/50] [abbrv] ignite git commit: IGNITE-1270 Update version number of OSGi modules to 1.5.0-b1-SNAPSHOT. Adjust test that assumed major.minor.micro to take into account qualifier.

Posted by sb...@apache.org.
IGNITE-1270 Update version number of OSGi modules to 1.5.0-b1-SNAPSHOT.
Adjust test that assumed major.minor.micro to take into account qualifier.


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

Branch: refs/heads/ignite-1.5.1
Commit: 4ba911f0a4dfbfffd8bef019dcbaa284c5d98bf3
Parents: 96e0802
Author: Raul Kripalani <ra...@apache.org>
Authored: Thu Dec 3 14:46:23 2015 +0000
Committer: Raul Kripalani <ra...@apache.org>
Committed: Thu Dec 3 15:00:58 2015 +0000

----------------------------------------------------------------------
 modules/osgi-karaf/pom.xml                                        | 3 +--
 modules/osgi-paxlogging/pom.xml                                   | 3 +--
 modules/osgi/pom.xml                                              | 2 +-
 .../apache/ignite/osgi/IgniteKarafFeaturesInstallationTest.java   | 2 +-
 4 files changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4ba911f0/modules/osgi-karaf/pom.xml
----------------------------------------------------------------------
diff --git a/modules/osgi-karaf/pom.xml b/modules/osgi-karaf/pom.xml
index f1ac66b..e1f53e2 100644
--- a/modules/osgi-karaf/pom.xml
+++ b/modules/osgi-karaf/pom.xml
@@ -31,9 +31,8 @@
         <relativePath>../../parent</relativePath>
     </parent>
 
-    <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-osgi-karaf</artifactId>
-    <version>1.5.0-SNAPSHOT</version>
+    <version>1.5.0-b1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/4ba911f0/modules/osgi-paxlogging/pom.xml
----------------------------------------------------------------------
diff --git a/modules/osgi-paxlogging/pom.xml b/modules/osgi-paxlogging/pom.xml
index bd9e18e..15a5e0c 100644
--- a/modules/osgi-paxlogging/pom.xml
+++ b/modules/osgi-paxlogging/pom.xml
@@ -32,9 +32,8 @@
         <relativePath>../../parent</relativePath>
     </parent>
 
-    <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-osgi-paxlogging</artifactId>
-    <version>1.5.0-SNAPSHOT</version>
+    <version>1.5.0-b1-SNAPSHOT</version>
     <packaging>jar</packaging>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/4ba911f0/modules/osgi/pom.xml
----------------------------------------------------------------------
diff --git a/modules/osgi/pom.xml b/modules/osgi/pom.xml
index d6d46bf..47c7928 100644
--- a/modules/osgi/pom.xml
+++ b/modules/osgi/pom.xml
@@ -32,7 +32,7 @@
     </parent>
 
     <artifactId>ignite-osgi</artifactId>
-    <version>1.5.0-SNAPSHOT</version>
+    <version>1.5.0-b1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/4ba911f0/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteKarafFeaturesInstallationTest.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteKarafFeaturesInstallationTest.java b/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteKarafFeaturesInstallationTest.java
index c0eb06b..112a607 100644
--- a/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteKarafFeaturesInstallationTest.java
+++ b/modules/osgi/src/test/java/org/apache/ignite/osgi/IgniteKarafFeaturesInstallationTest.java
@@ -87,7 +87,7 @@ public class IgniteKarafFeaturesInstallationTest extends AbstractIgniteKarafTest
                 f.getName(), installed));
 
             assertTrue(installed);
-            assertEquals(PROJECT_VERSION.replaceAll("-", "."), f.getVersion());
+            assertEquals(PROJECT_VERSION.replaceAll("-", "."), f.getVersion().replaceAll("-", "."));
         }
     }
 


[20/50] [abbrv] ignite git commit: IGNITE-2041 Fixed JDBC cache store example. Reworked from deprecated code to new API and refactored code, made it more simple. Fixed bug in store in case of cache configured keepBinaryInStore(true). Added one more a

Posted by sb...@apache.org.
 IGNITE-2041 Fixed JDBC cache store example.
 Reworked from deprecated code to new API and refactored code, made it more simple.
 Fixed bug in store in case of cache configured keepBinaryInStore(true).
 Added one more auto cache store example: CacheBinaryAutoStoreExample.


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

Branch: refs/heads/ignite-1.5.1
Commit: 92ef7c7ce8229130762b64082cc9455579a05247
Parents: 6a109eb
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Dec 4 17:57:57 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 4 17:57:57 2015 +0700

----------------------------------------------------------------------
 examples/config/example-default.xml             |   7 +
 .../store/auto/CacheBinaryAutoStoreExample.java | 158 +++++++++++++++++++
 .../datagrid/store/auto/package-info.java       |  22 +++
 .../store/auto/CacheAutoStoreExample.java       |  93 +++++++++--
 .../auto/CacheAutoStoreLoadDataExample.java     |  85 ----------
 .../datagrid/store/auto/CacheConfig.java        |  81 ----------
 .../datagrid/store/auto/DbH2ServerStartup.java  |  79 ----------
 .../ignite/examples/util/DbH2ServerStartup.java |  79 ++++++++++
 .../ignite/examples/util/package-info.java      |  22 +++
 .../store/jdbc/CacheAbstractJdbcStore.java      |  12 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   9 ++
 11 files changed, 390 insertions(+), 257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/examples/config/example-default.xml
----------------------------------------------------------------------
diff --git a/examples/config/example-default.xml b/examples/config/example-default.xml
index e6c359d..6bd6f16 100644
--- a/examples/config/example-default.xml
+++ b/examples/config/example-default.xml
@@ -28,6 +28,13 @@
         http://www.springframework.org/schema/beans/spring-beans.xsd
         http://www.springframework.org/schema/util
         http://www.springframework.org/schema/util/spring-util.xsd">
+
+    <!-- Datasource for sample in-memory H2 database. -->
+    <bean id="h2-example-db" class="org.h2.jdbcx.JdbcDataSource">
+        <property name="URL" value="jdbc:h2:tcp://localhost/mem:ExampleDb" />
+        <property name="user" value="sa" />
+    </bean>
+
     <bean abstract="true" id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
         <!-- Set to true to enable distributed class loading for examples, default is false. -->
         <property name="peerClassLoadingEnabled" value="true"/>

http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java
new file mode 100644
index 0000000..9df9f79
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java
@@ -0,0 +1,158 @@
+/*
+ * 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.examples.binary.datagrid.store.auto;
+
+import java.sql.Types;
+import java.util.UUID;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore;
+import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory;
+import org.apache.ignite.cache.store.jdbc.JdbcType;
+import org.apache.ignite.cache.store.jdbc.JdbcTypeField;
+import org.apache.ignite.cache.store.jdbc.dialect.H2Dialect;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.examples.ExampleNodeStartup;
+import org.apache.ignite.examples.util.DbH2ServerStartup;
+import org.apache.ignite.examples.model.Person;
+import org.apache.ignite.transactions.Transaction;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+
+/**
+ * Demonstrates usage of cache with underlying persistent store configured.
+ * <p>
+ * This example uses {@link CacheJdbcPojoStore} as a persistent store.
+ * <p>
+ * To start the example, you should:
+ * <ul>
+ *     <li>Start H2 database TCP server using {@link DbH2ServerStartup}.</li>
+ *     <li>Start a few nodes using {@link ExampleNodeStartup} or by starting remote nodes as specified below.</li>
+ *     <li>Start example using {@link CacheBinaryAutoStoreExample}.</li>
+ * </ul>
+ * <p>
+ * Remote nodes should always be started with special configuration file which
+ * contains H2 data source bean descriptor: {@code 'ignite.{sh|bat} examples/config/example-ignite.xml'}.
+ * <p>
+ * Alternatively you can run {@link ExampleNodeStartup} in another JVM which will
+ * start node with {@code examples/config/example-ignite.xml} configuration.
+ */
+public class CacheBinaryAutoStoreExample {
+    /** Global person ID to use across entire example. */
+    private static final Long id = Math.abs(UUID.randomUUID().getLeastSignificantBits());
+
+    /** Cache name. */
+    public static final String CACHE_NAME = CacheBinaryAutoStoreExample.class.getSimpleName();
+
+    /**
+     * Configure cache with store.
+     */
+    private static CacheConfiguration<Long, Person> cacheConfiguration() {
+        CacheJdbcPojoStoreFactory<Long, Person> storeFactory = new CacheJdbcPojoStoreFactory<>();
+
+        storeFactory.setDataSourceBean("h2-example-db");
+        storeFactory.setDialect(new H2Dialect());
+
+        JdbcType jdbcType = new JdbcType();
+
+        jdbcType.setCacheName(CACHE_NAME);
+        jdbcType.setDatabaseSchema("PUBLIC");
+        jdbcType.setDatabaseTable("PERSON");
+
+        jdbcType.setKeyType("java.lang.Long");
+        jdbcType.setKeyFields(new JdbcTypeField(Types.BIGINT, "ID", Long.class, "id"));
+
+        jdbcType.setValueType("org.apache.ignite.examples.model.Person");
+        jdbcType.setValueFields(
+                new JdbcTypeField(Types.BIGINT, "ID", Long.class, "id"),
+                new JdbcTypeField(Types.VARCHAR, "FIRST_NAME", String.class, "firstName"),
+                new JdbcTypeField(Types.VARCHAR, "LAST_NAME", String.class, "lastName")
+        );
+
+        storeFactory.setTypes(jdbcType);
+
+        CacheConfiguration<Long, Person> cfg = new CacheConfiguration<>(CACHE_NAME);
+
+        cfg.setCacheStoreFactory(storeFactory);
+
+        // Set atomicity as transaction, since we are showing transactions in the example.
+        cfg.setAtomicityMode(TRANSACTIONAL);
+
+        // This option will allow to start remote nodes without having user classes in classpath.
+        cfg.setKeepBinaryInStore(true);
+
+        cfg.setReadThrough(true);
+        cfg.setWriteThrough(true);
+
+        return cfg;
+    }
+
+    /**
+     * Executes example.
+     *
+     * @param args Command line arguments, none required.
+     * @throws IgniteException If example execution failed.
+     */
+    public static void main(String[] args) throws IgniteException {
+        // To start ignite with desired configuration uncomment the appropriate line.
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            System.out.println();
+            System.out.println(">>> Cache auto store example started.");
+
+            try (IgniteCache<Long, Person> cache = ignite.getOrCreateCache(cacheConfiguration())) {
+                try (Transaction tx = ignite.transactions().txStart()) {
+                    Person val = cache.get(id);
+
+                    System.out.println("Read value: " + val);
+
+                    val = cache.getAndPut(id, new Person(id, "Isaac", "Newton"));
+
+                    System.out.println("Overwrote old value: " + val);
+
+                    val = cache.get(id);
+
+                    System.out.println("Read value: " + val);
+
+                    tx.commit();
+                }
+
+                System.out.println("Read value after commit: " + cache.get(id));
+
+                cache.clear();
+
+                System.out.println(">>> ------------------------------------------");
+                System.out.println(">>> Load data to cache from DB with cusom SQL...");
+
+                // Load cache on all data nodes with custom SQL statement.
+                cache.loadCache(null, "java.lang.Long", "select * from PERSON where id <= 3");
+
+                System.out.println("Loaded cache entries: " + cache.size());
+
+                cache.clear();
+
+                // Load cache on all data nodes with default SQL statement.
+                System.out.println(">>> Load ALL data to cache from DB...");
+                cache.loadCache(null);
+
+                System.out.println("Loaded cache entries: " + cache.size());
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/package-info.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/package-info.java
new file mode 100644
index 0000000..153f210
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Contains automatic JDBC store example.
+ */
+package org.apache.ignite.examples.binary.datagrid.store.auto;

http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java
index 37a31d7..a262c38 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreExample.java
@@ -17,15 +17,25 @@
 
 package org.apache.ignite.examples.datagrid.store.auto;
 
+import java.sql.Types;
 import java.util.UUID;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore;
+import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory;
+import org.apache.ignite.cache.store.jdbc.JdbcType;
+import org.apache.ignite.cache.store.jdbc.JdbcTypeField;
+import org.apache.ignite.cache.store.jdbc.dialect.H2Dialect;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.examples.ExampleNodeStartup;
 import org.apache.ignite.examples.model.Person;
+import org.apache.ignite.examples.util.DbH2ServerStartup;
 import org.apache.ignite.transactions.Transaction;
+import org.h2.jdbcx.JdbcConnectionPool;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 
 /**
  * Demonstrates usage of cache with underlying persistent store configured.
@@ -35,23 +45,70 @@ import org.apache.ignite.transactions.Transaction;
  * To start the example, you should:
  * <ul>
  *     <li>Start H2 database TCP server using {@link DbH2ServerStartup}.</li>
- *     <li>Start a few nodes using {@link ExampleNodeStartup} or by starting remote nodes as specified below.</li>
+ *     <li>Start a few nodes using {@link ExampleNodeStartup}.</li>
  *     <li>Start example using {@link CacheAutoStoreExample}.</li>
  * </ul>
  * <p>
- * Remote nodes should always be started with special configuration file which
- * enables P2P class loading: {@code 'ignite.{sh|bat} examples/config/example-ignite.xml'}.
- * <p>
- * Alternatively you can run {@link ExampleNodeStartup} in another JVM which will
+ * Remote nodes can be started with {@link ExampleNodeStartup} in another JVM which will
  * start node with {@code examples/config/example-ignite.xml} configuration.
  */
 public class CacheAutoStoreExample {
-    /** Cache name. */
-    public static final String CACHE_NAME = CacheAutoStoreLoadDataExample.class.getSimpleName();
-
     /** Global person ID to use across entire example. */
     private static final Long id = Math.abs(UUID.randomUUID().getLeastSignificantBits());
 
+    /** Cache name. */
+    public static final String CACHE_NAME = CacheAutoStoreExample.class.getSimpleName();
+
+    /**
+     * Example store factory.
+     */
+    private static final class CacheJdbcPojoStoreExampleFactory extends CacheJdbcPojoStoreFactory<Long, Person> {
+        /** {@inheritDoc} */
+        @Override public CacheJdbcPojoStore<Long, Person> create() {
+            JdbcType jdbcType = new JdbcType();
+
+            jdbcType.setCacheName(CACHE_NAME);
+            jdbcType.setDatabaseSchema("PUBLIC");
+            jdbcType.setDatabaseTable("PERSON");
+
+            jdbcType.setKeyType("java.lang.Long");
+            jdbcType.setKeyFields(new JdbcTypeField(Types.BIGINT, "ID", Long.class, "id"));
+
+            jdbcType.setValueType("org.apache.ignite.examples.model.Person");
+            jdbcType.setValueFields(
+                    new JdbcTypeField(Types.BIGINT, "ID", Long.class, "id"),
+                    new JdbcTypeField(Types.VARCHAR, "FIRST_NAME", String.class, "firstName"),
+                    new JdbcTypeField(Types.VARCHAR, "LAST_NAME", String.class, "lastName")
+            );
+
+            CacheJdbcPojoStore<Long, Person> store = new CacheJdbcPojoStore<>();
+
+            store.setDataSource(JdbcConnectionPool.create("jdbc:h2:tcp://localhost/mem:ExampleDb", "sa", ""));
+            store.setDialect(new H2Dialect());
+
+            store.setTypes(jdbcType);
+
+            return store;
+        }
+    }
+
+    /**
+     * Configure cache with store.
+     */
+    private static CacheConfiguration<Long, Person> cacheConfiguration() {
+        CacheConfiguration<Long, Person> cfg = new CacheConfiguration<>(CACHE_NAME);
+
+        cfg.setCacheStoreFactory(new CacheJdbcPojoStoreExampleFactory());
+
+        // Set atomicity as transaction, since we are showing transactions in the example.
+        cfg.setAtomicityMode(TRANSACTIONAL);
+
+        cfg.setReadThrough(true);
+        cfg.setWriteThrough(true);
+
+        return cfg;
+    }
+
     /**
      * Executes example.
      *
@@ -64,7 +121,7 @@ public class CacheAutoStoreExample {
             System.out.println();
             System.out.println(">>> Cache auto store example started.");
 
-            try (IgniteCache<Long, Person> cache = ignite.getOrCreateCache(CacheConfig.jdbcPojoStoreCache(CACHE_NAME))) {
+            try (IgniteCache<Long, Person> cache = ignite.getOrCreateCache(cacheConfiguration())) {
                 try (Transaction tx = ignite.transactions().txStart()) {
                     Person val = cache.get(id);
 
@@ -82,6 +139,24 @@ public class CacheAutoStoreExample {
                 }
 
                 System.out.println("Read value after commit: " + cache.get(id));
+
+                cache.clear();
+
+                System.out.println(">>> ------------------------------------------");
+                System.out.println(">>> Load data to cache from DB with cusom SQL...");
+
+                // Load cache on all data nodes with custom SQL statement.
+                cache.loadCache(null, "java.lang.Long", "select * from PERSON where id <= 3");
+
+                System.out.println("Loaded cache entries: " + cache.size());
+
+                cache.clear();
+
+                // Load cache on all data nodes with default SQL statement.
+                System.out.println(">>> Load ALL data to cache from DB...");
+                cache.loadCache(null);
+
+                System.out.println("Loaded cache entries: " + cache.size());
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreLoadDataExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreLoadDataExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreLoadDataExample.java
deleted file mode 100644
index 63a8c6f..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheAutoStoreLoadDataExample.java
+++ /dev/null
@@ -1,85 +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.examples.datagrid.store.auto;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.examples.ExampleNodeStartup;
-import org.apache.ignite.examples.ExamplesUtils;
-import org.apache.ignite.examples.model.Person;
-
-/**
- * Demonstrates how to load data from database.
- * <p>
- * This example uses {@link CacheJdbcPojoStore} as a persistent store.
- * <p>
- * To start the example, you should:
- * <ul>
- *     <li>Start H2 database TCP server using {@link DbH2ServerStartup}.</li>
- *     <li>Start a few nodes using {@link ExampleNodeStartup} or by starting remote nodes as specified below.</li>
- *     <li>Start example using {@link CacheAutoStoreLoadDataExample}.</li>
- * </ul>
- * <p>
- * Remote nodes should always be started with special configuration file which
- * enables P2P class loading: {@code 'ignite.{sh|bat} examples/config/example-ignite.xml'}.
- * <p>
- * Alternatively you can run {@link ExampleNodeStartup} in another JVM which will
- * start node with {@code examples/config/example-ignite.xml} configuration.
- */
-public class CacheAutoStoreLoadDataExample {
-    /** Cache name. */
-    public static final String CACHE_NAME = CacheAutoStoreLoadDataExample.class.getSimpleName();
-
-    /** Heap size required to run this example. */
-    public static final int MIN_MEMORY = 1024 * 1024 * 1024;
-
-    /**
-     * Executes example.
-     *
-     * @param args Command line arguments, none required.
-     * @throws IgniteException If example execution failed.
-     */
-    public static void main(String[] args) throws IgniteException {
-        ExamplesUtils.checkMinMemory(MIN_MEMORY);
-
-        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
-            System.out.println();
-            System.out.println(">>> Cache auto store load data example started.");
-
-            CacheConfiguration<Long, Person> cacheCfg = CacheConfig.jdbcPojoStoreCache(CACHE_NAME);
-
-            try (IgniteCache<Long, Person> cache = ignite.getOrCreateCache(cacheCfg)) {
-                // Load cache on all data nodes with custom SQL statement.
-                cache.loadCache(null, "java.lang.Long", "select * from PERSON where id <= 3");
-
-                System.out.println("Loaded cache entries: " + cache.size());
-
-                cache.clear();
-
-                // Load cache on all data nodes with default SQL statement.
-                cache.loadCache(null);
-
-                System.out.println("Loaded cache entries: " + cache.size());
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheConfig.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheConfig.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheConfig.java
deleted file mode 100644
index 3b38aeb..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/CacheConfig.java
+++ /dev/null
@@ -1,81 +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.examples.datagrid.store.auto;
-
-import java.sql.Types;
-import java.util.Arrays;
-import java.util.Collections;
-import javax.cache.configuration.Factory;
-import org.apache.ignite.cache.CacheTypeFieldMetadata;
-import org.apache.ignite.cache.CacheTypeMetadata;
-import org.apache.ignite.cache.store.CacheStore;
-import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.examples.model.Person;
-import org.h2.jdbcx.JdbcConnectionPool;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-
-/**
- * Predefined configuration for examples with {@link CacheJdbcPojoStore}.
- */
-public class CacheConfig {
-    /**
-     * Configure cache with store.
-     */
-    public static CacheConfiguration<Long, Person> jdbcPojoStoreCache(String name) {
-        CacheConfiguration<Long, Person> cfg = new CacheConfiguration<>(name);
-
-        // Set atomicity as transaction, since we are showing transactions in the example.
-        cfg.setAtomicityMode(TRANSACTIONAL);
-
-        cfg.setCacheStoreFactory(new Factory<CacheStore<? super Long, ? super Person>>() {
-            @Override public CacheStore<? super Long, ? super Person> create() {
-                CacheJdbcPojoStore<Long, Person> store = new CacheJdbcPojoStore<>();
-
-                store.setDataSource(JdbcConnectionPool.create("jdbc:h2:tcp://localhost/mem:ExampleDb", "sa", ""));
-
-                return store;
-            }
-        });
-
-        CacheTypeMetadata meta = new CacheTypeMetadata();
-
-        meta.setDatabaseTable("PERSON");
-
-        meta.setKeyType("java.lang.Long");
-        meta.setValueType("org.apache.ignite.examples.model.Person");
-
-        meta.setKeyFields(Collections.singletonList(new CacheTypeFieldMetadata("ID", Types.BIGINT, "id", Long.class)));
-
-        meta.setValueFields(Arrays.asList(
-            new CacheTypeFieldMetadata("ID", Types.BIGINT, "id", long.class),
-            new CacheTypeFieldMetadata("FIRST_NAME", Types.VARCHAR, "firstName", String.class),
-            new CacheTypeFieldMetadata("LAST_NAME", Types.VARCHAR, "lastName", String.class)
-        ));
-
-        cfg.setTypeMetadata(Collections.singletonList(meta));
-
-        cfg.setWriteBehindEnabled(true);
-
-        cfg.setReadThrough(true);
-        cfg.setWriteThrough(true);
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/DbH2ServerStartup.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/DbH2ServerStartup.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/DbH2ServerStartup.java
deleted file mode 100644
index f5e07a8..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/auto/DbH2ServerStartup.java
+++ /dev/null
@@ -1,79 +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.examples.datagrid.store.auto;
-
-import java.io.IOException;
-import java.io.StringReader;
-import java.sql.SQLException;
-import org.apache.ignite.IgniteException;
-import org.h2.jdbcx.JdbcConnectionPool;
-import org.h2.tools.RunScript;
-import org.h2.tools.Server;
-
-/**
- * Start H2 database TCP server in order to access sample in-memory database from other processes.
- */
-public class DbH2ServerStartup {
-    /** Create table script. */
-    private static final String CREATE_TABLE =
-        "create table PERSON(id bigint not null, first_name varchar(50), last_name varchar(50), PRIMARY KEY(id));";
-
-    /** Sample data script. */
-    private static final String POPULATE_TABLE =
-        "insert into PERSON(id, first_name, last_name) values(1, 'Johannes', 'Kepler');\n" +
-        "insert into PERSON(id, first_name, last_name) values(2, 'Galileo', 'Galilei');\n" +
-        "insert into PERSON(id, first_name, last_name) values(3, 'Henry', 'More');\n" +
-        "insert into PERSON(id, first_name, last_name) values(4, 'Polish', 'Brethren');\n" +
-        "insert into PERSON(id, first_name, last_name) values(5, 'Robert', 'Boyle');\n" +
-        "insert into PERSON(id, first_name, last_name) values(6, 'Isaac', 'Newton');";
-
-    /**
-     * Start H2 database TCP server.
-     *
-     * @param args Command line arguments, none required.
-     * @throws IgniteException If start H2 database TCP server failed.
-     */
-    public static void main(String[] args) throws IgniteException {
-        try {
-            // Start H2 database TCP server in order to access sample in-memory database from other processes.
-            Server.createTcpServer("-tcpDaemon").start();
-
-            // Try to connect to database TCP server.
-            JdbcConnectionPool dataSrc = JdbcConnectionPool.create("jdbc:h2:tcp://localhost/mem:ExampleDb", "sa", "");
-
-            // Create Person table in database.
-            RunScript.execute(dataSrc.getConnection(), new StringReader(CREATE_TABLE));
-
-            // Populates Person table with sample data in database.
-            RunScript.execute(dataSrc.getConnection(), new StringReader(POPULATE_TABLE));
-        }
-        catch (SQLException e) {
-            throw new IgniteException("Failed to start database TCP server", e);
-        }
-
-        try {
-            do {
-                System.out.println("Type 'q' and press 'Enter' to stop H2 TCP server...");
-            }
-            while ('q' != System.in.read());
-        }
-        catch (IOException ignored) {
-            // No-op.
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/examples/src/main/java/org/apache/ignite/examples/util/DbH2ServerStartup.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/util/DbH2ServerStartup.java b/examples/src/main/java/org/apache/ignite/examples/util/DbH2ServerStartup.java
new file mode 100644
index 0000000..01717d0
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/util/DbH2ServerStartup.java
@@ -0,0 +1,79 @@
+/*
+ * 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.examples.util;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.sql.SQLException;
+import org.apache.ignite.IgniteException;
+import org.h2.jdbcx.JdbcConnectionPool;
+import org.h2.tools.RunScript;
+import org.h2.tools.Server;
+
+/**
+ * Start H2 database TCP server in order to access sample in-memory database from other processes.
+ */
+public class DbH2ServerStartup {
+    /** Create table script. */
+    private static final String CREATE_TABLE =
+        "create table PERSON(id bigint not null, first_name varchar(50), last_name varchar(50), PRIMARY KEY(id));";
+
+    /** Sample data script. */
+    private static final String POPULATE_TABLE =
+        "insert into PERSON(id, first_name, last_name) values(1, 'Johannes', 'Kepler');\n" +
+        "insert into PERSON(id, first_name, last_name) values(2, 'Galileo', 'Galilei');\n" +
+        "insert into PERSON(id, first_name, last_name) values(3, 'Henry', 'More');\n" +
+        "insert into PERSON(id, first_name, last_name) values(4, 'Polish', 'Brethren');\n" +
+        "insert into PERSON(id, first_name, last_name) values(5, 'Robert', 'Boyle');\n" +
+        "insert into PERSON(id, first_name, last_name) values(6, 'Isaac', 'Newton');";
+
+    /**
+     * Start H2 database TCP server.
+     *
+     * @param args Command line arguments, none required.
+     * @throws IgniteException If start H2 database TCP server failed.
+     */
+    public static void main(String[] args) throws IgniteException {
+        try {
+            // Start H2 database TCP server in order to access sample in-memory database from other processes.
+            Server.createTcpServer("-tcpDaemon").start();
+
+            // Try to connect to database TCP server.
+            JdbcConnectionPool dataSrc = JdbcConnectionPool.create("jdbc:h2:tcp://localhost/mem:ExampleDb", "sa", "");
+
+            // Create Person table in database.
+            RunScript.execute(dataSrc.getConnection(), new StringReader(CREATE_TABLE));
+
+            // Populates Person table with sample data in database.
+            RunScript.execute(dataSrc.getConnection(), new StringReader(POPULATE_TABLE));
+        }
+        catch (SQLException e) {
+            throw new IgniteException("Failed to start database TCP server", e);
+        }
+
+        try {
+            do {
+                System.out.println("Type 'q' and press 'Enter' to stop H2 TCP server...");
+            }
+            while ('q' != System.in.read());
+        }
+        catch (IOException ignored) {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/examples/src/main/java/org/apache/ignite/examples/util/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/util/package-info.java b/examples/src/main/java/org/apache/ignite/examples/util/package-info.java
new file mode 100644
index 0000000..1d87d02
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/util/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Contains utility classes for examples.
+ */
+package org.apache.ignite.examples.util;

http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/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 7617e48..366262c 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
@@ -49,6 +49,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.CacheTypeFieldMetadata;
 import org.apache.ignite.cache.CacheTypeMetadata;
 import org.apache.ignite.cache.store.CacheStore;
@@ -1442,8 +1443,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
      * @return Next index for parameters.
      * @throws CacheException If failed to set statement parameters.
      */
-    protected int fillKeyParameters(PreparedStatement stmt, int idx, EntryMapping em,
-        Object key) throws CacheException {
+    protected int fillKeyParameters(PreparedStatement stmt, int idx, EntryMapping em, Object key) throws CacheException {
         for (JdbcTypeField field : em.keyColumns()) {
             Object fieldVal = extractParameter(em.cacheName, em.keyType(), em.keyKind(), field.getJavaFieldName(), key);
 
@@ -1474,8 +1474,14 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
      */
     protected int fillValueParameters(PreparedStatement stmt, int idx, EntryMapping em, Object val)
         throws CacheWriterException {
+        TypeKind valKind = em.valueKind();
+
+        // Object could be passed by cache in binary format in case of cache configured with setKeepBinaryInStore(true).
+        if (valKind == TypeKind.POJO && val instanceof BinaryObject)
+            valKind = TypeKind.BINARY;
+
         for (JdbcTypeField field : em.uniqValFlds) {
-            Object fieldVal = extractParameter(em.cacheName, em.valueType(), em.valueKind(), field.getJavaFieldName(), val);
+            Object fieldVal = extractParameter(em.cacheName, em.valueType(), valKind, field.getJavaFieldName(), val);
 
             fillParameter(stmt, idx++, field, fieldVal);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92ef7c7c/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 abc4b2e..a25df04 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
@@ -354,6 +354,15 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
             this.getter = getter;
             this.setter = setter;
             this.field = field;
+
+            if (getter != null)
+                getter.setAccessible(true);
+
+            if (setter != null)
+                setter.setAccessible(true);
+
+            if (field != null)
+                field.setAccessible(true);
         }
 
         /**


[07/50] [abbrv] ignite git commit: Hotfix IGNITE-2079

Posted by sb...@apache.org.
Hotfix IGNITE-2079


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

Branch: refs/heads/ignite-1.5.1
Commit: dc0131da6582fb0f0d1ff25c62eb7462e5f460af
Parents: 86ec37e
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 3 16:52:26 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 3 16:52:26 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0131da/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 b06526e..9a391e5 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
@@ -562,7 +562,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             default:
                 throw new IgniteCheckedException("Failed to send response to node. Unsupported direct type [message="
-                    + msg + "]");
+                    + msg + "]", msg.classError());
         }
     }
 


[15/50] [abbrv] ignite git commit: IGNITE-1947 More info about why query was not found in error message.

Posted by sb...@apache.org.
IGNITE-1947 More info about why query was not found in error message.


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

Branch: refs/heads/ignite-1.5.1
Commit: e6f6f407ef4f0c3555fd80fe5ba328739d94e3f0
Parents: baab3d2
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Dec 4 15:14:49 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 4 15:14:49 2015 +0700

----------------------------------------------------------------------
 .../processors/rest/handlers/query/QueryCommandHandler.java    | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e6f6f407/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
index bb929a4..0c44077 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
@@ -443,14 +443,16 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
 
                 if (qryCurIt == null)
                     return new GridRestResponse(GridRestResponse.STATUS_FAILED,
-                        "Failed to find query with ID: " + req.queryId());
+                        "Failed to find query with ID: " + req.queryId() + ". " +
+                        "Possible reasons: wrong query ID, no more data to fetch from query, query was closed by timeout" +
+                        " or node where query was executed is not found.");
 
                 qryCurIt.lock();
 
                 try {
                     if (qryCurIt.timestamp() == -1)
                         return new GridRestResponse(GridRestResponse.STATUS_FAILED,
-                            "Query is closed by timeout. Restart query with ID: " + req.queryId());
+                            "Query with ID: " + req.queryId() + " was closed by timeout");
 
                     qryCurIt.timestamp(U.currentTimeMillis());
 


[13/50] [abbrv] ignite git commit: Fixed failover tests for binary marshaller

Posted by sb...@apache.org.
Fixed failover tests for binary marshaller


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

Branch: refs/heads/ignite-1.5.1
Commit: 171e9b172564bcaafd54532abbce3b16c7ffbc65
Parents: 4ba911f
Author: ashutak <as...@gridgain.com>
Authored: Thu Dec 3 18:15:40 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu Dec 3 18:15:40 2015 +0300

----------------------------------------------------------------------
 .../failover/IgniteTransactionalInvokeRetryBenchmark.java      | 4 ++--
 .../failover/IgniteTransactionalWriteInvokeBenchmark.java      | 6 +++---
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/171e9b17/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalInvokeRetryBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalInvokeRetryBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalInvokeRetryBenchmark.java
index 16b0959..b5a08da 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalInvokeRetryBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalInvokeRetryBenchmark.java
@@ -169,7 +169,7 @@ public class IgniteTransactionalInvokeRetryBenchmark extends IgniteFailoverAbstr
                 if (ex != null)
                     throw ex;
 
-                asyncCache.invoke(key, new IncrementCacheEntryProcessor());
+                asyncCache.invoke(key, new IncrementInvokeRetryCacheEntryProcessor());
                 asyncCache.future().get(args.cacheOperationTimeoutMillis());
 
                 AtomicLong prevVal = map.putIfAbsent(key, new AtomicLong(0));
@@ -195,7 +195,7 @@ public class IgniteTransactionalInvokeRetryBenchmark extends IgniteFailoverAbstr
 
     /**
      */
-    private static class IncrementCacheEntryProcessor implements CacheEntryProcessor<String, Long, Long> {
+    private static class IncrementInvokeRetryCacheEntryProcessor implements CacheEntryProcessor<String, Long, Long> {
         /** */
         private static final long serialVersionUID = 0;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/171e9b17/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
index a52ea78..9f9b5f7 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
@@ -123,7 +123,7 @@ public class IgniteTransactionalWriteInvokeBenchmark extends IgniteFailoverAbstr
                         Set<Long> values = new HashSet<>(map.values());
 
                         if (values.size() != 1)
-                            throw new IgniteConsistencyException("Found different values for keys [map="+map+"]");
+                            throw new IgniteConsistencyException("Found different values for keys [map=" + map + "]");
 
                         break;
                     case 1: // Invoke scenario.
@@ -137,7 +137,7 @@ public class IgniteTransactionalWriteInvokeBenchmark extends IgniteFailoverAbstr
                         asyncCache.future().get(timeout);
 
                         for (String key : keys) {
-                            asyncCache.invoke(key, new IncrementCacheEntryProcessor(), cacheName());
+                            asyncCache.invoke(key, new IncrementWriteInvokeCacheEntryProcessor(), cacheName());
                             Object o = asyncCache.future().get(timeout);
 
                             if (o != null)
@@ -165,7 +165,7 @@ public class IgniteTransactionalWriteInvokeBenchmark extends IgniteFailoverAbstr
 
     /**
      */
-    private static class IncrementCacheEntryProcessor implements CacheEntryProcessor<String, Long, Object> {
+    private static class IncrementWriteInvokeCacheEntryProcessor implements CacheEntryProcessor<String, Long, Object> {
         /** */
         private static final long serialVersionUID = 0;
 


[37/50] [abbrv] ignite git commit: IGNITE-2064 More test fixes

Posted by sb...@apache.org.
IGNITE-2064 More test fixes


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

Branch: refs/heads/ignite-1.5.1
Commit: 8ca163bd0f06cc832df126733dbbe50cea35c2ac
Parents: 67ebd02
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Dec 8 15:46:10 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Dec 8 15:46:10 2015 +0300

----------------------------------------------------------------------
 .../GridCacheConditionalDeploymentSelfTest.java | 20 ++++++++++++--------
 .../cache/GridCacheDeploymentSelfTest.java      | 11 ++++++++---
 2 files changed, 20 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8ca163bd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConditionalDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConditionalDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConditionalDeploymentSelfTest.java
index c03eb08..4098225 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConditionalDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConditionalDeploymentSelfTest.java
@@ -18,11 +18,11 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.CO;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -139,19 +139,23 @@ public class GridCacheConditionalDeploymentSelfTest extends GridCommonAbstractTe
     public void testAddedDeploymentInfo2() throws Exception {
         GridCacheContext ctx = cacheContext();
 
-        assertTrue(ctx.deploymentEnabled());
+        if (grid(0).configuration().getMarshaller() instanceof BinaryMarshaller)
+            assertFalse(ctx.deploymentEnabled());
+        else {
+            assertTrue(ctx.deploymentEnabled());
 
-        GridCacheIoManager ioMgr = cacheIoManager();
+            GridCacheIoManager ioMgr = cacheIoManager();
 
-        TestMessage msg = new TestMessage();
+            TestMessage msg = new TestMessage();
 
-        assertNull(msg.deployInfo());
+            assertNull(msg.deployInfo());
 
-        msg.addDepInfo = false;
+            msg.addDepInfo = false;
 
-        IgniteUtils.invoke(GridCacheIoManager.class, ioMgr, "onSend", msg, grid(1).cluster().localNode().id());
+            IgniteUtils.invoke(GridCacheIoManager.class, ioMgr, "onSend", msg, grid(1).cluster().localNode().id());
 
-        assertNull(msg.deployInfo());
+            assertNull(msg.deployInfo());
+        }
     }
 
     protected GridCacheContext cacheContext() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8ca163bd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
index e18520d..aa64959 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DeploymentMode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -207,7 +208,9 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest {
             }
 
             assertEquals(0, g1.cache(null).localSize());
-            assertEquals(0, g2.cache(null).localSize());
+
+            assertEquals(g2.configuration().getMarshaller() instanceof BinaryMarshaller ? 1 : 0,
+                g2.cache(null).localSize());
 
             startGrid(3);
         }
@@ -223,6 +226,10 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest {
             depMode = CONTINUOUS;
 
             Ignite g1 = startGrid(1);
+
+            if (g1.configuration().getMarshaller() instanceof BinaryMarshaller)
+                fail("http://atlassian.gridgain.com/jira/browse/GG-10791");
+
             Ignite g2 = startGrid(2);
 
             Ignite g0 = startGrid(GRID_NAME);
@@ -250,10 +257,8 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest {
             stopGrid(GRID_NAME);
 
             assertEquals(1, g1.cache(null).localSize(CachePeekMode.ALL));
-            assertEquals(1, g1.cache(null).localSize(CachePeekMode.ALL));
 
             assertEquals(1, g2.cache(null).localSize(CachePeekMode.ALL));
-            assertEquals(1, g2.cache(null).localSize(CachePeekMode.ALL));
 
             startGrid(3);
         }


[17/50] [abbrv] ignite git commit: Fixed IgniteTransactionalWriteInvokeBenchmark

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


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

Branch: refs/heads/ignite-1.5.1
Commit: e5de23e9b61acc435ead4e70be9970a307906dbb
Parents: 11d668b
Author: ashutak <as...@gridgain.com>
Authored: Fri Dec 4 12:11:15 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Fri Dec 4 12:11:15 2015 +0300

----------------------------------------------------------------------
 .../IgniteTransactionalWriteInvokeBenchmark.java    | 16 +++++++---------
 1 file changed, 7 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e5de23e9/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
index 9f9b5f7..35befad 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
@@ -68,17 +68,15 @@ public class IgniteTransactionalWriteInvokeBenchmark extends IgniteFailoverAbstr
 
         long start = System.nanoTime();
 
-        if (cfg.memberId() == 0) {
-            try (IgniteDataStreamer<String, Long> dataLdr = ignite().dataStreamer(cacheName())) {
-                for (int k = 0; k < args.range() && !Thread.currentThread().isInterrupted(); k++) {
-                    dataLdr.addData("key-" + k + "-master", INITIAL_VALUE);
+        try (IgniteDataStreamer<String, Long> dataLdr = ignite().dataStreamer(cacheName())) {
+            for (int k = 0; k < args.range() && !Thread.currentThread().isInterrupted(); k++) {
+                dataLdr.addData("key-" + k + "-master", INITIAL_VALUE);
 
-                    for (int i = 0; i < args.keysCount(); i++)
-                        dataLdr.addData("key-" + k + "-" + i, INITIAL_VALUE);
+                for (int i = 0; i < args.keysCount(); i++)
+                    dataLdr.addData("key-" + k + "-" + i, INITIAL_VALUE);
 
-                    if (k % 100000 == 0)
-                        println(cfg, "Populated accounts: " + k);
-                }
+                if (k % 100000 == 0)
+                    println(cfg, "Populated accounts: " + k);
             }
         }
 


[11/50] [abbrv] ignite git commit: IGNITE-1270 Implement basic support for OSGi.

Posted by sb...@apache.org.
IGNITE-1270 Implement basic support for OSGi.

* OSGi manifests for most modules.
* Apache Karaf features file to facilitate deployment (along with dependencies).
* Make Ignite classloading OSGi-friendly and support different strategies.
* Several integration tests with Pax Exam.
* POM restructuring to promote 3rd party dep versions to build params.


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

Branch: refs/heads/ignite-1.5.1
Commit: 96e0802775fe94e9ba77fab147129142b809e432
Parents: 5beed16
Author: Raul Kripalani <ra...@apache.org>
Authored: Thu Dec 3 14:44:12 2015 +0000
Committer: Raul Kripalani <ra...@apache.org>
Committed: Thu Dec 3 14:44:12 2015 +0000

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 examples/schema-import/pom.xml                  |   2 +-
 modules/aop/pom.xml                             |  12 +-
 modules/aws/pom.xml                             |  18 +-
 modules/camel/pom.xml                           |  11 +-
 modules/core/pom.xml                            |  28 +-
 modules/flume/pom.xml                           |  16 +-
 modules/geospatial/pom.xml                      |  20 ++
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |  10 +-
 modules/indexing/pom.xml                        |  22 +-
 modules/jcl/pom.xml                             |  11 +
 modules/jms11/pom.xml                           |  20 +-
 modules/jta/pom.xml                             |  17 +
 modules/kafka/pom.xml                           |  16 +-
 modules/log4j/pom.xml                           |  10 +
 modules/log4j2/pom.xml                          |  11 +
 modules/mqtt/pom.xml                            |  24 +-
 modules/osgi-karaf/README.txt                   |  18 +
 modules/osgi-karaf/licenses/apache-2.0.txt      | 202 ++++++++++++
 modules/osgi-karaf/pom.xml                      |  85 +++++
 .../osgi-karaf/src/main/resources/features.xml  | 327 +++++++++++++++++++
 modules/osgi-paxlogging/README.txt              |  12 +
 modules/osgi-paxlogging/licenses/apache-2.0.txt | 202 ++++++++++++
 modules/osgi-paxlogging/pom.xml                 |  70 ++++
 modules/osgi/README.txt                         |  65 ++++
 modules/osgi/licenses/apache-2.0.txt            | 202 ++++++++++++
 modules/osgi/pom.xml                            | 171 ++++++++++
 .../IgniteAbstractOsgiContextActivator.java     | 238 ++++++++++++++
 .../org/apache/ignite/osgi/IgniteOsgiUtils.java |  69 ++++
 .../BundleDelegatingClassLoader.java            | 147 +++++++++
 .../classloaders/ContainerSweepClassLoader.java | 134 ++++++++
 .../OsgiClassLoadingStrategyType.java           |  29 ++
 .../ignite/osgi/AbstractIgniteKarafTest.java    | 109 +++++++
 .../IgniteKarafFeaturesInstallationTest.java    | 100 ++++++
 .../ignite/osgi/IgniteOsgiServiceTest.java      | 131 ++++++++
 .../apache/ignite/osgi/IgniteOsgiTestSuite.java |  32 ++
 .../activators/BasicIgniteTestActivator.java    |  76 +++++
 .../ignite/osgi/activators/TestOsgiFlags.java   |  53 +++
 .../osgi/activators/TestOsgiFlagsImpl.java      |  83 +++++
 modules/rest-http/pom.xml                       |  34 +-
 modules/scalar-2.10/pom.xml                     |   9 +-
 modules/scalar/pom.xml                          |   9 +-
 modules/schedule/pom.xml                        |  25 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |  12 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spring/pom.xml                          |  30 +-
 modules/ssh/pom.xml                             |  19 +-
 modules/twitter/pom.xml                         |  12 +-
 modules/urideploy/pom.xml                       |  17 +-
 modules/visor-console-2.10/pom.xml              |   4 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   4 +-
 modules/web/pom.xml                             |  13 +-
 modules/zookeeper/pom.xml                       |  14 +-
 parent/pom.xml                                  | 136 +++++++-
 pom.xml                                         |   3 +
 58 files changed, 3079 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index e7cd059..33c4f51 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>javax.cache</groupId>
             <artifactId>cache-api</artifactId>
-            <version>1.0.0</version>
+            <version>${javax.cache.version}</version>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/examples/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/examples/schema-import/pom.xml b/examples/schema-import/pom.xml
index a6b635c..d315c63 100644
--- a/examples/schema-import/pom.xml
+++ b/examples/schema-import/pom.xml
@@ -41,7 +41,7 @@
         <dependency>
             <groupId>javax.cache</groupId>
             <artifactId>cache-api</artifactId>
-            <version>1.0.0</version>
+            <version>${javax.cache.version}</version>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index f410f65..8598863 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -44,13 +44,13 @@
         <dependency>
             <groupId>org.aspectj</groupId>
             <artifactId>aspectjrt</artifactId>
-            <version>1.7.2</version>
+            <version>${aspectj.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.aspectj</groupId>
             <artifactId>aspectjweaver</artifactId>
-            <version>1.7.2</version>
+            <version>${aspectj.version}</version>
         </dependency>
 
         <dependency>
@@ -107,5 +107,13 @@
                 </excludes>
             </testResource>
         </testResources>
+        
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 53979cb..68bb4c8 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -44,25 +44,25 @@
         <dependency>
             <groupId>com.amazonaws</groupId>
             <artifactId>aws-java-sdk</artifactId>
-            <version>1.3.21.1</version>
+            <version>${aws.sdk.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.httpcomponents</groupId>
             <artifactId>httpclient</artifactId>
-            <version>4.2.3</version>
+            <version>${httpclient.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.httpcomponents</groupId>
             <artifactId>httpcore</artifactId>
-            <version>4.2.3</version>
+            <version>${httpcore.version}</version>
         </dependency>
 
         <dependency>
             <groupId>commons-codec</groupId>
             <artifactId>commons-codec</artifactId>
-            <version>1.6</version>
+            <version>${commons.codec.version}</version>
         </dependency>
 
         <dependency>
@@ -100,4 +100,14 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/camel/pom.xml
----------------------------------------------------------------------
diff --git a/modules/camel/pom.xml b/modules/camel/pom.xml
index 328780a..e1c20f2 100644
--- a/modules/camel/pom.xml
+++ b/modules/camel/pom.xml
@@ -35,7 +35,6 @@
     <url>http://ignite.apache.org</url>
 
     <properties>
-        <camel.version>2.16.0</camel.version>
         <guava.version>18.0</guava.version>
         <okhttp.version>2.5.0</okhttp.version>
     </properties>
@@ -98,4 +97,14 @@
 
     </dependencies>
 
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 907c553..88e1c25 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -43,13 +43,29 @@
 
     <properties>
         <ignite.update.notifier.product>apache-ignite</ignite.update.notifier.product>
+
+        <!-- Imports:
+                - com.sun.jmx.mbeanserver => only used from TCKMBeanServerBuilder which has no usages within Ignite's
+                  runtime codebase. Therefore, it's likely that code will not be hit during normal operation and we exclude it.
+                - javax.enterprise.util is optional.
+        -->
+        <osgi.import.package>
+            javax.enterprise.util;resolution:=optional,
+            !com.sun.jmx.mbeanserver,
+            *
+        </osgi.import.package>
+        <osgi.export.package>
+            org.apache.ignite.*,
+            org.jsr166.*;version=1.0.0;
+            {local-packages}
+        </osgi.export.package>
     </properties>
 
     <dependencies>
         <dependency>
             <groupId>javax.cache</groupId>
             <artifactId>cache-api</artifactId>
-            <version>1.0.0</version>
+            <version>${javax.cache.version}</version>
         </dependency>
 
         <dependency>
@@ -90,7 +106,7 @@
         <dependency>
             <groupId>com.h2database</groupId>
             <artifactId>h2</artifactId>
-            <version>1.3.175</version>
+            <version>${h2.version}</version>
             <scope>test</scope>
         </dependency>
 
@@ -139,7 +155,7 @@
         <dependency>
             <groupId>net.sf.json-lib</groupId>
             <artifactId>json-lib</artifactId>
-            <version>2.4</version>
+            <version>${jsonlib.version}</version>
             <classifier>jdk15</classifier>
             <scope>test</scope>
         </dependency>
@@ -276,6 +292,12 @@
                     </execution>
                 </executions>
             </plugin>
+
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
         </plugins>
     </build>
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/flume/pom.xml
----------------------------------------------------------------------
diff --git a/modules/flume/pom.xml b/modules/flume/pom.xml
index 0768410..8598248 100644
--- a/modules/flume/pom.xml
+++ b/modules/flume/pom.xml
@@ -34,10 +34,6 @@
     <version>1.5.0-b1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
-    <properties>
-        <flume-ng.version>1.6.0</flume-ng.version>
-    </properties>
-
     <dependencies>
         <dependency>
             <groupId>org.apache.ignite</groupId>
@@ -48,7 +44,7 @@
         <dependency>
             <groupId>org.apache.flume</groupId>
             <artifactId>flume-ng-core</artifactId>
-            <version>${flume-ng.version}</version>
+            <version>${flume.ng.version}</version>
         </dependency>
 
         <dependency>
@@ -73,4 +69,14 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index eefe279..229ecef 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -82,4 +82,24 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. 
+                 This bundle is a fragment attached to the ignite-core bundle, as it contains and exports classes in 
+                 the org.apache.ignite.internal.processors.query.h2.opt in the same manner as ignite-indexing, thus 
+                 leading to a split package situation in OSGi.
+            -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <configuration>
+                    <instructions>
+                        <Fragment-Host>org.apache.ignite.ignite-core</Fragment-Host>
+                    </instructions>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+    
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 75a9eea..df39996 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -104,7 +104,7 @@
         <dependency>
             <groupId>commons-beanutils</groupId>
             <artifactId>commons-beanutils</artifactId>
-            <version>1.8.3</version>
+            <version>${commons.beanutils.version}</version>
             <scope>test</scope>
         </dependency>
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index d5a1be4..766ca97 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -71,7 +71,7 @@
         <dependency>
             <groupId>com.h2database</groupId>
             <artifactId>h2</artifactId>
-            <version>1.3.175</version>
+            <version>${h2.version}</version>
             <scope>test</scope>
         </dependency>
 
@@ -134,5 +134,13 @@
                 </excludes>
             </testResource>
         </testResources>
+
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 13a11b4..e7a1c3b 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -44,19 +44,19 @@
         <dependency>
             <groupId>commons-codec</groupId>
             <artifactId>commons-codec</artifactId>
-            <version>1.6</version>
+            <version>${commons.codec.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.lucene</groupId>
             <artifactId>lucene-core</artifactId>
-            <version>3.5.0</version>
+            <version>${lucene.version}</version>
         </dependency>
 
         <dependency>
             <groupId>com.h2database</groupId>
             <artifactId>h2</artifactId>
-            <version>1.3.175</version>
+            <version>${h2.version}</version>
         </dependency>
 
         <dependency>
@@ -108,6 +108,22 @@
                     </execution>
                 </executions>
             </plugin>
+
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. 
+                 This bundle is a fragment attached to the ignite-core bundle, as it contains and exports classes in 
+                 the org.apache.ignite.internal.processors.query.h2.opt in the same manner as ignite-geospatial, thus 
+                 leading to a split package situation in OSGi. It also contains an internal processor.
+            -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <configuration>
+                    <instructions>
+                        <Fragment-Host>org.apache.ignite.ignite-core</Fragment-Host>
+                    </instructions>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
+    
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index a2ee45f..8678caa 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -55,4 +55,15 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+    
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/jms11/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jms11/pom.xml b/modules/jms11/pom.xml
index 4665ce9..2e2ffe6 100644
--- a/modules/jms11/pom.xml
+++ b/modules/jms11/pom.xml
@@ -34,10 +34,6 @@
     <version>1.5.0-b1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
-    <properties>
-        <activemq.version>5.12.0</activemq.version>
-    </properties>
-
     <dependencies>
         <dependency>
             <groupId>org.apache.ignite</groupId>
@@ -46,9 +42,9 @@
         </dependency>
 
         <dependency>
-            <groupId>javax.jms</groupId>
-            <artifactId>jms-api</artifactId>
-            <version>1.1-rev-1</version>
+            <groupId>org.apache.geronimo.specs</groupId>
+            <artifactId>geronimo-jms_1.1_spec</artifactId>
+            <version>${jms.spec.version}</version>
         </dependency>
 
         <dependency>
@@ -88,4 +84,14 @@
         </dependency>
     </dependencies>
 
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index 4724c4f..9dabe28 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -83,4 +83,21 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <!-- This is a fragment because it's a processor module. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <configuration>
+                    <instructions>
+                        <Fragment-Host>org.apache.ignite.ignite-core</Fragment-Host>
+                    </instructions>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+    
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index b62f177..443d73c 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -44,7 +44,7 @@
         <dependency>
             <groupId>org.apache.kafka</groupId>
             <artifactId>kafka_2.10</artifactId>
-            <version>0.8.2.1</version>
+            <version>${kafka.version}</version>
             <exclusions>
                 <exclusion>
                     <groupId>com.sun.jmx</groupId>
@@ -72,7 +72,7 @@
         <dependency>
             <groupId>org.apache.zookeeper</groupId>
             <artifactId>zookeeper</artifactId>
-            <version>3.4.5</version>
+            <version>${zookeeper.version}</version>
         </dependency>
 
         <dependency>
@@ -84,7 +84,7 @@
         <dependency>
             <groupId>org.ow2.asm</groupId>
             <artifactId>asm-all</artifactId>
-            <version>4.2</version>
+            <version>${asm.version}</version>
         </dependency>
 
         <dependency>
@@ -103,4 +103,14 @@
         </dependency>
     </dependencies>
 
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index f08217e..e27732e 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -54,4 +54,14 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+          </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/log4j2/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j2/pom.xml b/modules/log4j2/pom.xml
index db351cb..e195ce7 100644
--- a/modules/log4j2/pom.xml
+++ b/modules/log4j2/pom.xml
@@ -61,4 +61,15 @@
             <version>2.3</version>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+    
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/mqtt/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mqtt/pom.xml b/modules/mqtt/pom.xml
index a1ce973..c0b17e6 100644
--- a/modules/mqtt/pom.xml
+++ b/modules/mqtt/pom.xml
@@ -34,12 +34,6 @@
     <version>1.5.0-b1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
-    <properties>
-        <paho.version>1.0.2</paho.version>
-        <activemq.version>5.12.0</activemq.version>
-        <guava-retryier.version>2.0.0</guava-retryier.version>
-    </properties>
-
     <dependencies>
         <dependency>
             <groupId>org.apache.ignite</groupId>
@@ -56,7 +50,13 @@
         <dependency>
             <groupId>com.github.rholder</groupId>
             <artifactId>guava-retrying</artifactId>
-            <version>${guava-retryier.version}</version>
+            <version>${guava.retrying.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>${guava.version}</version>
         </dependency>
 
         <dependency>
@@ -110,4 +110,14 @@
         </repository>
     </repositories>
 
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi-karaf/README.txt
----------------------------------------------------------------------
diff --git a/modules/osgi-karaf/README.txt b/modules/osgi-karaf/README.txt
new file mode 100644
index 0000000..6bd1555
--- /dev/null
+++ b/modules/osgi-karaf/README.txt
@@ -0,0 +1,18 @@
+Apache Ignite OSGi Karaf Integration Module
+-------------------------------------------
+
+This module contains a feature repository to facilitate installing Apache Ignite into an Apache Karaf container.
+
+Use the following Karaf command:
+
+    karaf@root()> feature:repo-add mvn:org.apache.ignite/ignite-osgi-karaf/${ignite.version}/xml/features
+
+Replacing ${ignite.version} with the Apache Ignite version you woudl like to install.
+
+You may now list the Ignite features that are available for installation:
+
+    karaf@root()> feature:list | grep ignite
+
+Each feature installs the corresponding ignite module + its dependencies.
+
+We include an global feature with name 'ignite-all' that collectively installs all Ignite features at once.

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi-karaf/pom.xml
----------------------------------------------------------------------
diff --git a/modules/osgi-karaf/pom.xml b/modules/osgi-karaf/pom.xml
new file mode 100644
index 0000000..f1ac66b
--- /dev/null
+++ b/modules/osgi-karaf/pom.xml
@@ -0,0 +1,85 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+  
+       http://www.apache.org/licenses/LICENSE-2.0
+       
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!--
+    POM file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
+    <groupId>org.apache.ignite</groupId>
+    <artifactId>ignite-osgi-karaf</artifactId>
+    <version>1.5.0-SNAPSHOT</version>
+    <packaging>pom</packaging>
+
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+                <filtering>true</filtering>
+            </resource>
+        </resources>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-resources-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>filter</id>
+                        <phase>generate-resources</phase>
+                        <goals>
+                            <goal>resources</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>1.9.1</version>
+                <executions>
+                    <execution>
+                        <id>attach-artifacts</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>attach-artifact</goal>
+                        </goals>
+                        <configuration>
+                            <artifacts>
+                                <artifact>
+                                    <file>target/classes/features.xml</file>
+                                    <type>xml</type>
+                                    <classifier>features</classifier>
+                                </artifact>
+                            </artifacts>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi-karaf/src/main/resources/features.xml
----------------------------------------------------------------------
diff --git a/modules/osgi-karaf/src/main/resources/features.xml b/modules/osgi-karaf/src/main/resources/features.xml
new file mode 100644
index 0000000..4dca39b
--- /dev/null
+++ b/modules/osgi-karaf/src/main/resources/features.xml
@@ -0,0 +1,327 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<features name="ignite" xmlns="http://karaf.apache.org/xmlns/features/v1.3.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://karaf.apache.org/xmlns/features/v1.3.0 http://karaf.apache.org/xmlns/features/v1.3.0">
+
+    <feature name="ignite-all" version="${project.version}" description="Apache Ignite :: All">
+        <details>
+            <![CDATA[Aggregate feature for installing all Apache Ignite module + their dependencies.
+            
+            NOTE: Due to a bug in Apache Karaf (KARAF-4129), you must install the ignite-log4j feature explicitly and then either:
+            - manually refresh the pax-logging-api framework bundle  - or - 
+            - restart the Apache Karaf container.
+            You may safely ignore the 'Resource has no uri' exception if you follow this method.]]>
+        </details>
+        <feature>ignite-core</feature>
+        <feature>ignite-aop</feature>
+        <feature>ignite-aws</feature>
+        <feature>ignite-camel</feature>
+        <feature>ignite-flume</feature>
+        <feature>ignite-indexing</feature>
+        <feature>ignite-hibernate</feature>
+        <feature>ignite-jcl</feature>
+        <feature>ignite-jms11</feature>
+        <feature>ignite-jta</feature>
+        <feature>ignite-kafka</feature>
+        <feature>ignite-mqtt</feature>
+        <!-- KARAF-4129 Karaf feature containing a fragment that attaches to pax-logging-api cannot be installed
+        <feature>ignite-log4j</feature>-->
+        <feature>ignite-rest-http</feature>
+        <feature>ignite-scalar-2.11</feature>
+        <feature>ignite-schedule</feature>
+        <feature>ignite-slf4j</feature>
+        <feature>ignite-spring</feature>
+        <feature>ignite-ssh</feature>
+        <feature>ignite-twitter</feature>
+        <feature>ignite-urideploy</feature>
+        <feature>ignite-web</feature>
+        <feature>ignite-zookeeper</feature>
+    </feature>
+
+    <feature name="ignite-core" version="${project.version}" description="Apache Ignite :: Core">
+        <details><![CDATA[The Apache Ignite core module. This feature also installs the JCache 1.0 specification API.]]></details>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.javax-cache-api/${javax.cache.bundle.version}</bundle>
+        <bundle dependency="true">mvn:org.apache.ignite/ignite-osgi/${project.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-core/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-aop" version="${project.version}" description="Apache Ignite :: AOP">
+        <details><![CDATA[The Apache Ignite AOP module + dependencies.]]></details>
+        <feature dependency="true" version="[${spring41.osgi.feature.version},4.2)">spring-aspects</feature>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.aopalliance/${aopalliance.bundle.version}</bundle>        
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.aspectj/${aspectj.bundle.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-aop/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-aws" version="${project.version}" description="Apache Ignite :: AWS">
+        <details><![CDATA[The Apache Ignite AWS module + dependencies.]]></details>
+        <bundle start="true" dependency="true">mvn:org.apache.httpcomponents/httpcore-osgi/${httpcore.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.httpcomponents/httpclient-osgi/${httpclient.version}</bundle>
+        <bundle start="true" dependency="true">mvn:commons-codec/commons-codec/${commons.codec.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.aws-java-sdk/${aws.sdk.bundle.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-aws/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-camel" version="${project.version}" description="Apache Ignite :: Camel">
+        <details>
+            <![CDATA[The Apache Ignite Camel module.
+
+            Make sure to install the Apache Camel feature repository before installing this feature.
+
+                mvn:org.apache.camel.karaf/apache-camel/\${camel.version}/xml/features
+            
+            Installing this feature will trigger the installation of the 'camel-core' feature from the Camel repository.]]>
+        </details>
+        <feature>camel-core</feature>
+        <bundle start="true">mvn:org.apache.ignite/ignite-camel/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-flume" version="${project.version}" description="Apache Ignite :: Flume">
+        <details>
+            <![CDATA[The Apache Ignite Flume module + dependencies.]]>
+        </details>
+        <feature prerequisite="true">wrap</feature>
+        <bundle start="true" dependency="true">wrap:mvn:org.apache.flume/flume-ng-core/${flume.ng.version}$Bundle-SymbolicName=flume-ng-core&amp;Bundle-Version=${flume.ng.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-flume/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-hibernate" version="${project.version}" description="Apache Ignite :: Hibernate">
+        <details>
+            <![CDATA[The Apache Ignite Hibernate module + dependencies. 
+            
+            Installing this feature will trigger the installation of the 'hibernate' feature from the Apache Karaf distribution.]]>
+        </details>
+        <feature>hibernate</feature>
+        <bundle start="true">mvn:org.apache.ignite/ignite-hibernate/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-indexing" version="${project.version}" description="Apache Ignite :: Indexing">
+        <details>
+            <![CDATA[The Apache Ignite Indexing module + dependencies. This module is a fragment of ignite-core. 
+            
+            Be sure to refresh ignite-core manually in case it is not refreshed automatically.]]>
+        </details>
+        <bundle start="true" dependency="true">mvn:org.osgi/org.osgi.enterprise/${osgi.enterprise.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.lucene/${lucene.bundle.version}</bundle>
+        <bundle start="true" dependency="true">mvn:com.h2database/h2/${h2.version}</bundle>
+        <bundle start="false">mvn:org.apache.ignite/ignite-indexing/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-jcl" version="${project.version}" description="Apache Ignite :: JCL">
+        <details>
+            <![CDATA[The Apache Ignite JCL integration module. In Apache Karaf, this module will integrate with Pax Logging.]]>
+        </details>
+        <bundle start="true">mvn:org.apache.ignite/ignite-jcl/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-jms11" version="${project.version}" description="Apache Ignite :: JMS 1.1">
+        <details>
+            <![CDATA[The Apache Ignite JMS 1.1 module. Make sure to install your broker's JMS client bundle as well.]]>
+        </details>
+        <bundle start="true" dependency="true">mvn:org.apache.geronimo.specs/geronimo-jms_1.1_spec/${jms.spec.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-jms11/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-jta" version="${project.version}" description="Apache Ignite :: JTA">
+        <details>
+            <![CDATA[The Apache Ignite JTA module + dependencies. This module is a fragment of ignite-core. 
+            
+            Be sure to refresh ignite-core manually in case it is not refreshed automatically.
+            
+            Installing this feature will trigger the installation of the 'transaction' feature from the Apache Karaf distribution.]]>
+        </details>
+        <feature dependency="true">transaction</feature>
+        <bundle start="true">mvn:org.apache.ignite/ignite-jta/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-kafka" version="${project.version}" description="Apache Ignite :: Kafka">
+        <details>
+            <![CDATA[The Apache Ignite Kafka module + dependencies. This module installs the Scala 2.10 library bundle.]]>
+        </details>
+        <feature prerequisite="true">wrap</feature>
+        <bundle start="true" dependency="true">mvn:org.scala-lang/scala-library/${scala210.library.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.zookeeper/zookeeper/${zookeeper.version}</bundle>
+        <bundle start="true" dependency="true">wrap:mvn:com.101tec/zkclient/${zkclient.version}$Bundle-SymbolicName=zkclient&amp;Bundle-Version=${zkclient.version}&amp;Export-Package=*;-noimport:=true;version=${zkclient.version}</bundle>
+        <bundle start="true" dependency="true">wrap:mvn:com.yammer.metrics/metrics-core/${yammer.metrics.core.version}$Bundle-SymbolicName=yammer-metrics-core&amp;Bundle-Version=2.2.0&amp;Export-Package=*;-noimport:=true;version=${yammer.metrics.core.version}</bundle>
+        <bundle start="true" dependency="true">wrap:mvn:com.yammer.metrics/metrics-annotation/${yammer.metrics.annotation.version}$Bundle-SymbolicName=yammer-metrics-annotation&amp;Bundle-Version=2.2.0&amp;Export-Package=*;-noimport:=true;version=${yammer.metrics.annotation.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.kafka-clients/${kafka.clients.bundle.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.kafka_2.10/${kafka.bundle.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-kafka/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-log4j" version="${project.version}" description="Apache Ignite :: log4j">
+        <details>
+            <![CDATA[The Apache Ignite log4j module + dependencies. 
+            
+            This module installs an OSGi fragment that exposes extra packages from the Pax Logging bundle required by Ignite. 
+            
+            Be sure to refresh the Pax Logging bundles manually in case this does not happen automatically.]]>
+        </details>
+        <bundle dependency="true" start-level="8">mvn:org.apache.ignite/ignite-osgi-paxlogging/${project.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-log4j/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-mqtt" version="${project.version}" description="Apache Ignite :: MQTT">
+        <details>
+            <![CDATA[The Apache Ignite MQTT module + dependencies.]]>
+        </details>
+        <feature prerequisite="true">wrap</feature>
+        <bundle start="true" dependency="true">mvn:com.google.guava/guava/${guava.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.eclipse.paho/org.eclipse.paho.client.mqttv3/${paho.version}</bundle>
+        <bundle start="true" dependency="true">wrap:mvn:com.github.rholder/guava-retrying/${guava.retrying.version}$Bundle-SymbolicName=guava-retrying&amp;Bundle-SymbolicName=guava-retrying&amp;Bundle-Version=${guava.retrying.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-mqtt/${project.version}</bundle>
+    </feature>   
+
+    <feature name="ignite-rest-http" version="${project.version}" description="Apache Ignite :: REST HTTP">
+         <!-- NOTICE: XOM cannot be included by default due to an incompatible license; 
+                      please review its license model and install the dependency manually if you agree. -->
+        <details>
+            <![CDATA[The Apache Ignite REST HTTP module + dependencies. 
+            
+            Installing this feature will trigger the installation of the 'http' feature from the Apache Karaf distribution.
+            
+            NOTE: Before using this feature you must review the license of the XOM bundle and install it manually if you accept it: 
+            install -s mvn:xom/xom/1.2.5]]>
+        </details>
+        <feature dependency="true">http</feature>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.commons-beanutils/${commons.beanutils.bundle.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.oro/${oro.bundle.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.ezmorph/${ezmorph.bundle.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.json-lib/${jsonlib.bundle.version}</bundle>
+        <bundle start="true" dependency="true">mvn:commons-lang/commons-lang/${commons.lang.version}</bundle>
+        <bundle start="true" dependency="true">mvn:commons-collections/commons-collections/${commons.collections.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-rest-http/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-scalar-2.10" version="${project.version}" description="Apache Ignite :: Scala 2.10">
+        <details>
+            <![CDATA[The Apache Ignite Scala 2.11 integration module + dependencies. This module installs the Scala 2.10 library bundle.]]>
+        </details>
+        <bundle start="true" dependency="true">mvn:org.scala-lang/scala-library/${scala210.library.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-scalar_2.10/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-scalar-2.11" version="${project.version}" description="Apache Ignite :: Scala 2.11">
+        <details>
+            <![CDATA[The Apache Ignite Scala 2.11 integration module + dependencies. This module installs the Scala 2.11 library bundle.]]>
+        </details>
+        <bundle start="true" dependency="true">mvn:org.scala-lang/scala-library/${scala211.library.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-scalar/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-schedule" version="${project.version}" description="Apache Ignite :: Schedule">
+        <details>
+            <![CDATA[The Apache Ignite Schedule module + dependencies. This module is a fragment of ignite-core.]]>
+        </details>
+        <feature prerequisite="true">wrap</feature>
+        <bundle start="true" dependency="true">wrap:mvn:it.sauronsoftware.cron4j/cron4j/${cron4j.version}$Bundle-SymbolicName=cron4j&amp;Bundle-Version=${cron4j.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-schedule/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-slf4j" version="${project.version}" description="Apache Ignite :: slf4j">
+        <details>
+            <![CDATA[The Apache Ignite slf4j module.]]>
+        </details>
+        <bundle start="true">mvn:org.apache.ignite/ignite-slf4j/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-spring" version="${project.version}" description="Apache Ignite :: Spring Support">
+        <details>
+            <![CDATA[The Apache Ignite Spring integration module + dependencies. This module is a fragment of ignite-core. 
+            
+            Be sure to refresh ignite-core in case it is not refreshed automatically.
+            
+            This feature installs the following features from the Apache Karaf distribution:
+              - spring
+              - spring-aspects
+              - spring-tx
+              - spring-jdbc
+              
+            With version range: [${spring41.osgi.feature.version},4.2).]]>
+        </details>
+        <feature dependency="true" version="[${spring41.osgi.feature.version},4.2)">spring</feature>
+        <feature dependency="true" version="[${spring41.osgi.feature.version},4.2)">spring-aspects</feature>
+        <feature dependency="true" version="[${spring41.osgi.feature.version},4.2)">spring-tx</feature>
+        <feature dependency="true" version="[${spring41.osgi.feature.version},4.2)">spring-jdbc</feature>
+        <bundle start="true">mvn:org.apache.ignite/ignite-spring/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-ssh" version="${project.version}" description="Apache Ignite :: SSH">
+        <details>
+            <![CDATA[The Apache Ignite SSH module + dependencies.]]>
+        </details>
+        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.jsch/${jsch.bundle.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-ssh/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-twitter" version="${project.version}" description="Apache Ignite :: Twitter">
+        <details>
+            <![CDATA[The Apache Ignite Twitter module + dependencies.]]>
+        </details>
+        <feature prerequisite="true">wrap</feature>
+        <bundle start="true" dependency="true">mvn:com.google.guava/guava/${guava14.version}</bundle>
+        <bundle start="true" dependency="true">wrap:mvn:com.twitter/hbc-core/${twitter.hbc.version}$Bundle-SymbolicName=Hosebird Client Core&amp;Bundle-Version=${twitter.hbc.version}</bundle>
+        <bundle start="true" dependency="true">wrap:mvn:com.twitter/hbc-twitter4j/${twitter.hbc.version}$Bundle-SymbolicName=Hosebird Client Twitter4J&amp;Bundle-Version=${twitter.hbc.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-twitter/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-urideploy" version="${project.version}" description="Apache Ignite :: URI Deploy">
+        <details>
+            <![CDATA[The Apache Ignite URI Deploy module + dependencies.
+            
+            This feature installs the following features from the Apache Karaf distribution:
+              - spring
+              - spring-aspects
+              - spring-tx
+              
+            With version range: [${spring41.osgi.feature.version},4.2).]]>
+        </details>
+        <feature prerequisite="true">wrap</feature>
+        <feature dependency="true" version="[${spring41.osgi.feature.version},4.2)">spring</feature>
+        <feature dependency="true" version="[${spring41.osgi.feature.version},4.2)">spring-aspects</feature>
+        <feature dependency="true" version="[${spring41.osgi.feature.version},4.2)">spring-tx</feature>
+        <bundle start="true" dependency="true">wrap:mvn:net.sf.jtidy/jtidy/${jtidy.version}$Bundle-SymbolicName=JTidy&amp;Bundle-Version=938</bundle>
+        <bundle start="true" dependency="true">mvn:commons-codec/commons-codec/${commons.codec.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-urideploy/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-web" version="${project.version}" description="Apache Ignite :: Web">
+        <details>
+            <![CDATA[The Apache Ignite Web module + dependencies.
+            
+            This feature installs the 'http' feature from the Apache Karaf distribution.]]>
+        </details>
+        <feature dependency="true">http</feature>
+        <bundle start="true">mvn:org.apache.ignite/ignite-web/${project.version}</bundle>
+    </feature>
+
+    <feature name="ignite-zookeeper" version="${project.version}" description="Apache Ignite :: ZooKeeper">
+        <details>
+            <![CDATA[The Apache Ignite ZooKeeper module + dependencies.]]>
+        </details>
+        <bundle start="true" dependency="true">mvn:com.google.guava/guava/${guava16.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.codehaus.jackson/jackson-core-asl/${jackson.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.codehaus.jackson/jackson-mapper-asl/${jackson.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.curator/curator-client/${curator.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.curator/curator-framework/${curator.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.curator/curator-recipes/${curator.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.apache.curator/curator-x-discovery/${curator.version}</bundle>
+        <bundle start="true">mvn:org.apache.ignite/ignite-zookeeper/${project.version}</bundle>
+    </feature>
+
+</features>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi-paxlogging/README.txt
----------------------------------------------------------------------
diff --git a/modules/osgi-paxlogging/README.txt b/modules/osgi-paxlogging/README.txt
new file mode 100644
index 0000000..f6346f1
--- /dev/null
+++ b/modules/osgi-paxlogging/README.txt
@@ -0,0 +1,12 @@
+Apache Ignite OSGi Pax Logging Fragment Module
+----------------------------------------------
+
+This module is an OSGi fragment that exposes the following packages from the Pax Logging API bundle:
+
+  - org.apache.log4j.varia
+  - org.apache.log4j.xml
+
+These packages are required when installing the ignite-log4j bundle, and are not exposed by default
+by the Pax Logging API - the logging framework used by Apache Karaf.
+
+This fragment exposes them.

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi-paxlogging/pom.xml
----------------------------------------------------------------------
diff --git a/modules/osgi-paxlogging/pom.xml b/modules/osgi-paxlogging/pom.xml
new file mode 100644
index 0000000..bd9e18e
--- /dev/null
+++ b/modules/osgi-paxlogging/pom.xml
@@ -0,0 +1,70 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!--
+    POM file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
+    <groupId>org.apache.ignite</groupId>
+    <artifactId>ignite-osgi-paxlogging</artifactId>
+    <version>1.5.0-SNAPSHOT</version>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <version>${maven.bundle.plugin.version}</version>
+                <extensions>true</extensions>
+                <configuration>
+                    <instructions>
+                        <Fragment-Host>org.ops4j.pax.logging.pax-logging-api</Fragment-Host>
+                        <Export-Package>
+                            org.apache.log4j.varia;-noimport:=true,
+                            org.apache.log4j.xml;-noimport:=true
+                        </Export-Package>
+                        <Import-Package>!*</Import-Package>
+                        <_invalidfilenames />
+                        <_nodefaultversion>true</_nodefaultversion>
+                    </instructions>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/modules/osgi/README.txt
----------------------------------------------------------------------
diff --git a/modules/osgi/README.txt b/modules/osgi/README.txt
new file mode 100644
index 0000000..35b133d
--- /dev/null
+++ b/modules/osgi/README.txt
@@ -0,0 +1,65 @@
+Apache Ignite OSGi Integration Module
+-------------------------------------
+
+This module provides the bridging components to make Apache Ignite run seamlessly inside an OSGi container
+like Apache Karaf. It provides a Bundle Activator to initialize Ignite, along with different classloaders
+facilitate class resolution within an OSGi environment.
+
+If using Ignite within Apache Karaf, please refer to the osgi-karaf and osgi-paxlogging modules too:
+
+  - osgi-karaf contains a feature repository to facilitate installing Ignite into a Karaf container.
+  - osgi-paxlogging contains an OSGi fragment required to make pax-logging-api expose certain log4j packages
+    required by ignite-log4j
+
+Importing the ignite-osgi module in a Maven project
+---------------------------------------------------
+
+If you are using Maven to manage dependencies of your project, you can add the ignite-osgi module
+dependency like this (replace '${ignite.version}' with actual Ignite version you are interested in):
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+                        http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    ...
+    <dependencies>
+        ...
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-osgi</artifactId>
+            <version>${ignite.version}</version>
+        </dependency>
+        ...
+    </dependencies>
+    ...
+</project>
+
+Running the tests in this module
+--------------------------------
+
+We use the Pax Exam framework to fire up an Apache Karaf container (forked process) in order to execute the OSGi tests.
+
+Bundles are provisioned into the container via mvn: URLs. For this to work, you must have run a full build from the
+top directory of the Ignite source tree, including the install goal, which provisions the modules into your local
+Maven repository:
+
+   mvn clean install -Plgpl
+
+Neither compiling and running the tests, nor generating Javadocs are necessary. To disable these steps,
+use these switches:
+
+   -DskipTests -Dmaven.test.skip=true -Dmaven.javadoc.skip=true
+
+You may then run the OSGi test suite:
+
+   mvn test -Dtest=IgniteOsgiTestSuite
+
+NOTE: This test uses environment variables set by the maven-surefire-plugin configuration. If you are running the
+test suite from within an IDE, either run it via Maven or set these environment variables manually in your
+Run/Debug configuration:
+
+  - projectVersion
+  - karafVersion
+  - camelVersion
+
+See the pom.xml file of this module to understand which values to set.


[16/50] [abbrv] ignite git commit: IGNITE-2019 Fixed schema import on Mac OSX El Capitan. - Fixes #292.

Posted by sb...@apache.org.
IGNITE-2019 Fixed schema import on Mac OSX El Capitan. - Fixes #292.

Signed-off-by: Andrey <an...@gridgain.com>


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

Branch: refs/heads/ignite-1.5.1
Commit: 11d668b2f77b12e7dc9bbc8ebdbfeb5e2ddad716
Parents: e6f6f40
Author: Andrey <an...@gridgain.com>
Authored: Fri Dec 4 15:52:14 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Fri Dec 4 15:52:14 2015 +0700

----------------------------------------------------------------------
 .../org/apache/ignite/schema/ui/SchemaImportApp.java  | 14 ++++++++++++++
 1 file changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/11d668b2/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 495c316..6f9e05b 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
@@ -22,6 +22,7 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.lang.reflect.Field;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.sql.Connection;
@@ -1738,6 +1739,19 @@ public class SchemaImportApp extends Application {
             catch (Exception ignore) {
                 // No-op.
             }
+
+            // Workaround for JDK 7/JavaFX 2 application on Mac OSX El Capitan.
+            try {
+                Class<?> fontFinderCls = Class.forName("com.sun.t2k.MacFontFinder");
+
+                Field psNameToPathMap = fontFinderCls.getDeclaredField("psNameToPathMap");
+
+                psNameToPathMap.setAccessible(true);
+                psNameToPathMap.set(null, new HashMap<String, String>());
+            }
+            catch (Exception ignore) {
+                // No-op.
+            }
         }
 
         launch(args);


[32/50] [abbrv] ignite git commit: ignite-1.5 Fixed hang on client reconnect (should not do blocking calls from reconnect callback)

Posted by sb...@apache.org.
ignite-1.5 Fixed hang on client reconnect (should not do blocking calls from reconnect callback)


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

Branch: refs/heads/ignite-1.5.1
Commit: d5791837890a70e1777b86aab281245701afe1eb
Parents: 3b26859
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 8 12:42:25 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 8 12:42:25 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/GridComponent.java   |  3 +-
 .../ignite/internal/GridPluginComponent.java    |  4 +-
 .../apache/ignite/internal/IgniteKernal.java    | 18 +++-
 .../internal/managers/GridManagerAdapter.java   |  5 +-
 .../deployment/GridDeploymentManager.java       |  5 +-
 .../processors/GridProcessorAdapter.java        |  5 +-
 .../processors/cache/GridCacheContext.java      |  6 +-
 .../processors/cache/GridCacheProcessor.java    | 26 +++++-
 .../datastructures/DataStructuresProcessor.java |  4 +-
 .../IgniteClientReconnectAbstractTest.java      | 95 +++++++++++++++++---
 .../IgniteClientReconnectAtomicsTest.java       | 57 ++++++++++++
 .../IgniteClientReconnectCacheTest.java         |  5 +-
 .../IgniteClientReconnectCollectionsTest.java   | 51 +++++++++++
 13 files changed, 254 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
index 6078c5d..0e234cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
@@ -131,6 +131,7 @@ public interface GridComponent {
      *
      * @param clusterRestarted Cluster restarted flag.
      * @throws IgniteCheckedException If failed.
+     * @return Future to wait before completing reconnect future.
      */
-    public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException;
+    @Nullable public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
index ac2a3a7..89dc243 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
@@ -70,8 +70,8 @@ public class GridPluginComponent implements GridComponent {
     }
 
     /** {@inheritDoc} */
-    @Override public void onReconnected(boolean clusterRestarted) {
-        // No-op.
+    @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) {
+        return null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 87ccf93..ab62c13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -136,6 +136,7 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridTimerTask;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
@@ -3083,16 +3084,27 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     /**
      * @param clusterRestarted {@code True} if all cluster nodes restarted while client was disconnected.
      */
+    @SuppressWarnings("unchecked")
     public void onReconnected(final boolean clusterRestarted) {
         Throwable err = null;
 
         try {
             ctx.disconnected(false);
 
-            for (GridComponent comp : ctx.components())
-                comp.onReconnected(clusterRestarted);
+            GridCompoundFuture<?, ?> reconnectFut = new GridCompoundFuture<>();
+
+            for (GridComponent comp : ctx.components()) {
+                IgniteInternalFuture<?> fut = comp.onReconnected(clusterRestarted);
+
+                if (fut != null)
+                    reconnectFut.add((IgniteInternalFuture)fut);
+            }
+
+            reconnectFut.add((IgniteInternalFuture)ctx.cache().context().exchange().reconnectExchangeFuture());
+
+            reconnectFut.markInitialized();
 
-            ctx.cache().context().exchange().reconnectExchangeFuture().listen(new CI1<IgniteInternalFuture<?>>() {
+            reconnectFut.listen(new CI1<IgniteInternalFuture<?>>() {
                 @Override public void apply(IgniteInternalFuture<?> fut) {
                     try {
                         fut.get();

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
index 1fd5bff..21a80c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
@@ -32,6 +32,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.GridComponent;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.timeout.GridSpiTimeoutObject;
@@ -192,9 +193,11 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
     }
 
     /** {@inheritDoc} */
-    @Override public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
         for (T t : spis)
             t.onClientReconnected(clusterRestarted);
+
+        return null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManager.java
index a2da75c..cea1786 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManager.java
@@ -27,6 +27,7 @@ import org.apache.ignite.compute.ComputeTask;
 import org.apache.ignite.compute.ComputeTaskName;
 import org.apache.ignite.configuration.DeploymentMode;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.managers.GridManagerAdapter;
 import org.apache.ignite.internal.managers.deployment.protocol.gg.GridProtocolHandler;
 import org.apache.ignite.internal.processors.task.GridInternal;
@@ -123,8 +124,10 @@ public class GridDeploymentManager extends GridManagerAdapter<DeploymentSpi> {
     }
 
     /** {@inheritDoc} */
-    @Override public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
         storesOnKernalStart();
+
+        return null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
index f7f42bd..e4896fd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
@@ -23,6 +23,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteFuture;
@@ -68,8 +69,8 @@ public abstract class GridProcessorAdapter implements GridProcessor {
     }
 
     /** {@inheritDoc} */
-    @Override public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
-        // No-op.
+    @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+        return null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index d689ba6..07f6b9a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -242,7 +242,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     private boolean depEnabled;
 
     /** */
-    private boolean deferredDelete;
+    private boolean deferredDel;
 
     /**
      * Empty constructor required for {@link Externalizable}.
@@ -512,7 +512,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     public void cache(GridCacheAdapter<K, V> cache) {
         this.cache = cache;
 
-        deferredDelete = cache.isDht() || cache.isDhtAtomic() || cache.isColocated() ||
+        deferredDel = cache.isDht() || cache.isDhtAtomic() || cache.isColocated() ||
             (cache.isNear() && cache.configuration().getAtomicityMode() == ATOMIC);
     }
 
@@ -576,7 +576,7 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @return {@code True} if entries should not be deleted from cache immediately.
      */
     public boolean deferredDelete() {
-        return deferredDelete;
+        return deferredDel;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index e53f186..02e6403 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -96,6 +96,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManag
 import org.apache.ignite.internal.processors.plugin.CachePluginManager;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.F0;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -955,10 +956,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
         List<GridCacheAdapter> reconnected = new ArrayList<>(caches.size());
 
-        for (GridCacheAdapter cache : caches.values()) {
+        GridCompoundFuture<?, ?> stopFut = null;
+
+        for (final GridCacheAdapter cache : caches.values()) {
             String name = cache.name();
 
             boolean stopped;
@@ -985,8 +988,17 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 caches.remove(maskNull(cache.name()));
                 jCacheProxies.remove(maskNull(cache.name()));
 
-                onKernalStop(cache, true);
-                stopCache(cache, true);
+                IgniteInternalFuture<?> fut = ctx.closure().runLocalSafe(new Runnable() {
+                    @Override public void run() {
+                        onKernalStop(cache, true);
+                        stopCache(cache, true);
+                    }
+                });
+
+                if (stopFut == null)
+                    stopFut = new GridCompoundFuture<>();
+
+                stopFut.add((IgniteInternalFuture)fut);
             }
             else {
                 cache.onReconnected();
@@ -1008,6 +1020,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cache.context().gate().reconnected(false);
 
         cachesOnDisconnect = null;
+
+        if (stopFut != null)
+            stopFut.markInitialized();
+
+        return stopFut;
     }
 
     /**
@@ -1200,6 +1217,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param pluginMgr Cache plugin manager.
      * @param cacheType Cache type.
      * @param cacheObjCtx Cache object context.
+     * @param updatesAllowed Updates allowed flag.
      * @return Cache context.
      * @throws IgniteCheckedException If failed to create cache.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 9ed9350..51c4067 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -276,7 +276,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
         for (Map.Entry<GridCacheInternal, GridCacheRemovable> e : dsMap.entrySet()) {
             GridCacheRemovable obj = e.getValue();
 
@@ -291,6 +291,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         for (GridCacheContext cctx : ctx.cache().context().cacheContexts())
             cctx.dataStructures().onReconnected(clusterRestarted);
+
+        return null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
index 0c1df7f..180047a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
@@ -19,8 +19,10 @@ package org.apache.ignite.internal;
 
 import java.io.IOException;
 import java.net.Socket;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -99,6 +101,14 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
      * @throws Exception If failed.
      */
     protected void waitReconnectEvent(CountDownLatch latch) throws Exception {
+        waitReconnectEvent(log, latch);
+    }
+
+    /**
+     * @param latch Latch.
+     * @throws Exception If failed.
+     */
+    protected static void waitReconnectEvent(IgniteLogger log, CountDownLatch latch) throws Exception {
         if (!latch.await(RECONNECT_TIMEOUT, MILLISECONDS)) {
             log.error("Failed to wait for reconnect event, will dump threads, latch count: " + latch.getCount());
 
@@ -124,7 +134,7 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
      * @param ignite Node.
      * @return Discovery SPI.
      */
-    protected TestTcpDiscoverySpi spi(Ignite ignite) {
+    protected static TestTcpDiscoverySpi spi(Ignite ignite) {
         return ((TestTcpDiscoverySpi)ignite.configuration().getDiscoverySpi());
     }
 
@@ -201,18 +211,38 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
      */
     protected void reconnectClientNode(Ignite client, Ignite srv, @Nullable Runnable disconnectedC)
         throws Exception {
-        reconnectClientNodes(Collections.singletonList(client), srv, disconnectedC);
+        reconnectClientNodes(log, Collections.singletonList(client), srv, disconnectedC);
     }
 
     /**
      * Reconnect client node.
      *
+     * @param log  Logger.
+     * @param client Client.
+     * @param srv Server.
+     * @param disconnectedC Closure which will be run when client node disconnected.
+     * @throws Exception If failed.
+     */
+    public static void reconnectClientNode(IgniteLogger log,
+        Ignite client,
+        Ignite srv,
+        @Nullable Runnable disconnectedC)
+        throws Exception {
+        reconnectClientNodes(log, Collections.singletonList(client), srv, disconnectedC);
+    }
+
+    /**
+     * Reconnect client node.
+     *
+     * @param log  Logger.
      * @param clients Clients.
      * @param srv Server.
      * @param disconnectedC Closure which will be run when client node disconnected.
      * @throws Exception If failed.
      */
-    protected void reconnectClientNodes(List<Ignite> clients, Ignite srv, @Nullable Runnable disconnectedC)
+    protected static void reconnectClientNodes(final IgniteLogger log,
+        List<Ignite> clients, Ignite srv,
+        @Nullable Runnable disconnectedC)
         throws Exception {
         final TestTcpDiscoverySpi srvSpi = spi(srv);
 
@@ -227,12 +257,12 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
         IgnitePredicate<Event> p = new IgnitePredicate<Event>() {
             @Override public boolean apply(Event evt) {
                 if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
-                    info("Disconnected: " + evt);
+                    log.info("Disconnected: " + evt);
 
                     disconnectLatch.countDown();
                 }
                 else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
-                    info("Reconnected: " + evt);
+                    log.info("Reconnected: " + evt);
 
                     reconnectLatch.countDown();
                 }
@@ -247,7 +277,7 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
         for (Ignite client : clients)
             srvSpi.failNode(client.cluster().localNode().id(), null);
 
-        waitReconnectEvent(disconnectLatch);
+        waitReconnectEvent(log, disconnectLatch);
 
         if (disconnectedC != null)
             disconnectedC.run();
@@ -257,13 +287,58 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
         for (Ignite client : clients)
             spi(client).writeLatch.countDown();
 
-        waitReconnectEvent(reconnectLatch);
+        waitReconnectEvent(log, reconnectLatch);
 
         for (Ignite client : clients)
             client.events().stopLocalListen(p);
     }
 
     /**
+     * @param log Logger.
+     * @param client Client node.
+     * @param srvs Server nodes to stop.
+     * @param srvStartC Closure starting server nodes.
+     * @throws Exception If failed.
+     * @return Restarted servers.
+     */
+    public static Collection<Ignite> reconnectServersRestart(final IgniteLogger log,
+        Ignite client,
+        Collection<Ignite> srvs,
+        Callable<Collection<Ignite>> srvStartC)
+        throws Exception {
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    log.info("Disconnected: " + evt);
+
+                    disconnectLatch.countDown();
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    log.info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        for (Ignite srv : srvs)
+            srv.close();
+
+        assertTrue(disconnectLatch.await(30_000, MILLISECONDS));
+
+        Collection<Ignite> startedSrvs = srvStartC.call();
+
+        assertTrue(reconnectLatch.await(10_000, MILLISECONDS));
+
+        return startedSrvs;
+    }
+
+    /**
      * @param e Client disconnected exception.
      * @return Reconnect future.
      */
@@ -303,7 +378,7 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
     /**
      *
      */
-    protected static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
+    public static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
         /** */
         volatile CountDownLatch writeLatch;
 
@@ -342,7 +417,7 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
         private IgniteLogger log;
 
         /** {@inheritDoc} */
-        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackClosure)
+        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC)
             throws IgniteSpiException {
             Class msgCls0 = msgCls;
 
@@ -356,7 +431,7 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
                 return;
             }
 
-            super.sendMessage(node, msg, ackClosure);
+            super.sendMessage(node, msg, ackC);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
index c46b5c8..13cac81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal;
 
+import java.util.Collection;
+import java.util.Collections;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.Ignite;
@@ -47,6 +49,61 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
     /**
      * @throws Exception If failed.
      */
+    public void testAtomicsReconnectClusterRestart() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final IgniteAtomicLong atomicLong = client.atomicLong("atomicLong", 1L, true);
+        final IgniteAtomicReference<Integer> atomicRef = client.atomicReference("atomicRef", 1, true);
+        final IgniteAtomicStamped<Integer, Integer> atomicStamped = client.atomicStamped("atomicStamped", 1, 1, true);
+        final IgniteCountDownLatch latch = client.countDownLatch("latch", 1, true, true);
+        final IgniteAtomicSequence seq = client.atomicSequence("seq", 1L, true);
+
+        Ignite srv = grid(0);
+
+        reconnectServersRestart(log, client, Collections.singleton(srv), new Callable<Collection<Ignite>>() {
+            @Override public Collection<Ignite> call() throws Exception {
+                return Collections.singleton((Ignite)startGrid(0));
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                atomicStamped.compareAndSet(1, 1, 2, 2);
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                atomicRef.compareAndSet(1, 2);
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                atomicLong.incrementAndGet();
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                seq.getAndAdd(1L);
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testAtomicSeqReconnect() throws Exception {
         Ignite client = grid(serverCount());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
index 14a770a..05da0b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
@@ -971,7 +971,8 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
                     info("Disconnected: " + evt);
 
                     disconnectLatch.countDown();
-                } else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
                     info("Reconnected: " + evt);
 
                     reconnectLatch.countDown();
@@ -1096,7 +1097,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
         for (int iter = 0; iter < 3; iter++) {
             log.info("Iteration: " + iter);
 
-            reconnectClientNodes(clients, grid(0), null);
+            reconnectClientNodes(log, clients, grid(0), null);
 
             for (Ignite client : clients) {
                 IgniteCache<Object, Object> cache = client.cache(null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5791837/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
index f6f038d..100e8de 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal;
 
+import java.util.Collection;
+import java.util.Collections;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.Ignite;
@@ -49,6 +51,55 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
     /**
      * @throws Exception If failed.
      */
+    public void testCollectionsReconnectClusterRestart() throws Exception {
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(TRANSACTIONAL);
+
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final IgniteQueue<Object> queue = client.queue("q", 0, colCfg);
+        final IgniteSet<Object> set = client.set("s", colCfg);
+
+        Ignite srv = grid(0);
+
+        reconnectServersRestart(log, client, Collections.singleton(srv), new Callable<Collection<Ignite>>() {
+            @Override public Collection<Ignite> call() throws Exception {
+                return Collections.singleton((Ignite)startGrid(0));
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                queue.add(1);
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                set.add(1);
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        try (IgniteQueue<Object> queue2 = client.queue("q", 0, colCfg)) {
+            queue2.add(1);
+        }
+
+        try (IgniteSet<Object> set2 = client.set("s", colCfg)) {
+            set2.add(1);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testQueueReconnect() throws Exception {
         CollectionConfiguration colCfg = new CollectionConfiguration();
 


[09/50] [abbrv] ignite git commit: IGNITE-1270 Implement basic support for OSGi.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index a7ae644..4fd7156 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -35,15 +35,87 @@
 
     <properties>
         <ignite.edition>fabric</ignite.edition>
-        <hadoop.version>2.4.1</hadoop.version>
-        <spark.version>1.5.2</spark.version>
-        <spring.version>4.1.0.RELEASE</spring.version>
+
+        <!-- Build parameters. -->
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <maven.build.timestamp.format>MMMM d yyyy</maven.build.timestamp.format>
         <doxygen.exec>doxygen</doxygen.exec>
         <git.exec>git</git.exec>
-        <jetty.version>9.2.11.v20150529</jetty.version>
+        <maven.bundle.plugin.version>2.5.4</maven.bundle.plugin.version>
         <javadoc.opts>-XDenableSunApiLintControl</javadoc.opts>
+
+        <!-- Dependency versions. -->
+        <activemq.version>5.12.0</activemq.version>
+        <aopalliance.bundle.version>1.0_6</aopalliance.bundle.version>
+        <asm.version>4.2</asm.version>
+        <aspectj.bundle.version>1.7.2_1</aspectj.bundle.version>
+        <aspectj.version>1.7.2</aspectj.version>
+        <aws.sdk.bundle.version>1.10.12_1</aws.sdk.bundle.version>
+        <aws.sdk.version>1.10.29</aws.sdk.version>
+        <camel.version>2.16.0</camel.version>
+        <commons.beanutils.bundle.version>1.8.3_1</commons.beanutils.bundle.version>
+        <commons.beanutils.version>1.8.3</commons.beanutils.version>
+        <commons.codec.version>1.6</commons.codec.version>
+        <commons.collections.version>3.2.1</commons.collections.version>
+        <commons.lang.version>2.6</commons.lang.version>
+        <cron4j.version>2.2.5</cron4j.version>
+        <curator.version>2.9.1</curator.version>
+        <ezmorph.bundle.version>1.0.6_1</ezmorph.bundle.version>
+        <ezmorph.version>1.0.6</ezmorph.version>
+        <flume.ng.version>1.6.0</flume.ng.version>
+        <guava.retrying.version>2.0.0</guava.retrying.version>
+        <guava.version>18.0</guava.version>
+        <guava14.version>14.0.1</guava14.version>
+        <guava16.version>16.0.1</guava16.version>
+        <h2.version>1.3.175</h2.version>
+        <hadoop.version>2.4.1</hadoop.version>
+        <httpclient.version>4.5.1</httpclient.version>
+        <httpcore.version>4.4.3</httpcore.version>
+        <jackson.version>1.9.13</jackson.version>
+        <javax.cache.bundle.version>1.0.0_1</javax.cache.bundle.version>
+        <javax.cache.version>1.0.0</javax.cache.version>
+        <jetty.version>9.2.11.v20150529</jetty.version>
+        <jms.spec.version>1.1.1</jms.spec.version>
+        <jsch.bundle.version>0.1.53_1</jsch.bundle.version>
+        <jsch.version>0.1.53</jsch.version>
+        <jsonlib.bundle.version>2.4_1</jsonlib.bundle.version>
+        <jsonlib.version>2.4</jsonlib.version>
+        <jtidy.version>r938</jtidy.version>
+        <kafka.bundle.version>0.8.2.1_1</kafka.bundle.version>
+        <kafka.clients.bundle.version>0.8.2.0_1</kafka.clients.bundle.version>
+        <kafka.clients.version>0.8.2.0</kafka.clients.version>
+        <kafka.version>0.8.2.1</kafka.version>
+        <kafka.version>0.8.2.1</kafka.version>
+        <karaf.version>4.0.2</karaf.version>
+        <lucene.bundle.version>3.5.0_1</lucene.bundle.version>
+        <lucene.version>3.5.0</lucene.version>
+        <oro.bundle.version>2.0.8_6</oro.bundle.version>
+        <osgi.core.version>5.0.0</osgi.core.version>
+        <osgi.enterprise.version>5.0.0</osgi.enterprise.version>
+        <paho.version>1.0.2</paho.version>
+        <scala210.jline.version>2.10.4</scala210.jline.version>
+        <scala210.library.version>2.10.4</scala210.library.version>
+        <scala211.library.version>2.11.7</scala211.library.version>
+        <slf4j.version>1.7.7</slf4j.version>
+        <slf4j16.version>1.6.4</slf4j16.version>
+        <spark.version>1.5.2</spark.version>
+        <spring.version>4.1.0.RELEASE</spring.version>
+        <spring41.osgi.feature.version>4.1.7.RELEASE_1</spring41.osgi.feature.version>
+        <tomcat.version>8.0.23</tomcat.version>
+        <twitter.hbc.version>2.2.0</twitter.hbc.version>
+        <yammer.metrics.annotation.version>2.2.0</yammer.metrics.annotation.version>
+        <yammer.metrics.core.version>2.2.0</yammer.metrics.core.version>
+        <zkclient.version>0.5</zkclient.version>
+        <zookeeper.version>3.4.6</zookeeper.version>
+
+        <!-- OSGI Manifest generation default property values -->
+        <osgi.import.package>*</osgi.import.package>
+        <osgi.export.package>{local-packages}</osgi.export.package>
+        <osgi.private.package></osgi.private.package>
+        <osgi.embed.dependency></osgi.embed.dependency>
+        <osgi.embed.transitive>false</osgi.embed.transitive>
+        <osgi.fail.ok>false</osgi.fail.ok>
+
     </properties>
 
     <groupId>org.apache.ignite</groupId>
@@ -182,6 +254,15 @@
                         </execution>
                     </executions>
                 </plugin>
+
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-jar-plugin</artifactId>
+                    <configuration>
+                        <useDefaultManifestFile>true</useDefaultManifestFile>
+                    </configuration>
+                </plugin>
+
                 <plugin>
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-javadoc-plugin</artifactId>
@@ -402,6 +483,52 @@
                         </bottom>
                     </configuration>
                 </plugin>
+
+                <plugin>
+                    <groupId>org.apache.felix</groupId>
+                    <artifactId>maven-bundle-plugin</artifactId>
+                    <version>${maven.bundle.plugin.version}</version>
+                    <extensions>true</extensions>
+                    <configuration>
+                        <archive>
+                            <addMavenDescriptor>true</addMavenDescriptor>
+                        </archive>
+                        <supportedProjectTypes>
+                            <supportedProjectType>jar</supportedProjectType>
+                            <supportedProjectType>war</supportedProjectType>
+                        </supportedProjectTypes>
+                        <instructions>
+                            <Bundle-SymbolicName>${project.groupId}.${project.artifactId}</Bundle-SymbolicName>
+                            <Bundle-Version>${project.version}</Bundle-Version>
+                            <Bundle-Vendor>${project.organization.name}</Bundle-Vendor>
+                            <Bundle-Description>${project.description}</Bundle-Description>
+                            <Bundle-DocURL>${project.url}</Bundle-DocURL>
+                            <Import-Package>
+                                ${osgi.import.package}
+                            </Import-Package>
+                            <Export-Package>
+                                ${osgi.export.package}
+                            </Export-Package>
+                            <Private-Package>
+                                ${osgi.private.package}
+                            </Private-Package>
+                            <Embed-Dependency>${osgi.embed.dependency}</Embed-Dependency>
+                            <Embed-Directory>lib</Embed-Directory>
+                            <Embed-Transitive>${osgi.embed.transitive}</Embed-Transitive>
+                            <_failok>${osgi.fail.ok}</_failok>
+                            <_invalidfilenames />
+                        </instructions>
+                    </configuration>
+                    <executions>
+                        <execution>
+                            <id>bundle-manifest</id>
+                            <phase>process-classes</phase>
+                            <goals>
+                                <goal>manifest</goal>
+                            </goals>
+                        </execution>
+                    </executions>
+                </plugin>
             </plugins>
         </pluginManagement>
 
@@ -625,6 +752,7 @@
                     </execution>
                 </executions>
             </plugin>
+
         </plugins>
     </build>
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/96e08027/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5f86648..993eab2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -80,6 +80,9 @@
         <module>modules/mqtt</module>
         <module>modules/zookeeper</module>
         <module>modules/camel</module>
+        <module>modules/osgi-paxlogging</module>
+        <module>modules/osgi-karaf</module>
+        <module>modules/osgi</module>
     </modules>
 
     <profiles>


[06/50] [abbrv] ignite git commit: ignite-2042 Added special queue/set key classes to make collocation work with BinaryMarshaller. Also fixed issue with 'invoke' result with binary marshaller.

Posted by sb...@apache.org.
ignite-2042 Added special queue/set key classes to make collocation work with BinaryMarshaller. Also fixed issue with 'invoke' result with binary marshaller.


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

Branch: refs/heads/ignite-1.5.1
Commit: 50f6c0131fd761f6231e7c2632a010c093000e70
Parents: 86ec37e
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 3 16:50:00 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 3 16:50:00 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryReaderExImpl.java   |   2 +-
 .../internal/portable/PortableContext.java      |  34 +-
 .../processors/cache/GridCacheContext.java      |  30 ++
 .../CacheDataStructuresManager.java             |  31 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  20 +-
 .../CacheObjectBinaryProcessorImpl.java         |   8 +
 .../cache/query/GridCacheQueryManager.java      |  12 +-
 .../transactions/IgniteTxLocalAdapter.java      |  11 +-
 .../datastructures/CollocatedQueueItemKey.java  |  75 ++++
 .../datastructures/CollocatedSetItemKey.java    |  87 +++++
 .../datastructures/DataStructuresProcessor.java |   7 +-
 .../GridAtomicCacheQueueImpl.java               |   8 +-
 .../datastructures/GridCacheQueueAdapter.java   |  30 +-
 .../datastructures/GridCacheQueueItemKey.java   |   9 +-
 .../datastructures/GridCacheSetImpl.java        |  37 +-
 .../datastructures/GridCacheSetItemKey.java     |  21 +-
 .../GridTransactionalCacheQueueImpl.java        |   2 +-
 .../processors/datastructures/QueueItemKey.java |  27 ++
 .../processors/datastructures/SetItemKey.java   |  36 ++
 .../cache/IgniteCacheInvokeAbstractTest.java    | 369 ++++++++++++++-----
 ...eAbstractDataStructuresFailoverSelfTest.java |   7 +-
 .../GridCacheQueueApiSelfAbstractTest.java      |  18 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |   6 +-
 .../GridCachePartitionedQueueApiSelfTest.java   |   5 +
 ...dCachePartitionedQueueEntryMoveSelfTest.java |   2 +-
 .../IgnitePartitionedQueueNoBackupsTest.java    |  92 +++++
 .../GridCacheReplicatedQueueApiSelfTest.java    |   5 +
 .../GridCacheWriteBehindStoreAbstractTest.java  |   2 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |   3 +
 29 files changed, 778 insertions(+), 218 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
index ddbf6ba..91b67f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
@@ -246,7 +246,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
                 dataStart = start + DFLT_HDR_LEN;
             }
 
-            idMapper = userType ? ctx.userTypeIdMapper(typeId) : null;
+            idMapper = userType ? ctx.userTypeIdMapper(typeId) : BinaryInternalIdMapper.defaultInstance();
             schema = PortableUtils.hasSchema(flags) ? getOrCreateSchema() : null;
         }
         else {

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 1482df9..fd6c41d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -66,6 +66,8 @@ import org.apache.ignite.binary.BinarySerializer;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.datastructures.CollocatedQueueItemKey;
+import org.apache.ignite.internal.processors.datastructures.CollocatedSetItemKey;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
 import org.apache.ignite.internal.util.typedef.F;
@@ -233,7 +235,8 @@ public class PortableContext implements Externalizable {
 
     /**
      * @param marsh Portable marshaller.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @param cfg Configuration.
+     * @throws BinaryObjectException In case of error.
      */
     public void configure(BinaryMarshaller marsh, IgniteConfiguration cfg) throws BinaryObjectException {
         if (marsh == null)
@@ -265,7 +268,7 @@ public class PortableContext implements Externalizable {
      * @param globalIdMapper ID mapper.
      * @param globalSerializer Serializer.
      * @param typeCfgs Type configurations.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @throws BinaryObjectException In case of error.
      */
     private void configure(
         BinaryIdMapper globalIdMapper,
@@ -313,9 +316,8 @@ public class PortableContext implements Externalizable {
             }
         }
 
-        for (TypeDescriptor desc : descs.descriptors()) {
+        for (TypeDescriptor desc : descs.descriptors())
             registerUserType(desc.clsName, desc.idMapper, desc.serializer, desc.affKeyFieldName, desc.isEnum);
-        }
 
         BinaryInternalIdMapper dfltMapper = BinaryInternalIdMapper.create(globalIdMapper);
 
@@ -327,6 +329,20 @@ public class PortableContext implements Externalizable {
 
             affKeyFieldNames.putIfAbsent(typeId, entry.getValue());
         }
+
+        addSystemClassAffinityKey(CollocatedSetItemKey.class);
+        addSystemClassAffinityKey(CollocatedQueueItemKey.class);
+    }
+
+    /**
+     * @param cls Class.
+     */
+    private void addSystemClassAffinityKey(Class<?> cls) {
+        String fieldName = affinityFieldName(cls);
+
+        assert fieldName != null : cls;
+
+        affKeyFieldNames.putIfAbsent(cls.getName().hashCode(), affinityFieldName(cls));
     }
 
     /**
@@ -400,7 +416,7 @@ public class PortableContext implements Externalizable {
     /**
      * @param cls Class.
      * @return Class descriptor.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @throws BinaryObjectException In case of error.
      */
     public PortableClassDescriptor descriptorForClass(Class<?> cls, boolean deserialize)
         throws BinaryObjectException {
@@ -722,7 +738,7 @@ public class PortableContext implements Externalizable {
      * @param serializer Serializer.
      * @param affKeyFieldName Affinity key field name.
      * @param isEnum If enum.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @throws BinaryObjectException In case of error.
      */
     @SuppressWarnings("ErrorNotRethrown")
     public void registerUserType(String clsName,
@@ -808,7 +824,7 @@ public class PortableContext implements Externalizable {
     /**
      * @param typeId Type ID.
      * @return Meta data.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @throws BinaryObjectException In case of error.
      */
     @Nullable public BinaryType metadata(int typeId) throws BinaryObjectException {
         return metaHnd != null ? metaHnd.metadata(typeId) : null;
@@ -964,7 +980,7 @@ public class PortableContext implements Externalizable {
          * @param affKeyFieldName Affinity key field name.
          * @param isEnum Enum flag.
          * @param canOverride Whether this descriptor can be override.
-         * @throws org.apache.ignite.binary.BinaryObjectException If failed.
+         * @throws BinaryObjectException If failed.
          */
         private void add(String clsName,
             BinaryIdMapper idMapper,
@@ -1044,7 +1060,7 @@ public class PortableContext implements Externalizable {
          * Override portable class descriptor.
          *
          * @param other Other descriptor.
-         * @throws org.apache.ignite.binary.BinaryObjectException If failed.
+         * @throws BinaryObjectException If failed.
          */
         private void override(TypeDescriptor other) throws BinaryObjectException {
             assert clsName.equals(other.clsName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 5b4f22c..d689ba6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -36,6 +36,7 @@ import javax.cache.Cache;
 import javax.cache.configuration.Factory;
 import javax.cache.expiry.EternalExpiryPolicy;
 import javax.cache.expiry.ExpiryPolicy;
+import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheInterceptor;
@@ -53,6 +54,7 @@ import org.apache.ignite.internal.managers.deployment.GridDeploymentManager;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.managers.swapspace.GridSwapSpaceManager;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
@@ -1682,6 +1684,13 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return {@code True} if {@link BinaryMarshaller is configured}.
+     */
+    public boolean binaryMarshaller() {
+        return marshaller() instanceof BinaryMarshaller;
+    }
+
+    /**
      * @return Keep portable flag.
      */
     public boolean keepPortable() {
@@ -1752,6 +1761,27 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @param resMap Invoke results map.
+     * @param keepBinary Keep binary flag.
+     * @return Unwrapped results.
+     */
+    public Map unwrapInvokeResult(@Nullable Map<Object, EntryProcessorResult> resMap, final boolean keepBinary) {
+        return F.viewReadOnly(resMap, new C1<EntryProcessorResult, EntryProcessorResult>() {
+            @Override public EntryProcessorResult apply(EntryProcessorResult res) {
+                if (res instanceof CacheInvokeResult) {
+                    CacheInvokeResult invokeRes = (CacheInvokeResult)res;
+
+                    if (invokeRes.result() != null)
+                        res = CacheInvokeResult.fromResult(unwrapPortableIfNeeded(invokeRes.result(),
+                            keepBinary, false));
+                }
+
+                return res;
+            }
+        });
+    }
+
+    /**
      * @return Cache object context.
      */
     public CacheObjectContext cacheObjectContext() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
index ec787f8..6ec29b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
@@ -54,9 +54,9 @@ import org.apache.ignite.internal.processors.datastructures.GridCacheQueueProxy;
 import org.apache.ignite.internal.processors.datastructures.GridCacheSetHeader;
 import org.apache.ignite.internal.processors.datastructures.GridCacheSetHeaderKey;
 import org.apache.ignite.internal.processors.datastructures.GridCacheSetImpl;
-import org.apache.ignite.internal.processors.datastructures.GridCacheSetItemKey;
 import org.apache.ignite.internal.processors.datastructures.GridCacheSetProxy;
 import org.apache.ignite.internal.processors.datastructures.GridTransactionalCacheQueueImpl;
+import org.apache.ignite.internal.processors.datastructures.SetItemKey;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
@@ -79,7 +79,7 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
     private final ConcurrentMap<IgniteUuid, GridCacheSetProxy> setsMap;
 
     /** Set keys used for set iteration. */
-    private ConcurrentMap<IgniteUuid, GridConcurrentHashSet<GridCacheSetItemKey>> setDataMap =
+    private ConcurrentMap<IgniteUuid, GridConcurrentHashSet<SetItemKey>> setDataMap =
         new ConcurrentHashMap8<>();
 
     /** Queues map. */
@@ -311,12 +311,13 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
      *
      * @param key Key.
      * @param rmv {@code True} if entry was removed.
+     * @param keepPortable Keep portable flag.
      */
     public void onEntryUpdated(KeyCacheObject key, boolean rmv, boolean keepPortable) {
         Object key0 = cctx.cacheObjectContext().unwrapPortableIfNeeded(key, keepPortable, false);
 
-        if (key0 instanceof GridCacheSetItemKey)
-            onSetItemUpdated((GridCacheSetItemKey)key0, rmv);
+        if (key0 instanceof SetItemKey)
+            onSetItemUpdated((SetItemKey)key0, rmv);
     }
 
     /**
@@ -327,11 +328,11 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
     public void onPartitionEvicted(int part) {
         GridCacheAffinityManager aff = cctx.affinity();
 
-        for (GridConcurrentHashSet<GridCacheSetItemKey> set : setDataMap.values()) {
-            Iterator<GridCacheSetItemKey> iter = set.iterator();
+        for (GridConcurrentHashSet<SetItemKey> set : setDataMap.values()) {
+            Iterator<SetItemKey> iter = set.iterator();
 
             while (iter.hasNext()) {
-                GridCacheSetItemKey key = iter.next();
+                SetItemKey key = iter.next();
 
                 if (aff.partition(key) == part)
                     iter.remove();
@@ -415,7 +416,7 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
      * @param id Set ID.
      * @return Data for given set.
      */
-    @Nullable public GridConcurrentHashSet<GridCacheSetItemKey> setData(IgniteUuid id) {
+    @Nullable public GridConcurrentHashSet<SetItemKey> setData(IgniteUuid id) {
         return setDataMap.get(id);
     }
 
@@ -436,7 +437,7 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
             cctx.preloader().syncFuture().get();
         }
 
-        GridConcurrentHashSet<GridCacheSetItemKey> set = setDataMap.get(setId);
+        GridConcurrentHashSet<SetItemKey> set = setDataMap.get(setId);
 
         if (set == null)
             return;
@@ -445,9 +446,9 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
 
         final int BATCH_SIZE = 100;
 
-        Collection<GridCacheSetItemKey> keys = new ArrayList<>(BATCH_SIZE);
+        Collection<SetItemKey> keys = new ArrayList<>(BATCH_SIZE);
 
-        for (GridCacheSetItemKey key : set) {
+        for (SetItemKey key : set) {
             if (!loc && !aff.primary(cctx.localNode(), key, topVer))
                 continue;
 
@@ -555,14 +556,14 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
      * @param key Set item key.
      * @param rmv {@code True} if item was removed.
      */
-    private void onSetItemUpdated(GridCacheSetItemKey key, boolean rmv) {
-        GridConcurrentHashSet<GridCacheSetItemKey> set = setDataMap.get(key.setId());
+    private void onSetItemUpdated(SetItemKey key, boolean rmv) {
+        GridConcurrentHashSet<SetItemKey> set = setDataMap.get(key.setId());
 
         if (set == null) {
             if (rmv)
                 return;
 
-            GridConcurrentHashSet<GridCacheSetItemKey> old = setDataMap.putIfAbsent(key.setId(),
+            GridConcurrentHashSet<SetItemKey> old = setDataMap.putIfAbsent(key.setId(),
                 set = new GridConcurrentHashSet<>());
 
             if (old != null)
@@ -592,7 +593,7 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    private void retryRemoveAll(final IgniteInternalCache cache, final Collection<GridCacheSetItemKey> keys)
+    private void retryRemoveAll(final IgniteInternalCache cache, final Collection<SetItemKey> keys)
         throws IgniteCheckedException {
         DataStructuresProcessor.retry(log, new Callable<Void>() {
             @Override public Void call() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/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 d8ab62a..c5ec258 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
@@ -824,25 +824,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         return resFut.chain(new CX1<IgniteInternalFuture<Map<K, EntryProcessorResult<T>>>, Map<K, EntryProcessorResult<T>>>() {
             @Override public Map<K, EntryProcessorResult<T>> applyx(IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut) throws IgniteCheckedException {
-                Map<K, EntryProcessorResult<T>> resMap = fut.get();
+                Map<Object, EntryProcessorResult> resMap = (Map)fut.get();
 
-                if (resMap != null) {
-                    return F.viewReadOnly(resMap, new C1<EntryProcessorResult<T>, EntryProcessorResult<T>>() {
-                        @Override public EntryProcessorResult<T> apply(EntryProcessorResult<T> res) {
-                            if (res instanceof CacheInvokeResult) {
-                                CacheInvokeResult invokeRes = (CacheInvokeResult)res;
-
-                                if (invokeRes.result() != null)
-                                    res = CacheInvokeResult.fromResult((T)ctx.unwrapPortableIfNeeded(invokeRes.result(),
-                                        keepBinary, false));
-                            }
-
-                            return res;
-                        }
-                    });
-                }
-
-                return null;
+                return ctx.unwrapInvokeResult(resMap, keepBinary);
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index 220a45a..d172bca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -602,6 +602,14 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                 if (affKeyFieldName != null)
                     return po.field(affKeyFieldName);
             }
+            else if (po instanceof BinaryObjectEx) {
+                int id = ((BinaryObjectEx)po).typeId();
+
+                String affKeyFieldName = portableCtx.affinityKeyFieldName(id);
+
+                if (affKeyFieldName != null)
+                    return po.field(affKeyFieldName);
+            }
         }
         catch (BinaryObjectException e) {
             U.error(log, "Failed to get affinity field from portable object: " + po, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index bef587a..bb5d230 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -47,8 +47,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheA
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnreservedPartitionException;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.datastructures.GridCacheSetItemKey;
 import org.apache.ignite.internal.processors.datastructures.GridSetQueryPredicate;
+import org.apache.ignite.internal.processors.datastructures.SetItemKey;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
@@ -761,21 +761,21 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         IgniteUuid id = filter.setId();
 
-        Collection<GridCacheSetItemKey> data = cctx.dataStructures().setData(id);
+        Collection<SetItemKey> data = cctx.dataStructures().setData(id);
 
         if (data == null)
             data = Collections.emptyList();
 
         final GridIterator<IgniteBiTuple<K, V>> it = F.iterator(
             data,
-            new C1<GridCacheSetItemKey, IgniteBiTuple<K, V>>() {
-                @Override public IgniteBiTuple<K, V> apply(GridCacheSetItemKey e) {
+            new C1<SetItemKey, IgniteBiTuple<K, V>>() {
+                @Override public IgniteBiTuple<K, V> apply(SetItemKey e) {
                     return new IgniteBiTuple<>((K)e.item(), (V)Boolean.TRUE);
                 }
             },
             true,
-            new P1<GridCacheSetItemKey>() {
-                @Override public boolean apply(GridCacheSetItemKey e) {
+            new P1<SetItemKey>() {
+                @Override public boolean apply(SetItemKey e) {
                     return filter.apply(e, null);
                 }
             });

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index f13cff4..33c0fa9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -3220,8 +3220,15 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                         try {
                             txFut.get();
 
-                            return new GridCacheReturn(cacheCtx, true, keepBinary,
-                                implicitRes.value(), implicitRes.success());
+                            Object res = implicitRes.value();
+
+                            if (implicitRes.invokeResult()) {
+                                assert res == null || res instanceof Map : implicitRes;
+
+                                res = cacheCtx.unwrapInvokeResult((Map)res, keepBinary);
+                            }
+
+                            return new GridCacheReturn(cacheCtx, true, keepBinary, res, implicitRes.success());
                         }
                         catch (IgniteCheckedException | RuntimeException e) {
                             rollbackAsync();

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedQueueItemKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedQueueItemKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedQueueItemKey.java
new file mode 100644
index 0000000..8eb9fa0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedQueueItemKey.java
@@ -0,0 +1,75 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ *
+ */
+public class CollocatedQueueItemKey implements QueueItemKey {
+    /** */
+    private IgniteUuid queueId;
+
+    /** */
+    @AffinityKeyMapped
+    private int queueNameHash;
+
+    /** */
+    private long idx;
+
+    /**
+     * @param queueId Queue unique ID.
+     * @param queueName Queue name.
+     * @param idx Item index.
+     */
+    public CollocatedQueueItemKey(IgniteUuid queueId, String queueName, long idx) {
+        this.queueId = queueId;
+        this.queueNameHash = queueName.hashCode();
+        this.idx = idx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        CollocatedQueueItemKey itemKey = (CollocatedQueueItemKey)o;
+
+        return idx == itemKey.idx && queueId.equals(itemKey.queueId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = queueId.hashCode();
+
+        res = 31 * res + (int)(idx ^ (idx >>> 32));
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CollocatedQueueItemKey.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedSetItemKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedSetItemKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedSetItemKey.java
new file mode 100644
index 0000000..94cffd4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedSetItemKey.java
@@ -0,0 +1,87 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ *
+ */
+public class CollocatedSetItemKey implements SetItemKey {
+    /** */
+    private IgniteUuid setId;
+
+    /** */
+    @GridToStringInclude
+    private Object item;
+
+    /** */
+    @AffinityKeyMapped
+    private int setNameHash;
+
+    /**
+     * @param setName Set name.
+     * @param setId Set unique ID.
+     * @param item Set item.
+     */
+    public CollocatedSetItemKey(String setName, IgniteUuid setId, Object item) {
+        this.setNameHash = setName.hashCode();
+        this.setId = setId;
+        this.item = item;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid setId() {
+        return setId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object item() {
+        return item;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = setId.hashCode();
+
+        res = 31 * res + item.hashCode();
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        CollocatedSetItemKey that = (CollocatedSetItemKey)o;
+
+        return setId.equals(that.setId) && item.equals(that.item);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CollocatedSetItemKey.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 998bd92..9ed9350 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -113,12 +113,6 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
     /** Initial capacity. */
     private static final int INITIAL_CAPACITY = 10;
 
-    /** */
-    private static final int MAX_UPDATE_RETRIES = 100;
-
-    /** */
-    private static final long RETRY_DELAY = 1;
-
     /** Initialization latch. */
     private final CountDownLatch initLatch = new CountDownLatch(1);
 
@@ -986,6 +980,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                     hdr.id(),
                     name,
                     hdr.collocated(),
+                    cctx.binaryMarshaller(),
                     hdr.head(),
                     hdr.tail(),
                     0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridAtomicCacheQueueImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridAtomicCacheQueueImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridAtomicCacheQueueImpl.java
index b433887..58d3efe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridAtomicCacheQueueImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridAtomicCacheQueueImpl.java
@@ -55,7 +55,7 @@ public class GridAtomicCacheQueueImpl<T> extends GridCacheQueueAdapter<T> {
 
             checkRemoved(idx);
 
-            GridCacheQueueItemKey key = itemKey(idx);
+            QueueItemKey key = itemKey(idx);
 
             cache.getAndPut(key, item);
 
@@ -78,7 +78,7 @@ public class GridAtomicCacheQueueImpl<T> extends GridCacheQueueAdapter<T> {
 
                 checkRemoved(idx);
 
-                GridCacheQueueItemKey key = itemKey(idx);
+                QueueItemKey key = itemKey(idx);
 
                 T data = (T)cache.getAndRemove(key);
 
@@ -115,7 +115,7 @@ public class GridAtomicCacheQueueImpl<T> extends GridCacheQueueAdapter<T> {
 
             checkRemoved(idx);
 
-            Map<GridCacheQueueItemKey, T> putMap = new HashMap<>();
+            Map<QueueItemKey, T> putMap = new HashMap<>();
 
             for (T item : items) {
                 putMap.put(itemKey(idx), item);
@@ -140,7 +140,7 @@ public class GridAtomicCacheQueueImpl<T> extends GridCacheQueueAdapter<T> {
         if (idx != null) {
             checkRemoved(idx);
 
-            GridCacheQueueItemKey key = itemKey(idx);
+            QueueItemKey key = itemKey(idx);
 
             if (cache.remove(key))
                 return;

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
index df1bd88..ca0250d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
@@ -58,9 +58,6 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
     protected static final long QUEUE_REMOVED_IDX = Long.MIN_VALUE;
 
     /** */
-    protected static final long RETRY_DELAY = 1;
-
-    /** */
     private static final int DFLT_CLEAR_BATCH_SIZE = 100;
 
     /** Logger. */
@@ -98,6 +95,9 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
     @GridToStringExclude
     private final Semaphore writeSem;
 
+    /** */
+    private final boolean binaryMarsh;
+
     /**
      * @param queueName Queue name.
      * @param hdr Queue hdr.
@@ -112,6 +112,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
         collocated = hdr.collocated();
         queueKey = new GridCacheQueueHeaderKey(queueName);
         cache = cctx.kernalContext().cache().internalCache(cctx.name());
+        binaryMarsh = cctx.binaryMarshaller();
 
         log = cctx.logger(getClass());
 
@@ -369,7 +370,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
 
             checkRemoved(t.get1());
 
-            removeKeys(cache, id, queueName, collocated, t.get1(), t.get2(), batchSize);
+            removeKeys(cache, id, queueName, collocated, binaryMarsh, t.get1(), t.get2(), batchSize);
         }
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
@@ -407,6 +408,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
      * @param id Queue unique ID.
      * @param name Queue name.
      * @param collocated Collocation flag.
+     * @param binaryMarsh {@code True} if binary marshaller is configured.
      * @param startIdx Start item index.
      * @param endIdx End item index.
      * @param batchSize Batch size.
@@ -418,14 +420,15 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
         IgniteUuid id,
         String name,
         boolean collocated,
+        boolean binaryMarsh,
         long startIdx,
         long endIdx,
         int batchSize)
         throws IgniteCheckedException {
-        Set<GridCacheQueueItemKey> keys = new HashSet<>(batchSize > 0 ? batchSize : 10);
+        Set<QueueItemKey> keys = new HashSet<>(batchSize > 0 ? batchSize : 10);
 
         for (long idx = startIdx; idx < endIdx; idx++) {
-            keys.add(itemKey(id, name, collocated, idx));
+            keys.add(itemKey(id, name, collocated, binaryMarsh, idx));
 
             if (batchSize > 0 && keys.size() == batchSize) {
                 cache.removeAll(keys);
@@ -536,8 +539,8 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
      * @param idx Item index.
      * @return Item key.
      */
-    protected GridCacheQueueItemKey itemKey(Long idx) {
-        return itemKey(id, queueName, collocated(), idx);
+    protected QueueItemKey itemKey(Long idx) {
+        return itemKey(id, queueName, collocated(), binaryMarsh, idx);
     }
 
     /** {@inheritDoc} */
@@ -558,11 +561,18 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
      * @param id Queue unique ID.
      * @param queueName Queue name.
      * @param collocated Collocation flag.
+     * @param binaryMarsh {@code True} if binary marshaller is configured.
      * @param idx Item index.
      * @return Item key.
      */
-    private static GridCacheQueueItemKey itemKey(IgniteUuid id, String queueName, boolean collocated, long idx) {
-        return collocated ? new CollocatedItemKey(id, queueName, idx) : new GridCacheQueueItemKey(id, queueName, idx);
+    private static QueueItemKey itemKey(IgniteUuid id,
+        String queueName,
+        boolean collocated,
+        boolean binaryMarsh,
+        long idx) {
+        return collocated ?
+            (binaryMarsh ? new CollocatedQueueItemKey(id, queueName, idx) : new CollocatedItemKey(id, queueName, idx)) :
+            new GridCacheQueueItemKey(id, queueName, idx);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueItemKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueItemKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueItemKey.java
index c4cb7b1..df47e73 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueItemKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueItemKey.java
@@ -21,7 +21,6 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-import org.apache.ignite.internal.processors.cache.GridCacheInternal;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -29,7 +28,7 @@ import org.apache.ignite.lang.IgniteUuid;
 /**
  * Queue item key.
  */
-class GridCacheQueueItemKey implements Externalizable, GridCacheInternal {
+class GridCacheQueueItemKey implements Externalizable, QueueItemKey {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -110,11 +109,11 @@ class GridCacheQueueItemKey implements Externalizable, GridCacheInternal {
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        int result = queueId.hashCode();
+        int res = queueId.hashCode();
 
-        result = 31 * result + (int)(idx ^ (idx >>> 32));
+        res = 31 * res + (int)(idx ^ (idx >>> 32));
 
-        return result;
+        return res;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
index 62eab61..f25e361 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
@@ -66,7 +66,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
     private final GridCacheContext ctx;
 
     /** Cache. */
-    private final IgniteInternalCache<GridCacheSetItemKey, Boolean> cache;
+    private final IgniteInternalCache<SetItemKey, Boolean> cache;
 
     /** Logger. */
     private final IgniteLogger log;
@@ -86,6 +86,9 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
     /** Removed flag. */
     private volatile boolean rmvd;
 
+    /** */
+    private final boolean binaryMarsh;
+
     /**
      * @param ctx Cache context.
      * @param name Set name.
@@ -97,6 +100,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
         this.name = name;
         id = hdr.id();
         collocated = hdr.collocated();
+        binaryMarsh = ctx.binaryMarshaller();
 
         cache = ctx.cache();
 
@@ -140,7 +144,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
             onAccess();
 
             if (ctx.isLocal() || ctx.isReplicated()) {
-                GridConcurrentHashSet<GridCacheSetItemKey> set = ctx.dataStructures().setData(id);
+                GridConcurrentHashSet<SetItemKey> set = ctx.dataStructures().setData(id);
 
                 return set != null ? set.size() : 0;
             }
@@ -171,7 +175,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
     @Override public boolean isEmpty() {
         onAccess();
 
-        GridConcurrentHashSet<GridCacheSetItemKey> set = ctx.dataStructures().setData(id);
+        GridConcurrentHashSet<SetItemKey> set = ctx.dataStructures().setData(id);
 
         return (set == null || set.isEmpty()) && size() == 0;
     }
@@ -180,7 +184,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
     @Override public boolean contains(Object o) {
         onAccess();
 
-        final GridCacheSetItemKey key = itemKey(o);
+        final SetItemKey key = itemKey(o);
 
         return retry(new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
@@ -193,7 +197,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
     @Override public boolean add(T o) {
         onAccess();
 
-        final GridCacheSetItemKey key = itemKey(o);
+        final SetItemKey key = itemKey(o);
 
         return retry(new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
@@ -206,7 +210,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
     @Override public boolean remove(Object o) {
         onAccess();
 
-        final GridCacheSetItemKey key = itemKey(o);
+        final SetItemKey key = itemKey(o);
 
         return retry(new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
@@ -231,7 +235,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
 
         boolean add = false;
 
-        Map<GridCacheSetItemKey, Boolean> addKeys = null;
+        Map<SetItemKey, Boolean> addKeys = null;
 
         for (T obj : c) {
             if (add) {
@@ -247,7 +251,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
                 }
             }
             else
-                add |= add(obj);
+                add = add(obj);
         }
 
         if (!F.isEmpty(addKeys))
@@ -262,7 +266,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
 
         boolean rmv = false;
 
-        Set<GridCacheSetItemKey> rmvKeys = null;
+        Set<SetItemKey> rmvKeys = null;
 
         for (Object obj : c) {
             if (rmv) {
@@ -278,7 +282,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
                 }
             }
             else
-                rmv |= remove(obj);
+                rmv = remove(obj);
         }
 
         if (!F.isEmpty(rmvKeys))
@@ -295,7 +299,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
             try (GridCloseableIterator<T> iter = iterator0()) {
                 boolean rmv = false;
 
-                Set<GridCacheSetItemKey> rmvKeys = null;
+                Set<SetItemKey> rmvKeys = null;
 
                 for (T val : iter) {
                     if (!c.contains(val)) {
@@ -331,7 +335,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
             onAccess();
 
             try (GridCloseableIterator<T> iter = iterator0()) {
-                Collection<GridCacheSetItemKey> rmvKeys = new ArrayList<>(BATCH_SIZE);
+                Collection<SetItemKey> rmvKeys = new ArrayList<>(BATCH_SIZE);
 
                 for (T val : iter) {
                     rmvKeys.add(itemKey(val));
@@ -425,7 +429,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
     /**
      * @param keys Keys to remove.
      */
-    private void retryRemoveAll(final Collection<GridCacheSetItemKey> keys) {
+    private void retryRemoveAll(final Collection<SetItemKey> keys) {
         retry(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 cache.removeAll(keys);
@@ -438,7 +442,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
     /**
      * @param keys Keys to remove.
      */
-    private void retryPutAll(final Map<GridCacheSetItemKey, Boolean> keys) {
+    private void retryPutAll(final Map<SetItemKey, Boolean> keys) {
         retry(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 cache.putAll(keys);
@@ -523,8 +527,9 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
      * @param item Set item.
      * @return Item key.
      */
-    private GridCacheSetItemKey itemKey(Object item) {
-        return collocated ? new CollocatedItemKey(name, id, item) : new GridCacheSetItemKey(id, item);
+    private SetItemKey itemKey(Object item) {
+        return collocated ? (binaryMarsh ? new CollocatedSetItemKey(name, id, item) : new CollocatedItemKey(name, id, item))
+            : new GridCacheSetItemKey(id, item);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetItemKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetItemKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetItemKey.java
index d025dce..8b47b3d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetItemKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetItemKey.java
@@ -21,7 +21,6 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-import org.apache.ignite.internal.processors.cache.GridCacheInternal;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -30,7 +29,7 @@ import org.apache.ignite.lang.IgniteUuid;
 /**
  * Set item key.
  */
-public class GridCacheSetItemKey implements GridCacheInternal, Externalizable {
+public class GridCacheSetItemKey implements SetItemKey, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -57,27 +56,23 @@ public class GridCacheSetItemKey implements GridCacheInternal, Externalizable {
         this.item = item;
     }
 
-    /**
-     * @return Set UUID.
-     */
-    public IgniteUuid setId() {
+    /** {@inheritDoc} */
+    @Override public IgniteUuid setId() {
         return setId;
     }
 
-    /**
-     * @return Set item.
-     */
-    public Object item() {
+    /** {@inheritDoc} */
+    @Override public Object item() {
         return item;
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        int result = setId.hashCode();
+        int res = setId.hashCode();
 
-        result = 31 * result + item.hashCode();
+        res = 31 * res + item.hashCode();
 
-        return result;
+        return res;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridTransactionalCacheQueueImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridTransactionalCacheQueueImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridTransactionalCacheQueueImpl.java
index 4880324..32e94d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridTransactionalCacheQueueImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridTransactionalCacheQueueImpl.java
@@ -143,7 +143,7 @@ public class GridTransactionalCacheQueueImpl<T> extends GridCacheQueueAdapter<T>
                         if (idx != null) {
                             checkRemoved(idx);
 
-                            Map<GridCacheQueueItemKey, T> putMap = new HashMap<>();
+                            Map<QueueItemKey, T> putMap = new HashMap<>();
 
                             for (T item : items) {
                                 putMap.put(itemKey(idx), item);

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/QueueItemKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/QueueItemKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/QueueItemKey.java
new file mode 100644
index 0000000..fe0cef3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/QueueItemKey.java
@@ -0,0 +1,27 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.internal.processors.cache.GridCacheInternal;
+
+/**
+ *
+ */
+public interface QueueItemKey extends GridCacheInternal {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/SetItemKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/SetItemKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/SetItemKey.java
new file mode 100644
index 0000000..759945a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/SetItemKey.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.datastructures;
+
+import org.apache.ignite.internal.processors.cache.GridCacheInternal;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ *
+ */
+public interface SetItemKey extends GridCacheInternal {
+    /**
+     * @return Set UUID.
+     */
+    public IgniteUuid setId();
+
+    /**
+     * @return Set item.
+     */
+    public Object item();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeAbstractTest.java
index b881d90..51a70b9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeAbstractTest.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
@@ -139,6 +140,31 @@ public abstract class IgniteCacheInvokeAbstractTest extends IgniteCacheAbstractT
 
             tx = startTx(txMode);
 
+            TestValue testVal = cache.invoke(key, new UserClassValueProcessor());
+
+            if (tx != null)
+                tx.commit();
+
+            assertEquals("63", testVal.value());
+
+            checkValue(key, 63);
+
+            tx = startTx(txMode);
+
+            Collection<TestValue> testValCol = cache.invoke(key, new CollectionReturnProcessor());
+
+            if (tx != null)
+                tx.commit();
+
+            assertEquals(10, testValCol.size());
+
+            for (TestValue val : testValCol)
+                assertEquals("64", val.value());
+
+            checkValue(key, 63);
+
+            tx = startTx(txMode);
+
             GridTestUtils.assertThrows(log, new Callable<Void>() {
                 @Override public Void call() throws Exception {
                     cache.invoke(key, new ExceptionProcessor(63));
@@ -237,166 +263,226 @@ public abstract class IgniteCacheInvokeAbstractTest extends IgniteCacheAbstractT
 
         IncrementProcessor incProcessor = new IncrementProcessor();
 
-        Transaction tx = startTx(txMode);
+        {
+            Transaction tx = startTx(txMode);
 
-        Map<Integer, EntryProcessorResult<Integer>> resMap = cache.invokeAll(keys, incProcessor);
+            Map<Integer, EntryProcessorResult<Integer>> resMap = cache.invokeAll(keys, incProcessor);
 
-        if (tx != null)
-            tx.commit();
+            if (tx != null)
+                tx.commit();
 
-        Map<Object, Object> exp = new HashMap<>();
+            Map<Object, Object> exp = new HashMap<>();
 
-        for (Integer key : keys)
-            exp.put(key, -1);
+            for (Integer key : keys)
+                exp.put(key, -1);
 
-        checkResult(resMap, exp);
+            checkResult(resMap, exp);
 
-        for (Integer key : keys)
-            checkValue(key, 1);
+            for (Integer key : keys)
+                checkValue(key, 1);
+        }
 
-        tx = startTx(txMode);
+        {
+            Transaction tx = startTx(txMode);
 
-        resMap = cache.invokeAll(keys, incProcessor);
+            Map<Integer, EntryProcessorResult<TestValue>> resMap = cache.invokeAll(keys, new UserClassValueProcessor());
 
-        if (tx != null)
-            tx.commit();
+            if (tx != null)
+                tx.commit();
 
-        exp = new HashMap<>();
+            Map<Object, Object> exp = new HashMap<>();
 
-        for (Integer key : keys)
-            exp.put(key, 1);
+            for (Integer key : keys)
+                exp.put(key, new TestValue("1"));
 
-        checkResult(resMap, exp);
+            checkResult(resMap, exp);
 
-        for (Integer key : keys)
-            checkValue(key, 2);
+            for (Integer key : keys)
+                checkValue(key, 1);
+        }
 
-        tx = startTx(txMode);
+        {
+            Transaction tx = startTx(txMode);
 
-        resMap = cache.invokeAll(keys, new ArgumentsSumProcessor(), 10, 20, 30);
+            Map<Integer, EntryProcessorResult<Collection<TestValue>>> resMap =
+                cache.invokeAll(keys, new CollectionReturnProcessor());
 
-        if (tx != null)
-            tx.commit();
+            if (tx != null)
+                tx.commit();
 
-        for (Integer key : keys)
-            exp.put(key, 3);
+            Map<Object, Object> exp = new HashMap<>();
 
-        checkResult(resMap, exp);
+            for (Integer key : keys) {
+                List<TestValue> expCol = new ArrayList<>();
 
-        for (Integer key : keys)
-            checkValue(key, 62);
+                for (int i = 0; i < 10; i++)
+                    expCol.add(new TestValue("2"));
 
-        tx = startTx(txMode);
+                exp.put(key, expCol);
+            }
 
-        resMap = cache.invokeAll(keys, new ExceptionProcessor(null));
+            checkResult(resMap, exp);
 
-        if (tx != null)
-            tx.commit();
+            for (Integer key : keys)
+                checkValue(key, 1);
+        }
 
-        for (Integer key : keys) {
-            final EntryProcessorResult<Integer> res = resMap.get(key);
+        {
+            Transaction tx = startTx(txMode);
 
-            assertNotNull("No result for " + key);
+            Map<Integer, EntryProcessorResult<Integer>> resMap = cache.invokeAll(keys, incProcessor);
 
-            GridTestUtils.assertThrows(log, new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    res.get();
+            if (tx != null)
+                tx.commit();
 
-                    return null;
-                }
-            }, EntryProcessorException.class, "Test processor exception.");
+            Map<Object, Object> exp = new HashMap<>();
+
+            for (Integer key : keys)
+                exp.put(key, 1);
+
+            checkResult(resMap, exp);
+
+            for (Integer key : keys)
+                checkValue(key, 2);
         }
 
-        for (Integer key : keys)
-            checkValue(key, 62);
+        {
+            Transaction tx = startTx(txMode);
 
-        tx = startTx(txMode);
+            Map<Integer, EntryProcessorResult<Integer>> resMap =
+                cache.invokeAll(keys, new ArgumentsSumProcessor(), 10, 20, 30);
 
-        Map<Integer, EntryProcessor<Integer, Integer, Integer>> invokeMap = new HashMap<>();
+            if (tx != null)
+                tx.commit();
+
+            Map<Object, Object> exp = new HashMap<>();
+
+            for (Integer key : keys)
+                exp.put(key, 3);
+
+            checkResult(resMap, exp);
+
+            for (Integer key : keys)
+                checkValue(key, 62);
+        }
+
+        {
+            Transaction tx = startTx(txMode);
 
-        for (Integer key : keys) {
-            switch (key % 4) {
-                case 0: invokeMap.put(key, new IncrementProcessor()); break;
+            Map<Integer, EntryProcessorResult<Integer>> resMap = cache.invokeAll(keys, new ExceptionProcessor(null));
 
-                case 1: invokeMap.put(key, new RemoveProcessor(62)); break;
+            if (tx != null)
+                tx.commit();
 
-                case 2: invokeMap.put(key, new ArgumentsSumProcessor()); break;
+            for (Integer key : keys) {
+                final EntryProcessorResult<Integer> res = resMap.get(key);
 
-                case 3: invokeMap.put(key, new ExceptionProcessor(62)); break;
+                assertNotNull("No result for " + key);
 
-                default:
-                    fail();
+                GridTestUtils.assertThrows(log, new Callable<Void>() {
+                    @Override public Void call() throws Exception {
+                        res.get();
+
+                        return null;
+                    }
+                }, EntryProcessorException.class, "Test processor exception.");
             }
+
+            for (Integer key : keys)
+                checkValue(key, 62);
         }
 
-        resMap = cache.invokeAll(invokeMap, 10, 20, 30);
+        {
+            Transaction tx = startTx(txMode);
 
-        if (tx != null)
-            tx.commit();
+            Map<Integer, EntryProcessor<Integer, Integer, Integer>> invokeMap = new HashMap<>();
 
-        for (Integer key : keys) {
-            final EntryProcessorResult<Integer> res = resMap.get(key);
+            for (Integer key : keys) {
+                switch (key % 4) {
+                    case 0: invokeMap.put(key, new IncrementProcessor()); break;
 
-            switch (key % 4) {
-                case 0: {
-                    assertNotNull("No result for " + key, res);
+                    case 1: invokeMap.put(key, new RemoveProcessor(62)); break;
 
-                    assertEquals(62, (int)res.get());
+                    case 2: invokeMap.put(key, new ArgumentsSumProcessor()); break;
 
-                    checkValue(key, 63);
+                    case 3: invokeMap.put(key, new ExceptionProcessor(62)); break;
 
-                    break;
+                    default:
+                        fail();
                 }
+            }
 
-                case 1: {
-                    assertNull(res);
+            Map<Integer, EntryProcessorResult<Integer>> resMap = cache.invokeAll(invokeMap, 10, 20, 30);
 
-                    checkValue(key, null);
+            if (tx != null)
+                tx.commit();
 
-                    break;
-                }
+            for (Integer key : keys) {
+                final EntryProcessorResult<Integer> res = resMap.get(key);
 
-                case 2: {
-                    assertNotNull("No result for " + key, res);
+                switch (key % 4) {
+                    case 0: {
+                        assertNotNull("No result for " + key, res);
 
-                    assertEquals(3, (int)res.get());
+                        assertEquals(62, (int)res.get());
 
-                    checkValue(key, 122);
+                        checkValue(key, 63);
 
-                    break;
-                }
+                        break;
+                    }
+
+                    case 1: {
+                        assertNull(res);
+
+                        checkValue(key, null);
+
+                        break;
+                    }
+
+                    case 2: {
+                        assertNotNull("No result for " + key, res);
+
+                        assertEquals(3, (int)res.get());
+
+                        checkValue(key, 122);
 
-                case 3: {
-                    assertNotNull("No result for " + key, res);
+                        break;
+                    }
 
-                    GridTestUtils.assertThrows(log, new Callable<Void>() {
-                        @Override public Void call() throws Exception {
-                            res.get();
+                    case 3: {
+                        assertNotNull("No result for " + key, res);
 
-                            return null;
-                        }
-                    }, EntryProcessorException.class, "Test processor exception.");
+                        GridTestUtils.assertThrows(log, new Callable<Void>() {
+                            @Override public Void call() throws Exception {
+                                res.get();
 
-                    checkValue(key, 62);
+                                return null;
+                            }
+                        }, EntryProcessorException.class, "Test processor exception.");
 
-                    break;
+                        checkValue(key, 62);
+
+                        break;
+                    }
                 }
             }
         }
 
         cache.invokeAll(keys, new IncrementProcessor());
 
-        tx = startTx(txMode);
+        {
+            Transaction tx = startTx(txMode);
 
-        resMap = cache.invokeAll(keys, new RemoveProcessor(null));
+            Map<Integer, EntryProcessorResult<Integer>> resMap = cache.invokeAll(keys, new RemoveProcessor(null));
 
-        if (tx != null)
-            tx.commit();
+            if (tx != null)
+                tx.commit();
 
-        assertEquals("Unexpected results: " + resMap, 0, resMap.size());
+            assertEquals("Unexpected results: " + resMap, 0, resMap.size());
 
-        for (Integer key : keys)
-            checkValue(key, null);
+            for (Integer key : keys)
+                checkValue(key, null);
+        }
 
         IgniteCache<Integer, Integer> asyncCache = cache.withAsync();
 
@@ -406,9 +492,9 @@ public abstract class IgniteCacheInvokeAbstractTest extends IgniteCacheAbstractT
 
         IgniteFuture<Map<Integer, EntryProcessorResult<Integer>>> fut = asyncCache.future();
 
-        resMap = fut.get();
+        Map<Integer, EntryProcessorResult<Integer>> resMap = fut.get();
 
-        exp = new HashMap<>();
+        Map<Object, Object> exp = new HashMap<>();
 
         for (Integer key : keys)
             exp.put(key, -1);
@@ -418,7 +504,7 @@ public abstract class IgniteCacheInvokeAbstractTest extends IgniteCacheAbstractT
         for (Integer key : keys)
             checkValue(key, 1);
 
-        invokeMap = new HashMap<>();
+        Map<Integer, EntryProcessor<Integer, Integer, Integer>> invokeMap = new HashMap<>();
 
         for (Integer key : keys)
             invokeMap.put(key, incProcessor);
@@ -442,15 +528,16 @@ public abstract class IgniteCacheInvokeAbstractTest extends IgniteCacheAbstractT
      * @param resMap Result map.
      * @param exp Expected results.
      */
-    private void checkResult(Map<Integer, EntryProcessorResult<Integer>> resMap, Map<Object, Object> exp) {
+    @SuppressWarnings("unchecked")
+    private void checkResult(Map resMap, Map<Object, Object> exp) {
         assertNotNull(resMap);
 
         assertEquals(exp.size(), resMap.size());
 
         for (Map.Entry<Object, Object> expVal : exp.entrySet()) {
-            EntryProcessorResult<Integer> res = resMap.get(expVal.getKey());
+            EntryProcessorResult<?> res = (EntryProcessorResult)resMap.get(expVal.getKey());
 
-            assertNotNull("No result for " + expVal.getKey());
+            assertNotNull("No result for " + expVal.getKey(), res);
 
             assertEquals("Unexpected result for " + expVal.getKey(), res.get(), expVal.getValue());
         }
@@ -557,6 +644,44 @@ public abstract class IgniteCacheInvokeAbstractTest extends IgniteCacheAbstractT
     /**
      *
      */
+    protected static class UserClassValueProcessor implements EntryProcessor<Integer, Integer, TestValue> {
+        /** {@inheritDoc} */
+        @Override public TestValue process(MutableEntry<Integer, Integer> e, Object... arguments)
+            throws EntryProcessorException {
+            return new TestValue(String.valueOf(e.getValue()));
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(UserClassValueProcessor.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    protected static class CollectionReturnProcessor implements
+        EntryProcessor<Integer, Integer, Collection<TestValue>> {
+        /** {@inheritDoc} */
+        @Override public Collection<TestValue> process(MutableEntry<Integer, Integer> e, Object... arguments)
+            throws EntryProcessorException {
+            List<TestValue> vals = new ArrayList<>();
+
+            for (int i = 0; i < 10; i++)
+                vals.add(new TestValue(String.valueOf(e.getValue() + 1)));
+
+            return vals;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(CollectionReturnProcessor.class, this);
+        }
+    }
+
+    /**
+     *
+     */
     protected static class IncrementProcessor implements EntryProcessor<Integer, Integer, Integer> {
         /** {@inheritDoc} */
         @Override public Integer process(MutableEntry<Integer, Integer> e,
@@ -656,4 +781,50 @@ public abstract class IgniteCacheInvokeAbstractTest extends IgniteCacheAbstractT
             return S.toString(ExceptionProcessor.class, this);
         }
     }
+
+    /**
+     *
+     */
+    static class TestValue {
+        /** */
+        private String val;
+
+        /**
+         * @param val Value.
+         */
+        public TestValue(String val) {
+            this.val = val;
+        }
+
+        /**
+         * @return Value.
+         */
+        public String value() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestValue testVal = (TestValue) o;
+
+            return val.equals(testVal.val);
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestValue.class, this);
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
index 2751de1..ef96d9f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
@@ -170,11 +170,8 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig
             while (U.currentTimeMillis() < stopTime)
                 assertEquals(10, atomic.get());
         }
-        catch (IgniteException e) {
-            if (X.hasCause(e, ClusterTopologyServerNotFoundException.class))
-                return;
-
-            throw e;
+        catch (IgniteException ignore) {
+            return; // Test that client does not hang.
         }
 
         fail();

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
index cf638df..6366f09 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
@@ -244,15 +244,27 @@ public abstract class GridCacheQueueApiSelfAbstractTest extends IgniteCollection
     }
 
     /**
-     * JUnit.
-     *
      * @throws Exception If failed.
      */
     public void testIterator() throws Exception {
+        checkIterator(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIteratorCollocated() throws Exception {
+        checkIterator(true);
+    }
+
+    /**
+     * @param collocated Collocated flag.
+     */
+    private void checkIterator(boolean collocated) {
         // Random queue name.
         String queueName = UUID.randomUUID().toString();
 
-        IgniteQueue<String> queue = grid(0).queue(queueName, 0, config(false));
+        IgniteQueue<String> queue = grid(0).queue(queueName, 0, config(collocated));
 
         for (int i = 0; i < 100; i++)
             assert queue.add(Integer.toString(i));

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
index 74c9a4f..ca57205 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
@@ -31,7 +31,7 @@ import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.datastructures.GridCacheSetItemKey;
+import org.apache.ignite.internal.processors.datastructures.SetItemKey;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -183,8 +183,8 @@ public abstract class GridCacheSetFailoverAbstractSelfTest extends IgniteCollect
                     if (entry.hasValue()) {
                         cnt++;
 
-                        if (entry.key() instanceof GridCacheSetItemKey) {
-                            GridCacheSetItemKey setItem = (GridCacheSetItemKey)entry.key();
+                        if (entry.key() instanceof SetItemKey) {
+                            SetItemKey setItem = (SetItemKey)entry.key();
 
                             if (setIds.add(setItem.setId()))
                                 log.info("Unexpected set item [setId=" + setItem.setId() +

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueApiSelfTest.java
index 2420153..de2fa07 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueApiSelfTest.java
@@ -31,6 +31,11 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
  */
 public class GridCachePartitionedQueueApiSelfTest extends GridCacheQueueApiSelfAbstractTest {
     /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode collectionCacheMode() {
         return PARTITIONED;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
index 1d225a6..db11291 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
@@ -90,7 +90,7 @@ public class GridCachePartitionedQueueEntryMoveSelfTest extends IgniteCollection
      * @throws Exception If failed.
      */
     public void testQueue() throws Exception {
-        final String queueName = "queue-test-name";
+        final String queueName = "q";
 
         System.out.println(U.filler(20, '\n'));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedQueueNoBackupsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedQueueNoBackupsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedQueueNoBackupsTest.java
new file mode 100644
index 0000000..880c638
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedQueueNoBackupsTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.datastructures.partitioned;
+
+import java.util.Iterator;
+import java.util.UUID;
+import org.apache.ignite.IgniteQueue;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CollectionConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.testframework.GridTestUtils;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ *
+ */
+public class IgnitePartitionedQueueNoBackupsTest extends GridCachePartitionedQueueApiSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode collectionCacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode collectionMemoryMode() {
+        return ONHEAP_TIERED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode collectionCacheAtomicityMode() {
+        return ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CollectionConfiguration collectionConfiguration() {
+        CollectionConfiguration colCfg = super.collectionConfiguration();
+
+        colCfg.setBackups(0);
+
+        return colCfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCollocation() throws Exception {
+        IgniteQueue<Integer> queue = grid(0).queue("queue", 0, config(true));
+
+        for (int i = 0; i < 1000; i++)
+            assertTrue(queue.add(i));
+
+        assertEquals(1000, queue.size());
+
+        GridCacheContext cctx = GridTestUtils.getFieldValue(queue, "cctx");
+
+        UUID setNodeId = null;
+
+        for (int i = 0; i < gridCount(); i++) {
+            IgniteKernal grid = (IgniteKernal)grid(i);
+
+            Iterator<GridCacheEntryEx> entries =
+                grid.context().cache().internalCache(cctx.name()).map().allEntries0().iterator();
+
+            if (entries.hasNext()) {
+                if (setNodeId == null)
+                    setNodeId = grid.localNode().id();
+                else
+                    fail("For collocated queue all items should be stored on single node.");
+            }
+        }
+    }}

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedQueueApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedQueueApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedQueueApiSelfTest.java
index 1aea6d9..bad37a9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedQueueApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedQueueApiSelfTest.java
@@ -31,6 +31,11 @@ import static org.apache.ignite.cache.CacheMode.REPLICATED;
  */
 public class GridCacheReplicatedQueueApiSelfTest extends GridCacheQueueApiSelfAbstractTest {
     /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode collectionCacheMode() {
         return REPLICATED;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/50f6c013/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractTest.java
index 4a5141e..e9674f3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractTest.java
@@ -114,7 +114,7 @@ public abstract class GridCacheWriteBehindStoreAbstractTest extends GridCommonAb
 
         Map<Integer, String> map = store.getMap();
 
-        assert map.isEmpty();
+        assert map.isEmpty() : map;
 
         Transaction tx = grid().transactions().txStart(OPTIMISTIC, REPEATABLE_READ);
 


[25/50] [abbrv] ignite git commit: 2064 Test hotfix

Posted by sb...@apache.org.
2064 Test hotfix

Signed-off-by: Anton Vinogradov <av...@apache.org>


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

Branch: refs/heads/ignite-1.5.1
Commit: 11b426297cffcb98b6dd374cfc88184b2b04246f
Parents: 233231d
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Dec 4 19:34:46 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Dec 7 13:08:39 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java | 6 +++++-
 .../apache/ignite/testframework/config/GridTestProperties.java | 3 +++
 .../ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java  | 2 ++
 3 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/11b42629/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java
index 0873d2d..35ee6cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.config.GridTestProperties;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
@@ -45,7 +46,10 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** Entry processor */
-    protected static String TEST_ENT_PROCESSOR = "org.apache.ignite.tests.p2p.CacheDeploymentEntryProcessor";
+    protected static String TEST_ENT_PROCESSOR =
+        GridTestProperties.getProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME) != null ?
+            GridTestProperties.getProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME) :
+            "org.apache.ignite.tests.p2p.CacheDeploymentEntryProcessor";
 
     /** Test value. */
     protected static String TEST_VALUE = "org.apache.ignite.tests.p2p.CacheDeploymentTestValue";

http://git-wip-us.apache.org/repos/asf/ignite/blob/11b42629/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
index 1ea8c38..491f38f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
@@ -72,6 +72,9 @@ public final class GridTestProperties {
     /** */
     public static final String MARSH_CLASS_NAME = "marshaller.class";
 
+    /** */
+    public static final String ENTRY_PROCESSOR_CLASS_NAME = "entry.processor.class";
+
     /** Binary marshaller compact footers property. */
     public static final String BINARY_COMPACT_FOOTERS = "binary.marshaller.compact.footers";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/11b42629/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java
index 2778c97..3d25645 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java
@@ -31,6 +31,8 @@ public class IgniteBinaryObjectsCacheTestSuite3 {
      */
     public static TestSuite suite() throws Exception {
         GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+        GridTestProperties.setProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME,
+            "org.apache.ignite.tests.p2p.CacheDeploymentPortableEntryProcessor");
 
         return IgniteCacheTestSuite3.suite();
     }


[14/50] [abbrv] ignite git commit: ignite-2008 Call removeExplicitNodeLocks before waiting for partitionReleaseFuture.

Posted by sb...@apache.org.
ignite-2008 Call removeExplicitNodeLocks before waiting for partitionReleaseFuture.


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

Branch: refs/heads/ignite-1.5.1
Commit: baab3d2b6e047f32f33f1174474683096f2eb373
Parents: 171e9b1
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 4 09:04:00 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 4 09:04:00 2015 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionsExchangeFuture.java      | 9 ++++-----
 .../distributed/dht/GridCacheTxNodeFailureSelfTest.java     | 2 ++
 .../distributed/dht/GridNearCacheTxNodeFailureSelfTest.java | 4 ----
 .../org/apache/ignite/testsuites/IgniteCacheTestSuite2.java | 2 ++
 4 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/baab3d2b/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 f7e6acc..31bfa79 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
@@ -754,6 +754,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 // Assign to class variable so it will be included into toString() method.
                 this.partReleaseFut = partReleaseFut;
 
+                if (exchId.isLeft())
+                    cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
+
                 if (log.isDebugEnabled())
                     log.debug("Before waiting for partition release future: " + this);
 
@@ -778,9 +781,6 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 if (log.isDebugEnabled())
                     log.debug("After waiting for partition release future: " + this);
 
-                if (exchId.isLeft())
-                    cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
-
                 IgniteInternalFuture<?> locksFut = cctx.mvcc().finishLocks(exchId.topologyVersion());
 
                 dumpedObjects = 0;
@@ -1015,9 +1015,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 if (ready) {
                     GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true);
 
-                    if (useOldApi) {
+                    if (useOldApi)
                         locMap = new GridDhtPartitionFullMap(locMap.nodeId(), locMap.nodeOrder(), locMap.updateSequence(), locMap);
-                    }
 
                     m.addFullPartitionsMap(cacheCtx.cacheId(), locMap);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/baab3d2b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
index fe0b84e..78e7672 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
@@ -295,6 +295,8 @@ public class GridCacheTxNodeFailureSelfTest extends GridCommonAbstractTest {
             // Check that thread successfully finished.
             fut.get();
 
+            awaitPartitionMapExchange();
+
             // Check there are no hanging transactions.
             assertEquals(0, ((IgniteEx)ignite(0)).context().cache().context().tm().idMapSize());
             assertEquals(0, ((IgniteEx)ignite(2)).context().cache().context().tm().idMapSize());

http://git-wip-us.apache.org/repos/asf/ignite/blob/baab3d2b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNearCacheTxNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNearCacheTxNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNearCacheTxNodeFailureSelfTest.java
index ca23646..5735182 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNearCacheTxNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNearCacheTxNodeFailureSelfTest.java
@@ -28,8 +28,4 @@ public class GridNearCacheTxNodeFailureSelfTest extends GridCacheTxNodeFailureSe
     @Override protected CacheConfiguration cacheConfiguration(String gridName) {
         return super.cacheConfiguration(gridName).setNearConfiguration(new NearCacheConfiguration());
     }
-
-    @Override public void testPrimaryNodeFailureBackupCommitImplicit(){
-        fail("https://issues.apache.org/jira/browse/IGNITE-1611");
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/baab3d2b/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 74b688f..c94931e 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
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheEntryProcessorNode
 import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdateTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheLockReleaseNodeLeaveTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionNotLoadedEventSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionedNearDisabledTxMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTransformEventSelfTest;
@@ -244,6 +245,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridCacheNearTxForceKeyTest.class));
         suite.addTest(new TestSuite(CrossCacheTxRandomOperationsTest.class));
         suite.addTest(new TestSuite(IgniteDynamicCacheAndNodeStop.class));
+        suite.addTest(new TestSuite(CacheLockReleaseNodeLeaveTest.class));
 
         return suite;
     }


[30/50] [abbrv] ignite git commit: HashMap's offset retrieval in HashSet class definition for legacy IBM JRE

Posted by sb...@apache.org.
HashMap's offset retrieval in HashSet class definition for legacy IBM JRE


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

Branch: refs/heads/ignite-1.5.1
Commit: c077522038d050aac35ce034cc86d9af7959ebd5
Parents: ebddcb1
Author: Denis Magda <dm...@gridgain.com>
Authored: Mon Dec 7 20:52:00 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Dec 7 20:52:00 2015 +0300

----------------------------------------------------------------------
 .../optimized/OptimizedMarshallerUtils.java         | 16 +++++++++++++---
 1 file changed, 13 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c0775220/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
index 4d6afe6..da92b90 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
@@ -155,12 +155,22 @@ class OptimizedMarshallerUtils {
     static final JdkMarshaller JDK_MARSH = new JdkMarshaller();
 
     static {
+        long mapOff;
+
         try {
-            HASH_SET_MAP_OFF = UNSAFE.objectFieldOffset(HashSet.class.getDeclaredField("map"));
+            mapOff = UNSAFE.objectFieldOffset(HashSet.class.getDeclaredField("map"));
         }
         catch (NoSuchFieldException e) {
-            throw new IgniteException("Initialization failure.", e);
+            try {
+                // Workaround for legacy IBM JRE.
+                mapOff = UNSAFE.objectFieldOffset(HashSet.class.getDeclaredField("backingMap"));
+            }
+            catch (NoSuchFieldException e2) {
+                throw new IgniteException("Initialization failure.", e2);
+            }
         }
+
+        HASH_SET_MAP_OFF = mapOff;
     }
 
     /**
@@ -537,4 +547,4 @@ class OptimizedMarshallerUtils {
     static void setObject(Object obj, long off, Object val) {
         UNSAFE.putObject(obj, off, val);
     }
-}
\ No newline at end of file
+}


[27/50] [abbrv] ignite git commit: IGNITE-2090 .Net: EventsTest.TestSerialization fails

Posted by sb...@apache.org.
IGNITE-2090 .Net: EventsTest.TestSerialization fails


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

Branch: refs/heads/ignite-1.5.1
Commit: c10b112ad0693035d6bb2b1c782fe09f11aaded4
Parents: 34596c6
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Mon Dec 7 13:42:17 2015 +0300
Committer: Pavel Tupitsyn <pt...@gridgain.com>
Committed: Mon Dec 7 13:42:17 2015 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c10b112a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
index 9f22355..a538cb4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
@@ -568,7 +568,8 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(EventType.SwapSpaceCleared, evt.Type);
             Assert.IsNotNullOrEmpty(evt.Name);
             Assert.AreNotEqual(Guid.Empty, evt.Id.GlobalId);
-            Assert.IsTrue((evt.Timestamp - DateTime.Now).TotalSeconds < 10);
+            Assert.IsTrue(Math.Abs((evt.Timestamp - DateTime.UtcNow).TotalSeconds) < 20, 
+                "Invalid event timestamp: '{0}', current time: '{1}'", evt.Timestamp, DateTime.Now);
         }
 
         /// <summary>


[44/50] [abbrv] ignite git commit: 2064 Issue url fix

Posted by sb...@apache.org.
2064 Issue url fix


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

Branch: refs/heads/ignite-1.5.1
Commit: c30def8f049bddb4ec272528326f84bf90a1399b
Parents: 0e81a39
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Dec 8 16:48:58 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Dec 8 16:48:58 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/c30def8f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
index aa64959..bfa353b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
@@ -228,7 +228,7 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest {
             Ignite g1 = startGrid(1);
 
             if (g1.configuration().getMarshaller() instanceof BinaryMarshaller)
-                fail("http://atlassian.gridgain.com/jira/browse/GG-10791");
+                fail("https://issues.apache.org/jira/browse/IGNITE-2106");
 
             Ignite g2 = startGrid(2);
 


[04/50] [abbrv] ignite git commit: IGNITE-2074 Platforms C++ files have 1.5.0-EA version

Posted by sb...@apache.org.
IGNITE-2074 Platforms C++ files have 1.5.0-EA version

Signed-off-by: Anton Vinogradov <av...@apache.org>


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

Branch: refs/heads/ignite-1.5.1
Commit: 86ec37e5e7bb6302ebff35fecb5576206adc1e1a
Parents: ad9e4db
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Thu Dec 3 12:45:29 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 3 14:48:00 2015 +0300

----------------------------------------------------------------------
 pom.xml | 41 +++++++++++++++++++++++++++++------------
 1 file changed, 29 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/86ec37e5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 109dc94..5f86648 100644
--- a/pom.xml
+++ b/pom.xml
@@ -774,20 +774,37 @@
                                     <target>
                                         <script language="javascript">
                                             function setClientVersion(ggVer, clientVer) {
-                                            var p = project.getProperty(ggVer);
+                                                var p = project.getProperty(ggVer);
 
-                                            if (java.util.regex.Pattern.matches(".*-p\\d+", p))
-                                            project.setProperty(clientVer, p.replace("-p", "."));
-                                            else
-                                            if (java.util.regex.Pattern.matches(".*-[a-zA-Z]+\\d+.*", p))
-                                            project.setProperty(clientVer, p.replaceAll("-[a-zA-Z]+(\\d+)?.*", ".$1"));
-                                            else
-                                            project.setProperty(clientVer, p);
+                                                var pos = p.search("-");
+
+                                                if (pos > 0)
+                                                {
+                                                    var suffix = p.substring(pos);
+
+                                                    var ver = 0;
+
+                                                    var beta = /-b([0-9]+)/.exec(suffix);
+                                                    if (beta !== null)
+                                                        ver += parseInt(beta[1]);
+
+                                                    var patch = /-p([0-9]+)/.exec(suffix);
+                                                    if (patch !== null)
+                                                        ver += parseInt(patch[1]) * 100;
+
+                                                    if (suffix.search("final") > 0)
+                                                        ver += 10000;
+
+                                                    var resVer = p.substring(0, pos) +"." + ver;
+                                                    project.setProperty(clientVer, resVer);
+                                                }
+                                                else
+                                                    project.setProperty(clientVer, p);
                                             }
 
                                             function fix(dest, source) {
-                                            project.setProperty(dest, project.getProperty(source).replace("-SNAPSHOT",
-                                            ""));
+                                                project.setProperty(dest, project.getProperty(source).replace("-SNAPSHOT",
+                                                ""));
                                             }
 
                                             fix('ignite.version.fixed', 'project.version');
@@ -810,8 +827,8 @@
 
                                         <echo message="Update ignite.version in cpp client" />
                                         <replaceregexp byline="true" encoding="UTF-8">
-                                            <regexp pattern="(AC_INIT.+\[)\d.\d.\d(.\d)?(\].+)" />
-                                            <substitution expression="\1${new.client.version}\3" />
+                                            <regexp pattern="(AC_INIT.+\[)\d.\d.\d.*?(\].+)" />
+                                            <substitution expression="\1${new.client.version}\2" />
                                             <fileset dir="${basedir}/">
                                                 <include name="**/configure.ac" />
                                             </fileset>


[21/50] [abbrv] ignite git commit: ignite-1.5 - release notes

Posted by sb...@apache.org.
ignite-1.5 - release notes


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

Branch: refs/heads/ignite-1.5.1
Commit: 23ead500762095300417759a30c08cd76daf210d
Parents: 92ef7c7
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Dec 4 17:28:50 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Dec 4 17:28:50 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/23ead500/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index f7f48fa..243ec18 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -11,6 +11,7 @@ Apache Ignite In-Memory Data Fabric 1.5
 * Added MQTT Streamer.
 * Added Twitter Streamer.
 * Added Ignite Sink (integration with Apache Flume).
+* Fixed optimistic serializable transactions: implemented "deadlock-free" locking algorithm.
 * Fixed failover for continuous queries.
 * Fixed compilation and runtime errors under OpenJDK and IBM JDK.
 * Fixed Integer.size limitation for cache.


[46/50] [abbrv] ignite git commit: ignite-1.5 Cache map refactoring. This closes #279.

Posted by sb...@apache.org.
ignite-1.5 Cache map refactoring. This closes #279.


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

Branch: refs/heads/ignite-1.5.1
Commit: 3baf4d163c6edafa7367ece7901ad1efe695a8d4
Parents: c30def8
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 8 17:30:14 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 8 17:30:14 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  65 +-
 .../cache/GridCacheConcurrentMap.java           | 734 +++++--------------
 .../processors/cache/GridCacheMapEntry.java     |  44 +-
 .../cache/GridCacheMapEntryFactory.java         |  13 +-
 .../distributed/GridDistributedCacheEntry.java  |  14 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  13 +-
 .../distributed/dht/GridDhtCacheEntry.java      |  14 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |  15 +-
 .../distributed/dht/GridNoStorageCacheMap.java  |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  23 +-
 .../dht/atomic/GridDhtAtomicCacheEntry.java     |  16 +-
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |  15 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  13 +-
 .../colocated/GridDhtColocatedCacheEntry.java   |  14 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |  15 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |   4 +-
 .../distributed/near/GridNearCacheAdapter.java  |   8 +-
 .../distributed/near/GridNearCacheEntry.java    |  16 +-
 .../near/GridNearOffHeapCacheEntry.java         |  15 +-
 .../processors/cache/local/GridLocalCache.java  |   8 +-
 .../cache/local/GridLocalCacheEntry.java        |  15 +-
 .../local/atomic/GridLocalAtomicCache.java      |   8 +-
 .../ignite/internal/util/IgniteUtils.java       |  18 +-
 .../cache/GridCacheAlwaysEvictionPolicy.java    |   2 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |  11 -
 .../loadtests/hashmap/GridHashMapLoadTest.java  |   4 +-
 26 files changed, 357 insertions(+), 764 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 f96954e..034640f 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
@@ -925,25 +925,60 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      */
     @Nullable private GridCacheEntryEx entry0(KeyCacheObject key, AffinityTopologyVersion topVer, boolean create,
         boolean touch) {
-        GridTriple<GridCacheMapEntry> t = map.putEntryIfObsoleteOrAbsent(topVer, key, null, create);
+        GridCacheMapEntry cur = map.getEntry(key);
 
-        GridCacheEntryEx cur = t.get1();
-        GridCacheEntryEx created = t.get2();
-        GridCacheEntryEx doomed = t.get3();
+        if (cur == null || cur.obsolete()) {
+            GridTriple<GridCacheMapEntry> t = map.putEntryIfObsoleteOrAbsent(
+                topVer,
+                key,
+                null,
+                create);
 
-        if (doomed != null && ctx.events().isRecordable(EVT_CACHE_ENTRY_DESTROYED))
-            // Event notification.
-            ctx.events().addEvent(doomed.partition(), doomed.key(), locNodeId, (IgniteUuid)null, null,
-                EVT_CACHE_ENTRY_DESTROYED, null, false, null, false, null, null, null, true);
+            cur = t.get1();
 
-        if (created != null) {
-            // Event notification.
-            if (ctx.events().isRecordable(EVT_CACHE_ENTRY_CREATED))
-                ctx.events().addEvent(created.partition(), created.key(), locNodeId, (IgniteUuid)null, null,
-                    EVT_CACHE_ENTRY_CREATED, null, false, null, false, null, null, null, true);
+            GridCacheEntryEx created = t.get2();
+            GridCacheEntryEx doomed = t.get3();
 
-            if (touch)
-                ctx.evicts().touch(cur, topVer);
+            if (doomed != null && ctx.events().isRecordable(EVT_CACHE_ENTRY_DESTROYED))
+                // Event notification.
+                ctx.events().addEvent(doomed.partition(),
+                    doomed.key(),
+                    locNodeId,
+                    (IgniteUuid)null,
+                    null,
+                    EVT_CACHE_ENTRY_DESTROYED,
+                    null,
+                    false,
+                    null,
+                    false,
+                    null,
+                    null,
+                    null,
+                    true);
+
+            if (created != null) {
+                // Event notification.
+                if (ctx.events().isRecordable(EVT_CACHE_ENTRY_CREATED))
+                    ctx.events().addEvent(created.partition(),
+                        created.key(),
+                        locNodeId,
+                        (IgniteUuid)null,
+                        null,
+                        EVT_CACHE_ENTRY_CREATED,
+                        null,
+                        false,
+                        null,
+                        false,
+                        null,
+                        null,
+                        null,
+                        true);
+
+                if (touch)
+                    ctx.evicts().touch(
+                        cur,
+                        topVer);
+            }
         }
 
         return cur;

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
index 1c64387..f78a606 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
@@ -22,9 +22,6 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.io.ObjectStreamException;
-import java.lang.ref.Reference;
-import java.lang.ref.ReferenceQueue;
-import java.lang.ref.WeakReference;
 import java.lang.reflect.Array;
 import java.util.AbstractCollection;
 import java.util.AbstractSet;
@@ -35,7 +32,6 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Random;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantLock;
 import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
@@ -45,6 +41,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.lang.GridTriple;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
 import org.apache.ignite.internal.util.typedef.X;
@@ -55,7 +52,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentHashMap8;
 import org.jsr166.LongAdder8;
 
 /**
@@ -127,82 +123,6 @@ public class GridCacheConcurrentMap {
             }
         };
 
-    /** Soft iterator queue. */
-    private final ReferenceQueue<Iterator0> itQ = new ReferenceQueue<>();
-
-    /** Soft iterator set. */
-    private final Map<WeakIterator<KeyCacheObject, CacheObject>, SegmentHeader> itMap =
-        new ConcurrentHashMap8<>();
-
-    /**
-     * Checks phantom queue.
-     */
-    private void checkWeakQueue() {
-        // If queue is empty, then it is a lock-free volatile read which should happen
-        // in most cases. Otherwise queue uses synchronization to poll elements.
-        for (Reference<? extends Iterator0> itRef = itQ.poll();
-             itRef != null; itRef = itQ.poll()) {
-            assert itRef instanceof WeakIterator;
-
-            if (DEBUG)
-                X.println("Checking weak queue [itSetSize=" + itMap.size() + ']');
-
-            SegmentHeader lastSeg = removeWeakIterator(itRef);
-
-            // Segment may be null if iterator is empty at creation time.
-            if (lastSeg != null)
-                lastSeg.onReadEnd();
-        }
-    }
-
-    /**
-     * @param itRef Iterator reference.
-     * @return Last segment.
-     */
-    private SegmentHeader removeWeakIterator(
-        Reference<? extends Iterator0> itRef) {
-        assert itRef instanceof WeakIterator;
-
-        SegmentHeader hdr = itMap.remove(itRef);
-
-        if (DEBUG) {
-            if (hdr == null)
-                X.println("Removed non-existent iterator: " + itRef);
-            else
-                X.println("Removed iterator [hdrId=" + hdr.id() + ", it=" + itRef + ", mapSize=" + itMap.size() + ']');
-        }
-
-        return hdr;
-    }
-
-    /**
-     * @param itRef Iterator reference.
-     * @param hdr Segment header.
-     */
-    private void addWeakIterator(WeakIterator itRef, SegmentHeader hdr) {
-        SegmentHeader prev = itMap.put(itRef, hdr);
-
-        if (DEBUG)
-            if (prev == null)
-                X.println("Added weak reference: " + itMap.size());
-    }
-
-
-
-    /**
-     * @return Iterator set size.
-     */
-    int iteratorMapSize() {
-        return itMap.size();
-    }
-
-    /**
-     * @return Reference queue for iterators.
-     */
-    private ReferenceQueue<Iterator0> iteratorQueue() {
-        return itQ;
-    }
-
     /**
      * Applies a supplemental hash function to a given hashCode, which
      * defends against poor quality hash functions.  This is critical
@@ -408,8 +328,6 @@ public class GridCacheConcurrentMap {
      * @return {@code True} if map contains mapping for provided key.
      */
     public boolean containsKey(Object key) {
-        checkWeakQueue();
-
         int hash = hash(key.hashCode());
 
         return segmentFor(hash).containsKey(key, hash);
@@ -422,8 +340,6 @@ public class GridCacheConcurrentMap {
      * @return a collection view of the values contained in this map.
      */
     public <K, V> Collection<V> allValues(CacheEntryPredicate[] filter) {
-        checkWeakQueue();
-
         return new Values<>(this, filter);
     }
 
@@ -431,8 +347,6 @@ public class GridCacheConcurrentMap {
      * @return Random entry out of hash map.
      */
     @Nullable public GridCacheMapEntry randomEntry() {
-        checkWeakQueue();
-
         while (true) {
             if (mapPubSize.sum() == 0)
                 return null;
@@ -475,8 +389,6 @@ public class GridCacheConcurrentMap {
     @Nullable public GridCacheMapEntry getEntry(Object key) {
         assert key != null;
 
-        checkWeakQueue();
-
         int hash = hash(key.hashCode());
 
         return segmentFor(hash).get(key, hash);
@@ -491,8 +403,6 @@ public class GridCacheConcurrentMap {
     public GridCacheMapEntry putEntry(AffinityTopologyVersion topVer, KeyCacheObject key, @Nullable CacheObject val) {
         assert key != null;
 
-        checkWeakQueue();
-
         int hash = hash(key.hashCode());
 
         return segmentFor(hash).put(key, hash, val, topVer);
@@ -514,8 +424,6 @@ public class GridCacheConcurrentMap {
     {
         assert key != null;
 
-        checkWeakQueue();
-
         int hash = hash(key.hashCode());
 
         return segmentFor(hash).putIfObsolete(key, hash, val, topVer, create);
@@ -543,8 +451,6 @@ public class GridCacheConcurrentMap {
     public boolean removeEntry(GridCacheEntryEx e) {
         assert e != null;
 
-        checkWeakQueue();
-
         KeyCacheObject key = e.key();
 
         int hash = hash(key.hashCode());
@@ -576,8 +482,6 @@ public class GridCacheConcurrentMap {
     @Nullable public GridCacheMapEntry removeEntryIfObsolete(KeyCacheObject key) {
         assert key != null;
 
-        checkWeakQueue();
-
         int hash = hash(key.hashCode());
 
         return segmentFor(hash).remove(key, hash, obsolete);
@@ -589,8 +493,6 @@ public class GridCacheConcurrentMap {
      */
     @SuppressWarnings({"unchecked"})
     public <K, V> Set<Cache.Entry<K, V>> entries(CacheEntryPredicate... filter) {
-        checkWeakQueue();
-
         return new EntrySet<>(this, filter);
     }
 
@@ -602,8 +504,6 @@ public class GridCacheConcurrentMap {
      */
     @SuppressWarnings({"unchecked"})
     public <K, V> Set<Cache.Entry<K, V>> entriesx(CacheEntryPredicate... filter) {
-        checkWeakQueue();
-
         return new EntrySet<>(this, filter, true);
     }
 
@@ -613,8 +513,6 @@ public class GridCacheConcurrentMap {
      * @return Set of the mappings contained in this map.
      */
     public Set<GridCacheEntryEx> entries0() {
-        checkWeakQueue();
-
         return new Set0<>(this, GridCacheConcurrentMap.nonInternal());
     }
 
@@ -626,8 +524,6 @@ public class GridCacheConcurrentMap {
      * @return Striped entry iterator.
      */
     public Iterator<GridCacheEntryEx> stripedEntryIterator(int id, int totalCnt) {
-        checkWeakQueue();
-
         return new Iterator0<>(this, false, GridCacheConcurrentMap.nonInternal(), id, totalCnt);
     }
 
@@ -637,8 +533,6 @@ public class GridCacheConcurrentMap {
      * @return All internal entry set, including {@link GridCacheInternal} entries.
      */
     public Set<GridCacheEntryEx> allEntries0() {
-        checkWeakQueue();
-
         return new Set0<>(this, CU.empty0());
     }
 
@@ -649,8 +543,6 @@ public class GridCacheConcurrentMap {
      * @return Set of the keys contained in this map.
      */
     public <K, V> Set<K> keySet(CacheEntryPredicate... filter) {
-        checkWeakQueue();
-
         return new KeySet<>(this, filter, false);
     }
 
@@ -661,8 +553,6 @@ public class GridCacheConcurrentMap {
      * @return Set of the keys contained in this map.
      */
     public <K, V> Set<K> keySetx(CacheEntryPredicate... filter) {
-        checkWeakQueue();
-
         return new KeySet<>(this, filter, true);
     }
 
@@ -673,8 +563,6 @@ public class GridCacheConcurrentMap {
      * @return Collection view of the values contained in this map.
      */
     public <K, V> Collection<V> values(CacheEntryPredicate... filter) {
-        checkWeakQueue();
-
         return allValues(filter);
     }
 
@@ -699,9 +587,7 @@ public class GridCacheConcurrentMap {
 
                 X.println("    Segment [idx=" + i + ", size=" + seg.size() + ']');
 
-                SegmentHeader segHdr = seg.hdr;
-
-                GridCacheMapEntry[] tab = segHdr.table();
+                HashEntry[] tab = seg.tbl;
 
                 for (int j = 0; j < tab.length; j++)
                     X.println("        Bucket [idx=" + j + ", bucket=" + tab[j] + ']');
@@ -725,13 +611,13 @@ public class GridCacheConcurrentMap {
         IgniteLogger log = ctx.logger(GridCacheConcurrentMap.class);
 
         for (Segment s : segs) {
-            SegmentHeader segHdr = s.hdr;
+            assert s.isHeldByCurrentThread();
 
-            GridCacheMapEntry[] tab = segHdr.table();
+            HashEntry[] tab = s.tbl;
 
-            for (GridCacheMapEntry b : tab) {
+            for (HashEntry b : tab) {
                 if (b != null) {
-                    GridCacheMapEntry e = b;
+                    HashEntry e = b;
 
                     assert e != null;
 
@@ -743,12 +629,12 @@ public class GridCacheConcurrentMap {
 
                         log.info("Cache map entry: " + e);
 
-                        if (!e.deleted()) {
-                            if (!(e.key instanceof GridCacheInternal))
+                        if (!e.mapEntry.deleted()) {
+                            if (!(e.mapEntry.key instanceof GridCacheInternal))
                                 pubCnt++;
                         }
 
-                        e = e.next(segHdr.id());
+                        e = e.next;
                     }
 
                     size += cnt;
@@ -813,9 +699,6 @@ public class GridCacheConcurrentMap {
          */
         private int threshold;
 
-        /** Segment header. */
-        private volatile SegmentHeader hdr;
-
         /** The number of public elements in this segment's region. */
         private final LongAdder8 pubSize = new LongAdder8();
 
@@ -827,6 +710,12 @@ public class GridCacheConcurrentMap {
          */
         private final float loadFactor;
 
+        /** Entry table. */
+        private volatile HashEntry[] tbl;
+
+        /** The number of elements in this segment's region. */
+        private volatile int size;
+
         /**
          * @param initCap Initial capacity.
          * @param lf Load factor.
@@ -835,39 +724,19 @@ public class GridCacheConcurrentMap {
         Segment(int initCap, float lf) {
             loadFactor = lf;
 
-            hdr = new SegmentHeader(initCap, 0, null);
+            tbl = new HashEntry[initCap];
 
-            threshold = (int)(hdr.length() * loadFactor);
+            threshold = (int)(initCap * loadFactor);
         }
 
         /**
          * Returns properly casted first entry for given hash.
          *
-         * @param tbl Table.
          * @param hash Hash.
          * @return Entry for hash.
          */
-        @Nullable GridCacheMapEntry getFirst(GridCacheMapEntry[] tbl, int hash) {
-            GridCacheMapEntry bin = tbl[hash & (tbl.length - 1)];
-
-            return bin != null ? bin : null;
-        }
-
-        /**
-         * @return Segment header for read operation.
-         */
-        private SegmentHeader headerForRead() {
-            while (true) {
-                SegmentHeader hdr = this.hdr;
-
-                hdr.onReadStart();
-
-                // Check if 2 rehashes didn't happen in between.
-                if (hdr == this.hdr)
-                    return hdr;
-                else
-                    hdr.onReadEnd();
-            }
+        @Nullable HashEntry getFirst(HashEntry[] tbl, int hash) {
+            return tbl[hash & (tbl.length - 1)];
         }
 
         /**
@@ -876,25 +745,18 @@ public class GridCacheConcurrentMap {
          * @return Value.
          */
         @Nullable GridCacheMapEntry get(Object key, int hash) {
-            SegmentHeader hdr = headerForRead();
-
-            try {
-                if (hdr.size() != 0) {
-                    GridCacheMapEntry e = getFirst(hdr.table(), hash);
+            if (size != 0) {
+                HashEntry e = getFirst(tbl, hash);
 
-                    while (e != null) {
-                        if (e.hash() == hash && key.equals(e.key()))
-                            return e;
+                while (e != null) {
+                    if (e.mapEntry.hash() == hash && key.equals(e.mapEntry.key()))
+                        return e.mapEntry;
 
-                        e = e.next(hdr.id());
-                    }
+                    e = e.next;
                 }
-
-                return null;
-            }
-            finally {
-                hdr.onReadEnd();
             }
+
+            return null;
         }
 
         /**
@@ -903,25 +765,18 @@ public class GridCacheConcurrentMap {
          * @return {@code True} if segment contains value.
          */
         boolean containsKey(Object key, int hash) {
-            SegmentHeader hdr = headerForRead();
-
-            try {
-                if (hdr.size() != 0) {
-                    GridCacheMapEntry e = getFirst(hdr.table(), hash);
+            if (size != 0) {
+                HashEntry e = getFirst(tbl, hash);
 
-                    while (e != null) {
-                        if (e.hash() == hash && key.equals(e.key))
-                            return true;
+                while (e != null) {
+                    if (e.mapEntry.hash() == hash && key.equals(e.mapEntry.key()))
+                        return true;
 
-                        e = e.next(hdr.id());
-                    }
+                    e = e.next;
                 }
-
-                return false;
-            }
-            finally {
-                hdr.onReadEnd();
             }
+
+            return false;
         }
 
         /**
@@ -953,58 +808,48 @@ public class GridCacheConcurrentMap {
         @SuppressWarnings({"unchecked", "SynchronizationOnLocalVariableOrMethodParameter"})
         private GridCacheMapEntry put0(KeyCacheObject key, int hash, CacheObject val, AffinityTopologyVersion topVer) {
             try {
-                SegmentHeader hdr = this.hdr;
-
-                int c = hdr.size();
+                int c = size;
 
-                if (c++ > threshold) {// Ensure capacity.
+                if (c++ > threshold) // Ensure capacity.
                     rehash();
 
-                    hdr = this.hdr;
-                }
-
-                int hdrId = hdr.id();
-
-                GridCacheMapEntry[] tab = hdr.table();
+                HashEntry[] tab = tbl;
 
                 int idx = hash & (tab.length - 1);
 
-                GridCacheMapEntry bin = tab[idx];
+                HashEntry bin = tab[idx];
 
-                GridCacheMapEntry e = bin;
+                HashEntry e = bin;
 
-                while (e != null && (e.hash() != hash || !key.equals(e.key)))
-                    e = e.next(hdrId);
+                while (e != null && (e.mapEntry.hash() != hash || !key.equals(e.mapEntry.key())))
+                    e = e.next;
 
                 GridCacheMapEntry retVal;
 
                 if (e != null) {
-                    retVal = e;
+                    retVal = e.mapEntry;
 
-                    e.rawPut(val, 0);
+                    retVal.rawPut(val, 0);
                 }
                 else {
-                    GridCacheMapEntry next = bin != null ? bin : null;
+                    HashEntry next = bin != null ? bin : null;
 
-                    GridCacheMapEntry newRoot = factory.create(ctx, topVer, key, hash, val, next, hdr.id());
+                    GridCacheMapEntry newEntry = factory.create(ctx, topVer, key, hash, val);
 
-                    // Avoiding delete (decrement) before creation (increment).
-                    synchronized (newRoot) {
-                        tab[idx] = newRoot;
+                    tab[idx] = new HashEntry(newEntry, next);
 
-                        retVal = newRoot;
+                    retVal = newEntry;
 
-                        // Modify counters.
-                        if (!retVal.isInternal()) {
-                            mapPubSize.increment();
+                    // Modify counters.
+                    if (!retVal.isInternal()) {
+                        mapPubSize.increment();
 
-                            pubSize.increment();
-                        }
+                        pubSize.increment();
                     }
 
                     mapSize.increment();
 
-                    hdr.size(c);
+                    size = c;
                 }
 
                 return retVal;
@@ -1029,20 +874,16 @@ public class GridCacheConcurrentMap {
             int hash,
             @Nullable CacheObject val,
             AffinityTopologyVersion topVer,
-            boolean create)
-        {
+            boolean create
+        ) {
             lock();
 
             try {
-                SegmentHeader hdr = this.hdr;
-
-                int hdrId = hdr.id();
-
-                GridCacheMapEntry[] tab = hdr.table();
+                HashEntry[] tab = tbl;
 
                 int idx = hash & (tab.length - 1);
 
-                GridCacheMapEntry bin = tab[idx];
+                HashEntry bin = tab[idx];
 
                 GridCacheMapEntry cur = null;
                 GridCacheMapEntry created = null;
@@ -1055,20 +896,20 @@ public class GridCacheConcurrentMap {
                     return new GridTriple<>(cur, created, doomed);
                 }
 
-                GridCacheMapEntry e = bin;
+                HashEntry e = bin;
 
-                while (e != null && (e.hash() != hash || !key.equals(e.key)))
-                    e = e.next(hdrId);
+                while (e != null && (e.mapEntry.hash() != hash || !key.equals(e.mapEntry.key())))
+                    e = e.next;
 
                 if (e != null) {
-                    if (e.obsolete()) {
+                    if (e.mapEntry.obsolete()) {
                         doomed = remove(key, hash, null);
 
                         if (create)
                             cur = created = put0(key, hash, val, topVer);
                     }
                     else
-                        cur = e;
+                        cur = e.mapEntry;
                 }
                 else if (create)
                     cur = created = put0(key, hash, val, topVer);
@@ -1085,14 +926,7 @@ public class GridCacheConcurrentMap {
          */
         @SuppressWarnings("unchecked")
         void rehash() {
-            SegmentHeader oldHdr = hdr;
-
-            if (oldHdr.previous() != null && oldHdr.previous().hasReads())
-                return; // Wait for previous header to free up.
-
-            int oldId = hdr.id();
-
-            GridCacheMapEntry[] oldTbl = oldHdr.table();
+            HashEntry[] oldTbl = tbl;
 
             int oldCap = oldTbl.length;
 
@@ -1112,32 +946,51 @@ public class GridCacheConcurrentMap {
              * reader thread that may be in the midst of traversing table
              * right now.
              */
-            SegmentHeader newHdr = new SegmentHeader(oldCap << 1, oldId + 1, oldHdr);
+            HashEntry[] newTbl = new HashEntry[oldCap << 1];
+
+            threshold = (int)(newTbl.length * loadFactor);
 
-            oldHdr.next(newHdr); // Link.
+            int sizeMask = newTbl.length - 1;
 
-            newHdr.size(oldHdr.size());
+            for (int i = 0; i < oldCap ; i++) {
+                HashEntry e = oldTbl[i];
 
-            GridCacheMapEntry[] newTbl = newHdr.table();
+                if (e != null) {
+                    HashEntry next = e.next;
 
-            threshold = (int)(newTbl.length * loadFactor);
+                    int idx = e.mapEntry.hash() & sizeMask;
 
-            int sizeMask = newTbl.length - 1;
+                    if (next == null)   //  Single node on list
+                        newTbl[idx] = e;
+                    else { // Reuse consecutive sequence at same slot
+                        HashEntry lastRun = e;
 
-            for (GridCacheMapEntry bin1 : oldTbl) {
-                // Relink all nodes.
-                for (GridCacheMapEntry e = bin1; e != null; e = e.next(oldId)) {
-                    int idx = e.hash() & sizeMask;
+                        int lastIdx = idx;
 
-                    GridCacheMapEntry bin2 = newTbl[idx];
+                        for (HashEntry last = next; last != null; last = last.next) {
+                            int k = last.mapEntry.hash() & sizeMask;
 
-                    newTbl[idx] = e;
+                            if (k != lastIdx) {
+                                lastIdx = k;
+                                lastRun = last;
+                            }
+                        }
+
+                        newTbl[lastIdx] = lastRun;
+
+                        // Clone remaining nodes
+                        for (HashEntry p = e; p != lastRun; p = p.next) {
+                            int k = p.mapEntry.hash() & sizeMask;
+
+                            HashEntry n = newTbl[k];
 
-                    e.next(newHdr.id(), bin2);
+                            newTbl[k] = new HashEntry(p.mapEntry, n);
+                        }
+                    }
                 }
             }
 
-            hdr = newHdr;
+            tbl = newTbl;
 
             if (DEBUG)
                 checkSegmentConsistency();
@@ -1157,38 +1010,34 @@ public class GridCacheConcurrentMap {
             lock();
 
             try {
-                SegmentHeader hdr = this.hdr;
-
-                GridCacheMapEntry[] tbl = hdr.table();
-
                 int idx = hash & (tbl.length - 1);
 
-                GridCacheMapEntry bin = tbl[idx];
+                HashEntry bin = tbl[idx];
 
                 if (bin == null)
                     return null;
 
-                GridCacheMapEntry prev = null;
-                GridCacheMapEntry e = bin;
+                HashEntry prev = null;
+                HashEntry e = bin;
 
-                while (e != null && (e.hash() != hash || !key.equals(e.key))) {
+                while (e != null && (e.mapEntry.hash() != hash || !key.equals(e.mapEntry.key))) {
                     prev = e;
 
-                    e = e.next(hdr.id());
+                    e = e.next;
                 }
 
                 if (e != null) {
-                    if (filter != null && !filter.apply(e))
+                    if (filter != null && !filter.apply(e.mapEntry))
                         return null;
 
                     if (prev == null)
-                        tbl[idx] = e.next(hdr.id());
+                        tbl[idx] = e.next;
                     else
-                        prev.next(hdr.id(), e.next(hdr.id()));
+                        prev.next = e.next;
 
                     // Modify counters.
                     synchronized (e) {
-                        if (!e.isInternal() && !e.deleted()) {
+                        if (!e.mapEntry.isInternal() && !e.mapEntry.deleted()) {
                             mapPubSize.decrement();
 
                             pubSize.decrement();
@@ -1197,10 +1046,12 @@ public class GridCacheConcurrentMap {
 
                     mapSize.decrement();
 
-                    hdr.decrementSize();
+                    --size;
+
+                    return e.mapEntry;
                 }
 
-                return e;
+                return null;
             }
             finally {
                 if (DEBUG)
@@ -1214,7 +1065,7 @@ public class GridCacheConcurrentMap {
          * @return Entries count within segment.
          */
         int size() {
-            return hdr.size();
+            return size;
         }
 
         /**
@@ -1242,283 +1093,97 @@ public class GridCacheConcurrentMap {
          * @return Random cache map entry from this segment.
          */
         @Nullable GridCacheMapEntry randomEntry() {
-            SegmentHeader hdr = headerForRead();
+            if (size == 0)
+                return null;
 
-            try {
-                GridCacheMapEntry[] tbl = hdr.table();
+            HashEntry[] tbl = this.tbl;
 
-                Collection<GridCacheMapEntry> entries = new ArrayList<>(3);
+            Collection<GridCacheMapEntry> entries = new ArrayList<>(3);
 
-                int pubCnt = 0;
+            int pubCnt = 0;
 
-                int start = RAND.nextInt(tbl.length);
+            int start = RAND.nextInt(tbl.length);
 
+            outerLoop:
+            {
                 for (int i = start; i < start + tbl.length; i++) {
-                    GridCacheMapEntry first = tbl[i % tbl.length];
+                    HashEntry first = tbl[i & (tbl.length - 1)];
 
                     if (first == null)
                         continue;
 
-                    entries.add(first);
-
-                    for (GridCacheMapEntry e = first; e != null; e = e.next(hdr.id()))
-                        if (!e.isInternal())
+                    for (HashEntry e = first; e != null; e = e.next) {
+                        if (!e.mapEntry.isInternal())
                             pubCnt++;
 
-                    if (entries.size() == 3)
-                        break;
+                        entries.add(e.mapEntry);
+
+                        if (entries.size() == 3)
+                            break outerLoop;
+
+                    }
                 }
+            }
 
-                if (entries.isEmpty())
-                    return null;
+            if (entries.isEmpty())
+                return null;
 
-                if (pubCnt == 0)
-                    return null;
+            if (pubCnt == 0)
+                return null;
 
-                // Desired and current indexes.
-                int idx = RAND.nextInt(pubCnt);
+            // Desired and current indexes.
+            int idx = RAND.nextInt(pubCnt);
 
-                int i = 0;
+            int i = 0;
 
-                GridCacheMapEntry retVal = null;
+            GridCacheMapEntry retVal = null;
 
-                for (GridCacheMapEntry e : entries) {
-                    for (; e != null; e = e.next(hdr.id())) {
-                        if (!(e.key instanceof GridCacheInternal)) {
-                            // In case desired entry was deleted, we return the closest one from left.
-                            retVal = e;
+            for (GridCacheMapEntry e : entries) {
+                if (!(e.key instanceof GridCacheInternal)) {
+                    // In case desired entry was deleted, we return the closest one from left.
+                    retVal = e;
 
-                            if (idx == i++)
-                                break;
-                        }
-                    }
+                    if (idx == i++)
+                        break;
                 }
-
-                return retVal;
-            }
-            finally {
-                hdr.onReadEnd();
             }
+
+            return retVal;
         }
 
         /**
          *
          */
         void checkSegmentConsistency() {
-            SegmentHeader hdr = this.hdr;
-
-            GridCacheMapEntry[] tbl = hdr.table();
+            HashEntry[] tbl = this.tbl;
 
             int cnt = 0;
             int pubCnt = 0;
 
-            for (GridCacheMapEntry b : tbl) {
+            for (HashEntry b : tbl) {
                 if (b != null) {
-                    GridCacheMapEntry e = b;
+                    HashEntry e = b;
 
                     assert e != null;
 
                     while (e != null) {
                         cnt++;
 
-                        if (!(e.key instanceof GridCacheInternal))
+                        if (!(e.mapEntry.key instanceof GridCacheInternal))
                             pubCnt++;
 
-                        e = e.next(hdr.id());
+                        e = e.next;
                     }
                 }
             }
 
-            assert cnt == hdr.size() : "Entry count and header size mismatch [cnt=" + cnt + ", hdrSize=" +
-                hdr.size() + ", segment=" + this + ", hdrId=" + hdr.id() + ']';
+            assert cnt == size : "Entry count and header size mismatch [cnt=" + cnt + ", hdrSize=" +
+                size + ", segment=" + this + ']';
             assert pubCnt == pubSize.intValue();
         }
     }
 
     /**
-     * Segment header.
-     */
-    private static class SegmentHeader {
-        /** Entry table. */
-        private final GridCacheMapEntry[] tbl;
-
-        /** Id for rehash. */
-        private final int id;
-
-        /** Reads. */
-        private final LongAdder8 reads = new LongAdder8();
-
-        /** */
-        private volatile SegmentHeader prev;
-
-        /** */
-        private volatile SegmentHeader next;
-
-        /** The number of elements in this segment's region. */
-        private volatile int size;
-
-        /** Cleaned flag. */
-        private final AtomicBoolean cleaned = new AtomicBoolean();
-
-        /**
-         * Constructs new segment header. New header is created initially and then
-         * every time during rehash operation.
-         *
-         * @param size Size of the table.
-         * @param id ID.
-         * @param prev Previous header.
-         */
-        @SuppressWarnings("unchecked")
-        private SegmentHeader(int size, int id, @Nullable SegmentHeader prev) {
-            tbl = new GridCacheMapEntry[size];
-
-            assert id >= 0;
-
-            this.id = id;
-            this.prev = prev;
-        }
-
-        /**
-         * Increment reads.
-         */
-        void onReadStart() {
-            reads.increment();
-        }
-
-        /**
-         * Decrement reads.
-         */
-        void onReadEnd() {
-            reads.decrement();
-
-            checkClean();
-        }
-
-        /**
-         * Cleans stale links if needed.
-         */
-        void checkClean() {
-            // Check if rehashing didn't occur for the next segment.
-            if (next != null && next.next() == null) {
-                long leftReads = reads.sum();
-
-                assert leftReads >= 0;
-
-                // Clean up.
-                if (leftReads == 0 && cleaned.compareAndSet(false, true)) {
-                    for (GridCacheMapEntry bin : tbl) {
-                        if (bin != null) {
-                            for (GridCacheMapEntry e = bin; e != null; ) {
-                                GridCacheMapEntry next = e.next(id);
-
-                                e.next(id, null); // Unlink.
-
-                                e = next;
-                            }
-                        }
-                    }
-                }
-            }
-        }
-
-        /**
-         * @return {@code True} if has reads.
-         */
-        boolean hasReads() {
-            return reads.sum() > 0;
-        }
-
-        /**
-         * @return Header ID.
-         */
-        int id() {
-            return id;
-        }
-
-        /**
-         * @return Table.
-         */
-        GridCacheMapEntry[] table() {
-            return tbl;
-        }
-
-        /**
-         * @return Table length.
-         */
-        int length() {
-            return tbl.length;
-        }
-
-        /**
-         * @return Next header.
-         */
-        SegmentHeader next() {
-            return next;
-        }
-
-        /**
-         * @param next Next header.
-         */
-        void next(SegmentHeader next) {
-            this.next = next;
-        }
-
-        /**
-         * @return Previous header.
-         */
-        SegmentHeader previous() {
-            return prev;
-        }
-
-        /**
-         * @param prev Previous header.
-         */
-        void previous(SegmentHeader prev) {
-            this.prev = prev;
-        }
-
-        /**
-         * @return New size.
-         */
-        int decrementSize() {
-            return --size;
-        }
-
-        /**
-         * @return Size.
-         */
-        int size() {
-            return size;
-        }
-
-        /**
-         * @param size Size.
-         */
-        void size(int size) {
-            this.size = size;
-        }
-    }
-
-    /**
-     * Phantom segment header to be used in iterators.
-     */
-    private static class WeakIterator<K, V> extends WeakReference<Iterator0<K, V>> {
-        /**
-         * Creates a new phantom reference that refers to the given segment header
-         * and is registered with the given queue.
-         *
-         * @param ref Referred segment header.
-         * @param q Reference queue.
-         */
-        WeakIterator(Iterator0<K, V> ref, ReferenceQueue<Iterator0> q) {
-            super(ref, q);
-
-            assert ref != null;
-            assert q != null;
-        }
-    }
-
-    /**
      * Iterator over {@link GridCacheEntryEx} elements.
      *
      * @param <K> Key type.
@@ -1534,17 +1199,14 @@ public class GridCacheConcurrentMap {
         /** */
         private int nextTblIdx;
 
-        /** Segment header. */
-        private SegmentHeader curSegHdr;
-
         /** */
-        private GridCacheMapEntry[] curTbl;
+        private HashEntry[] curTbl;
 
         /** */
-        private GridCacheMapEntry nextEntry;
+        private HashEntry nextEntry;
 
         /** Next entry to return. */
-        private GridCacheMapEntry next;
+        private HashEntry next;
 
         /** Next value. */
         private V nextVal;
@@ -1556,7 +1218,7 @@ public class GridCacheConcurrentMap {
         private boolean isVal;
 
         /** Current entry. */
-        private GridCacheMapEntry cur;
+        private HashEntry cur;
 
         /** Iterator filter. */
         private CacheEntryPredicate[] filter;
@@ -1567,9 +1229,6 @@ public class GridCacheConcurrentMap {
         /** Cache context. */
         private GridCacheContext<K, V> ctx;
 
-        /** Soft reference. */
-        private final WeakIterator<K, V> weakRef;
-
         /** Mod. */
         private int id;
 
@@ -1580,7 +1239,7 @@ public class GridCacheConcurrentMap {
          * Empty constructor required for {@link Externalizable}.
          */
         public Iterator0() {
-            weakRef = null;
+            // No-op.
         }
 
         /**
@@ -1605,12 +1264,7 @@ public class GridCacheConcurrentMap {
             nextSegIdx = map.segs.length - 1;
             nextTblIdx = -1;
 
-            weakRef = new WeakIterator<>(this, map.iteratorQueue());
-
             advance();
-
-            if (curSegHdr != null)
-               map.addWeakIterator(weakRef, curSegHdr); // Keep pointer to soft reference.
         }
 
         /**
@@ -1622,7 +1276,7 @@ public class GridCacheConcurrentMap {
                 return;
 
             while (nextTblIdx >= 0) {
-                GridCacheMapEntry bucket = curTbl[nextTblIdx--];
+                HashEntry bucket = curTbl[nextTblIdx--];
 
                 if (bucket != null && advanceInBucket(bucket, false))
                     return;
@@ -1633,20 +1287,11 @@ public class GridCacheConcurrentMap {
 
                 GridCacheConcurrentMap.Segment seg = map.segs[nextSegIdx0];
 
-                if (seg.size() != 0 && (id == -1 || nextSegIdx0 % totalCnt == id)) {
-                    if (curSegHdr != null)
-                        curSegHdr.onReadEnd();
-
-                    curSegHdr = seg.headerForRead();
-
-                    assert curSegHdr != null;
-
-                    map.addWeakIterator(weakRef, curSegHdr);
-
-                    curTbl = curSegHdr.table();
+                if (seg.size != 0 && (id == -1 || nextSegIdx0 % totalCnt == id)) {
+                    curTbl = seg.tbl;
 
                     for (int j = curTbl.length - 1; j >= 0; --j) {
-                        GridCacheMapEntry bucket = curTbl[j];
+                        HashEntry bucket = curTbl[j];
 
                         if (bucket != null && advanceInBucket(bucket, false)) {
                             nextTblIdx = j - 1;
@@ -1664,7 +1309,7 @@ public class GridCacheConcurrentMap {
          * @return {@code True} if advance succeeded.
          */
         @SuppressWarnings( {"unchecked"})
-        private boolean advanceInBucket(@Nullable GridCacheMapEntry e, boolean skipFirst) {
+        private boolean advanceInBucket(@Nullable HashEntry e, boolean skipFirst) {
             if (e == null)
                 return false;
 
@@ -1675,11 +1320,11 @@ public class GridCacheConcurrentMap {
                     next = nextEntry;
 
                     // Check if entry is visitable first before doing projection-aware peek.
-                    if (!next.visitable(filter))
+                    if (!next.mapEntry.visitable(filter))
                         continue;
 
                     if (isVal) {
-                        nextVal = next.<K, V>wrap().getValue();
+                        nextVal = next.mapEntry.<K, V>wrap().getValue();
 
                         if (nextVal == null)
                             continue;
@@ -1691,9 +1336,7 @@ public class GridCacheConcurrentMap {
                 // Perform checks in any case.
                 skipFirst = false;
             }
-            while ((nextEntry = nextEntry.next(curSegHdr.id())) != null);
-
-            assert nextEntry == null;
+            while ((nextEntry = nextEntry.next) != null);
 
             next = null;
             nextVal = null;
@@ -1703,17 +1346,7 @@ public class GridCacheConcurrentMap {
 
         /** {@inheritDoc} */
         @Override public boolean hasNext() {
-            boolean hasNext = next != null && (!isVal || nextVal != null);
-
-            if (!hasNext && curSegHdr != null) {
-                curSegHdr.onReadEnd();
-
-                weakRef.clear(); // Do not enqueue.
-
-                map.removeWeakIterator(weakRef); // Remove hard pointer.
-            }
-
-            return hasNext;
+            return next != null && (!isVal || nextVal != null);
         }
 
         /**
@@ -1726,7 +1359,7 @@ public class GridCacheConcurrentMap {
         /** {@inheritDoc} */
         @SuppressWarnings({"unchecked"})
         @Override public GridCacheEntryEx next() {
-            GridCacheMapEntry e = next;
+            HashEntry e = next;
             V v = nextVal;
 
             if (e == null)
@@ -1737,7 +1370,7 @@ public class GridCacheConcurrentMap {
             cur = e;
             curVal = v;
 
-            return cur;
+            return cur.mapEntry;
         }
 
         /** {@inheritDoc} */
@@ -1745,13 +1378,13 @@ public class GridCacheConcurrentMap {
             if (cur == null)
                 throw new IllegalStateException();
 
-            GridCacheMapEntry e = cur;
+            HashEntry e = cur;
 
             cur = null;
             curVal = null;
 
             try {
-                ((IgniteKernal)ctx.grid()).getCache(ctx.name()).getAndRemove(e.key());
+                ((IgniteKernal)ctx.grid()).getCache(ctx.name()).getAndRemove(e.mapEntry.key);
             }
             catch (IgniteCheckedException ex) {
                 throw new IgniteException(ex);
@@ -2383,4 +2016,33 @@ public class GridCacheConcurrentMap {
             set = (Set0<K, V>)in.readObject();
         }
     }
+
+    /**
+     *
+     */
+    private static class HashEntry {
+        /** */
+        private final GridCacheMapEntry mapEntry;
+
+        /** */
+        @GridToStringExclude
+        private volatile HashEntry next;
+
+        /**
+         * @param mapEntry Entry.
+         * @param next Next.
+         */
+        private HashEntry(
+            GridCacheMapEntry mapEntry,
+            HashEntry next
+        ) {
+            this.mapEntry = mapEntry;
+            this.next = next;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(HashEntry.class, this, super.toString());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index aa06a24..ae0b412 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -136,14 +136,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     @GridToStringInclude
     protected GridCacheVersion ver;
 
-    /** Next entry in the linked list. */
-    @GridToStringExclude
-    private volatile GridCacheMapEntry next0;
-
-    /** Next entry in the linked list. */
-    @GridToStringExclude
-    private volatile GridCacheMapEntry next1;
-
     /** Key hash code. */
     @GridToStringInclude
     private final int hash;
@@ -167,16 +159,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
      * @param key Cache key.
      * @param hash Key hash value.
      * @param val Entry value.
-     * @param next Next entry in the linked list.
-     * @param hdrId Header id.
      */
-    protected GridCacheMapEntry(GridCacheContext<?, ?> cctx,
+    protected GridCacheMapEntry(
+        GridCacheContext<?, ?> cctx,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
+        CacheObject val
+    ) {
         if (log == null)
             log = U.logger(cctx.kernalContext(), logRef, GridCacheMapEntry.class);
 
@@ -194,8 +183,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             value(val);
         }
 
-        next(hdrId, next);
-
         ver = cctx.versions().next();
 
         startVer = ver.order();
@@ -3009,29 +2996,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         return hash;
     }
 
-    /**
-     * Gets next entry in bucket linked list within a hash map segment.
-     *
-     * @param segId Segment ID.
-     * @return Next entry.
-     */
-    GridCacheMapEntry next(int segId) {
-        return (segId & 1) == 0 ? next0 : next1;
-    }
-
-    /**
-     * Sets next entry in bucket linked list within a hash map segment.
-     *
-     * @param segId Segment ID.
-     * @param next Next entry.
-     */
-    void next(int segId, @Nullable GridCacheMapEntry next) {
-        if ((segId & 1) == 0)
-            next0 = next;
-        else
-            next1 = next;
-    }
-
     /** {@inheritDoc} */
     @Nullable @Override public CacheObject peek(boolean heap,
         boolean offheap,

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java
index 0f8dae3..4ee9385 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.jetbrains.annotations.Nullable;
 
 /**
  * Factory for cache entries.
@@ -30,15 +29,13 @@ public interface GridCacheMapEntryFactory {
      * @param key Cache key.
      * @param hash Key hash value.
      * @param val Entry value.
-     * @param next Next entry in the linked list.
-     * @param hdrId Header id.
      * @return New cache entry.
      */
-    public GridCacheMapEntry create(GridCacheContext ctx,
+    public GridCacheMapEntry create(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        @Nullable GridCacheMapEntry next,
-        int hdrId);
-}
\ No newline at end of file
+        CacheObject val
+    );
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 f95aa87..2d1b02e 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
@@ -22,7 +22,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -54,17 +53,14 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      * @param key Cache key.
      * @param hash Key hash value.
      * @param val Entry value.
-     * @param next Next entry in the linked list.
-     * @param hdrId Cache map header ID.
      */
-    public GridDistributedCacheEntry(GridCacheContext ctx,
+    public GridDistributedCacheEntry(
+        GridCacheContext ctx,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
-        super(ctx, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, key, hash, val);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 7648f10..ff8d315 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
@@ -178,18 +178,17 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     @Override protected void init() {
         map.setEntryFactory(new GridCacheMapEntryFactory() {
             /** {@inheritDoc} */
-            @Override public GridCacheMapEntry create(GridCacheContext ctx,
+            @Override public GridCacheMapEntry create(
+                GridCacheContext ctx,
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                GridCacheMapEntry next,
-                int hdrId)
-            {
+                CacheObject val
+            ) {
                 if (ctx.useOffheapEntry())
-                    return new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                    return new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash, val);
 
-                return new GridDhtCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                return new GridDhtCacheEntry(ctx, topVer, key, hash, val);
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index b9207db..14e3d3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -30,7 +30,6 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.GridCacheMultiTxFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheMvcc;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
@@ -78,18 +77,15 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
      * @param key Cache key.
      * @param hash Key hash value.
      * @param val Entry value.
-     * @param next Next entry in the linked list.
-     * @param hdrId Header id.
      */
-    public GridDhtCacheEntry(GridCacheContext ctx,
+    public GridDhtCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
-        super(ctx, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, key, hash, val);
 
         // Record this entry with partition.
         locPart = ctx.dht().topology().onAdded(topVer, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
index 8cb3fa4..07272ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 
 /**
@@ -36,17 +35,15 @@ public class GridDhtOffHeapCacheEntry extends GridDhtCacheEntry {
      * @param key    Cache key.
      * @param hash   Key hash value.
      * @param val    Entry value.
-     * @param next   Next entry in the linked list.
-     * @param hdrId  Header id.
      */
-    public GridDhtOffHeapCacheEntry(GridCacheContext ctx,
+    public GridDhtOffHeapCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId) {
-        super(ctx, topVer, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, topVer, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -63,4 +60,4 @@ public class GridDhtOffHeapCacheEntry extends GridDhtCacheEntry {
     @Override protected void offHeapPointer(long valPtr) {
         this.valPtr = valPtr;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
index c6b969d..274701f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
@@ -91,8 +91,8 @@ public class GridNoStorageCacheMap extends GridCacheConcurrentMap {
     {
         if (create) {
             GridCacheMapEntry entry = ctx.useOffheapEntry() ?
-                new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash(key.hashCode()), val, null, 0) :
-                new GridDhtCacheEntry(ctx, topVer, key, hash(key.hashCode()), val, null, 0);
+                new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash(key.hashCode()), val) :
+                new GridDhtCacheEntry(ctx, topVer, key, hash(key.hashCode()), val);
 
             return new GridTriple<>(entry, null, null);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 c5ec258..d5e849e 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
@@ -179,18 +179,17 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override protected void init() {
         map.setEntryFactory(new GridCacheMapEntryFactory() {
             /** {@inheritDoc} */
-            @Override public GridCacheMapEntry create(GridCacheContext ctx,
+            @Override public GridCacheMapEntry create(
+                GridCacheContext ctx,
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                GridCacheMapEntry next,
-                int hdrId)
-            {
+                CacheObject val
+            ) {
                 if (ctx.useOffheapEntry())
-                    return new GridDhtAtomicOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                    return new GridDhtAtomicOffHeapCacheEntry(ctx, topVer, key, hash, val);
 
-                return new GridDhtAtomicCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                return new GridDhtAtomicCacheEntry(ctx, topVer, key, hash, val);
             }
         });
 
@@ -2970,16 +2969,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      *
      */
     private static class FinishedLockFuture extends GridFinishedFuture<Boolean> implements GridDhtFuture<Boolean> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         * Empty constructor required by {@link Externalizable}.
-         */
-        public FinishedLockFuture() {
-            // No-op.
-        }
-
         /**
          * @param err Error.
          */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
index 78870c6..3f014d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -36,18 +35,15 @@ public class GridDhtAtomicCacheEntry extends GridDhtCacheEntry {
      * @param key Cache key.
      * @param hash Key hash value.
      * @param val Entry value.
-     * @param next Next entry in the linked list.
-     * @param hdrId Header id.
      */
-    public GridDhtAtomicCacheEntry(GridCacheContext ctx,
+    public GridDhtAtomicCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
-        super(ctx, topVer, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, topVer, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -59,4 +55,4 @@ public class GridDhtAtomicCacheEntry extends GridDhtCacheEntry {
     @Override public synchronized String toString() {
         return S.toString(GridDhtAtomicCacheEntry.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
index bd3dc10..85cfb80 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 
 /**
@@ -36,17 +35,15 @@ public class GridDhtAtomicOffHeapCacheEntry extends GridDhtAtomicCacheEntry {
      * @param key    Cache key.
      * @param hash   Key hash value.
      * @param val    Entry value.
-     * @param next   Next entry in the linked list.
-     * @param hdrId  Header id.
      */
-    public GridDhtAtomicOffHeapCacheEntry(GridCacheContext ctx,
+    public GridDhtAtomicOffHeapCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId) {
-        super(ctx, topVer, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, topVer, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -63,4 +60,4 @@ public class GridDhtAtomicOffHeapCacheEntry extends GridDhtAtomicCacheEntry {
     @Override protected void offHeapPointer(long valPtr) {
         this.valPtr = valPtr;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 b69b42c..19fefdb 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
@@ -119,18 +119,17 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
     @Override protected void init() {
         map.setEntryFactory(new GridCacheMapEntryFactory() {
             /** {@inheritDoc} */
-            @Override public GridCacheMapEntry create(GridCacheContext ctx,
+            @Override public GridCacheMapEntry create(
+                GridCacheContext ctx,
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                GridCacheMapEntry next,
-                int hdrId)
-            {
+                CacheObject val
+            ) {
                 if (ctx.useOffheapEntry())
-                    return new GridDhtColocatedOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                    return new GridDhtColocatedOffHeapCacheEntry(ctx, topVer, key, hash, val);
 
-                return new GridDhtColocatedCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                return new GridDhtColocatedCacheEntry(ctx, topVer, key, hash, val);
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java
index f5cd961..cc71e11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.colocated;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -35,18 +34,15 @@ public class GridDhtColocatedCacheEntry extends GridDhtCacheEntry {
      * @param key Cache key.
      * @param hash Key hash value.
      * @param val Entry value.
-     * @param next Next entry in the linked list.
-     * @param hdrId Header id.
      */
-    public GridDhtColocatedCacheEntry(GridCacheContext ctx,
+    public GridDhtColocatedCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId
+        CacheObject val
     ) {
-        super(ctx, topVer, key, hash, val, next, hdrId);
+        super(ctx, topVer, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -58,4 +54,4 @@ public class GridDhtColocatedCacheEntry extends GridDhtCacheEntry {
     @Override public synchronized String toString() {
         return S.toString(GridDhtColocatedCacheEntry.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
index ce17474..ea368d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.colocated;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 
 /**
@@ -36,17 +35,15 @@ public class GridDhtColocatedOffHeapCacheEntry extends GridDhtColocatedCacheEntr
      * @param key    Cache key.
      * @param hash   Key hash value.
      * @param val    Entry value.
-     * @param next   Next entry in the linked list.
-     * @param hdrId  Header id.
      */
-    public GridDhtColocatedOffHeapCacheEntry(GridCacheContext ctx,
+    public GridDhtColocatedOffHeapCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId) {
-        super(ctx, topVer, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, topVer, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -63,4 +60,4 @@ public class GridDhtColocatedOffHeapCacheEntry extends GridDhtColocatedCacheEntr
     @Override protected void offHeapPointer(long valPtr) {
         this.valPtr = valPtr;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
index c08f956..c06f68b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
@@ -41,7 +41,7 @@ public class GridDhtDetachedCacheEntry extends GridDistributedCacheEntry {
      */
     public GridDhtDetachedCacheEntry(GridCacheContext ctx, KeyCacheObject key, int hash, CacheObject val,
         GridCacheMapEntry next, int hdrId) {
-        super(ctx, key, hash, val, next, hdrId);
+        super(ctx, key, hash, val);
     }
 
     /**
@@ -97,4 +97,4 @@ public class GridDhtDetachedCacheEntry extends GridDistributedCacheEntry {
         // No-op for detached cache entry.
         return true;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 c92e4e8..27ef996 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
@@ -102,16 +102,14 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                GridCacheMapEntry next,
-                int hdrId
+                CacheObject val
             ) {
                 // Can't hold any locks here - this method is invoked when
                 // holding write-lock on the whole cache map.
                 if (ctx.useOffheapEntry())
-                    return new GridNearOffHeapCacheEntry(ctx, key, hash, val, next, hdrId);
+                    return new GridNearOffHeapCacheEntry(ctx, key, hash, val);
 
-                return new GridNearCacheEntry(ctx, key, hash, val, next, hdrId);
+                return new GridNearCacheEntry(ctx, key, hash, val);
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 afdc5f1..6520f3d 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
@@ -25,7 +25,6 @@ import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.GridCacheMvcc;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
@@ -66,17 +65,14 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
      * @param key Cache key.
      * @param hash Key hash value.
      * @param val Entry value.
-     * @param next Next entry in the linked list.
-     * @param hdrId Header id.
      */
-    public GridNearCacheEntry(GridCacheContext ctx,
+    public GridNearCacheEntry(
+        GridCacheContext ctx,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
-        super(ctx, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, key, hash, val);
 
         part = ctx.affinity().partition(key);
     }
@@ -719,4 +715,4 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
     @Override public synchronized String toString() {
         return S.toString(GridNearCacheEntry.class, this, "super", super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
index c3f3e02..1558f4c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 
 /**
@@ -34,16 +33,14 @@ public class GridNearOffHeapCacheEntry extends GridNearCacheEntry {
      * @param key   Cache key.
      * @param hash  Key hash value.
      * @param val   Entry value.
-     * @param next  Next entry in the linked list.
-     * @param hdrId Header id.
      */
-    public GridNearOffHeapCacheEntry(GridCacheContext ctx,
+    public GridNearOffHeapCacheEntry(
+        GridCacheContext ctx,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId) {
-        super(ctx, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -60,4 +57,4 @@ public class GridNearOffHeapCacheEntry extends GridNearCacheEntry {
     @Override protected void offHeapPointer(long valPtr) {
         this.valPtr = valPtr;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
index 821455a..4ce1f36 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
@@ -88,11 +88,9 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                GridCacheMapEntry next,
-                int hdrId
+                CacheObject val
             ) {
-                return new GridLocalCacheEntry(ctx, key, hash, val, next, hdrId);
+                return new GridLocalCacheEntry(ctx, key, hash, val);
             }
         });
     }
@@ -236,4 +234,4 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
                 log().debug("Explicitly removed future from map of futures: " + fut);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
index 76bfc46..6ddd2e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
@@ -45,17 +45,14 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
      * @param key  Cache key.
      * @param hash Key hash value.
      * @param val Entry value.
-     * @param next Next entry in the linked list.
-     * @param hdrId Header id.
      */
-    public GridLocalCacheEntry(GridCacheContext ctx,
+    public GridLocalCacheEntry(
+        GridCacheContext ctx,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
-        super(ctx, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -440,4 +437,4 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
     @Override public synchronized String toString() {
         return S.toString(GridLocalCacheEntry.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 0afd6bc..dda5ed2 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
@@ -119,11 +119,9 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                @Nullable GridCacheMapEntry next,
-                int hdrId
+                CacheObject val
             ) {
-                return new GridLocalCacheEntry(ctx, key, hash, val, next, hdrId);
+                return new GridLocalCacheEntry(ctx, key, hash, val);
             }
         });
     }
@@ -1647,4 +1645,4 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     @Override public void onDeferredDelete(GridCacheEntryEx entry, GridCacheVersion ver) {
         assert false : "Should not be called";
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 121cd46..e74b3f0 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
@@ -8208,15 +8208,15 @@ public abstract class IgniteUtils {
      * @return Hash value.
      */
     public static int hash(int h) {
-        // Apply base step of MurmurHash; see http://code.google.com/p/smhasher/
-        // Despite two multiplies, this is often faster than others
-        // with comparable bit-spread properties.
-        h ^= h >>> 16;
-        h *= 0x85ebca6b;
-        h ^= h >>> 13;
-        h *= 0xc2b2ae35;
-
-        return (h >>> 16) ^ h;
+        // Spread bits to regularize both segment and index locations,
+        // using variant of single-word Wang/Jenkins hash.
+        h += (h <<  15) ^ 0xffffcd7d;
+        h ^= (h >>> 10);
+        h += (h <<   3);
+        h ^= (h >>>  6);
+        h += (h <<   2) + (h << 14);
+
+        return h ^ (h >>> 16);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAlwaysEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAlwaysEvictionPolicy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAlwaysEvictionPolicy.java
index 92e1bab..e1b2fa3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAlwaysEvictionPolicy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAlwaysEvictionPolicy.java
@@ -45,4 +45,4 @@ public class GridCacheAlwaysEvictionPolicy<K, V> implements EvictionPolicy<K, V>
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         // No-op.
     }
-}
\ No newline at end of file
+}


[38/50] [abbrv] ignite git commit: IGNITE-2026: .NET: Fixed stack overflow caused by incorrect unboxing of value types.

Posted by sb...@apache.org.
IGNITE-2026: .NET: Fixed stack overflow caused by incorrect unboxing of value types.


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

Branch: refs/heads/ignite-1.5.1
Commit: efe632b18e760f699bedee906050f66eabadb077
Parents: 8ca163b
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Tue Dec 8 15:59:23 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Dec 8 15:59:23 2015 +0300

----------------------------------------------------------------------
 .../Binary/BinarySelfTest.cs                    | 67 ++++++++++++++++++++
 .../Impl/Binary/BinaryReflectiveActions.cs      |  4 +-
 .../Impl/Common/DelegateConverter.cs            | 15 +++--
 3 files changed, 78 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/efe632b1/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
index 88328ec..9232665 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
@@ -1028,6 +1028,20 @@ namespace Apache.Ignite.Core.Tests.Binary
             CheckObject(marsh, new OuterObjectType(), new InnerObjectType());
         }
 
+        [Test]
+        public void TestStructsReflective()
+        {
+            var marsh = new Marshaller(new BinaryConfiguration
+            {
+                TypeConfigurations = new[] {new BinaryTypeConfiguration(typeof (ReflectiveStruct))}
+            });
+
+            var obj = new ReflectiveStruct(15, 28.8);
+            var res = marsh.Unmarshal<ReflectiveStruct>(marsh.Marshal(obj));
+
+            Assert.AreEqual(res, obj);
+        }
+
         /**
          * <summary>Test handles.</summary>
          */
@@ -2228,5 +2242,58 @@ namespace Apache.Ignite.Core.Tests.Binary
                 return Foo;
             }
         }
+
+        private struct ReflectiveStruct : IEquatable<ReflectiveStruct>
+        {
+            private readonly int _x;
+            private readonly double _y;
+
+            public ReflectiveStruct(int x, double y)
+            {
+                _x = x;
+                _y = y;
+            }
+
+            public int X
+            {
+                get { return _x; }
+            }
+
+            public double Y
+            {
+                get { return _y; }
+            }
+
+            public bool Equals(ReflectiveStruct other)
+            {
+                return _x == other._x && _y.Equals(other._y);
+            }
+
+            public override bool Equals(object obj)
+            {
+                if (ReferenceEquals(null, obj))
+                    return false;
+
+                return obj is ReflectiveStruct && Equals((ReflectiveStruct) obj);
+            }
+
+            public override int GetHashCode()
+            {
+                unchecked
+                {
+                    return (_x*397) ^ _y.GetHashCode();
+                }
+            }
+
+            public static bool operator ==(ReflectiveStruct left, ReflectiveStruct right)
+            {
+                return left.Equals(right);
+            }
+
+            public static bool operator !=(ReflectiveStruct left, ReflectiveStruct right)
+            {
+                return !left.Equals(right);
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/efe632b1/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveActions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveActions.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveActions.cs
index b229898..15509fc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveActions.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveActions.cs
@@ -398,9 +398,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             // Assign field value
             var targetParam = Expression.Parameter(typeof(object));
-            var targetParamConverted = Expression.Convert(targetParam, field.DeclaringType);
-            var assignExpr = Expression.Call(DelegateConverter.GetWriteFieldMethod(field), targetParamConverted, 
-                readExpr);
+            var assignExpr = Expression.Call(DelegateConverter.GetWriteFieldMethod(field), targetParam, readExpr);
 
             // Compile and return
             return Expression.Lambda<BinaryReflectiveReadAction>(assignExpr, targetParam, readerParam).Compile();

http://git-wip-us.apache.org/repos/asf/ignite/blob/efe632b1/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
index d32d475..5d1a4e2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
@@ -250,20 +250,25 @@ namespace Apache.Ignite.Core.Impl.Common
         {
             Debug.Assert(field != null);
 
-            var module = Assembly.GetExecutingAssembly().GetModules()[0];
+            var declaringType = field.DeclaringType;
 
-            var method = new DynamicMethod(string.Empty, null, new[] { field.DeclaringType, field.FieldType }, module,
-                true);
+            Debug.Assert(declaringType != null);  // static fields are not supported
 
-            var il = method.GetILGenerator();
+            var method = new DynamicMethod(string.Empty, null, new[] { typeof(object), field.FieldType }, 
+                declaringType, true);
 
+            var il = method.GetILGenerator();
+            
             il.Emit(OpCodes.Ldarg_0);
+
+            if (declaringType.IsValueType)
+                il.Emit(OpCodes.Unbox, declaringType);   // modify boxed copy
+
             il.Emit(OpCodes.Ldarg_1);
             il.Emit(OpCodes.Stfld, field);
             il.Emit(OpCodes.Ret);
 
             return method;
         }
-
     }
 }
\ No newline at end of file


[23/50] [abbrv] ignite git commit: Printing "Failed to register marshalled class" warning in the versbose mode only

Posted by sb...@apache.org.
Printing "Failed to register marshalled class" warning in the versbose mode only


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

Branch: refs/heads/ignite-1.5.1
Commit: 8033bc46df9087971e72187ff75557749cee3f7b
Parents: 3d585d5
Author: Denis Magda <dm...@gridgain.com>
Authored: Sun Dec 6 11:38:17 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Sun Dec 6 11:38:17 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/MarshallerContextImpl.java     | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8033bc46/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 5c9b54f..276cdc3 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
@@ -123,10 +123,9 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
         }
         catch (CachePartialUpdateCheckedException | GridCacheTryPutFailedException e) {
             if (++failedCnt > 10) {
-                String msg = "Failed to register marshalled class for more than 10 times in a row " +
-                    "(may affect performance).";
-
-                U.quietAndWarn(log, msg, msg);
+                if (log.isQuiet())
+                    U.quiet(false, "Failed to register marshalled class for more than 10 times in a row " +
+                        "(may affect performance).");
 
                 failedCnt = 0;
             }


[26/50] [abbrv] ignite git commit: IGNITE-2081 Fixed javadoc.

Posted by sb...@apache.org.
IGNITE-2081 Fixed javadoc.


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

Branch: refs/heads/ignite-1.5.1
Commit: 34596c63c883af696daebcd6a24bee2ef55e7e2c
Parents: 11b4262
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Dec 7 13:29:40 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Dec 7 13:29:40 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/yarn/utils/package-info.java  | 22 ++++++++++++++++++++
 1 file changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/34596c63/modules/yarn/src/main/java/org/apache/ignite/yarn/utils/package-info.java
----------------------------------------------------------------------
diff --git a/modules/yarn/src/main/java/org/apache/ignite/yarn/utils/package-info.java b/modules/yarn/src/main/java/org/apache/ignite/yarn/utils/package-info.java
new file mode 100644
index 0000000..a73c390
--- /dev/null
+++ b/modules/yarn/src/main/java/org/apache/ignite/yarn/utils/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Utility and helper classes.
+ */
+package org.apache.ignite.yarn.utils;
\ No newline at end of file