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

[01/30] ignite git commit: ignite-1911 Added special stop method to avoid hangs inside Ignition.start.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1.5.1 c9b7149a8 -> 09b394e17


ignite-1911 Added special stop method to avoid hangs inside Ignition.start.


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

Branch: refs/heads/ignite-1.5.1
Commit: e0bd3395896cd6df433b04c87d94d26b0cbe0469
Parents: 1a3c784
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 10 09:54:39 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 10 09:54:39 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/Ignition.java   |  2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 24 +++++++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 11 ++--
 .../IgniteCacheEntryListenerAbstractTest.java   |  8 ++-
 ...lientDiscoverySpiFailureTimeoutSelfTest.java |  5 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 63 ++++++++++++++++++--
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |  4 --
 7 files changed, 95 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e0bd3395/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 4fdc849..99ee1d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignition.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignition.java
@@ -221,7 +221,7 @@ public class Ignition {
      *      not found).
      */
     public static boolean stop(@Nullable String name, boolean cancel) {
-        return IgnitionEx.stop(name, cancel);
+        return IgnitionEx.stop(name, cancel, false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0bd3395/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 a73fdeb..9b886e8 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
@@ -286,7 +286,7 @@ public class IgnitionEx {
      *      {@code false} otherwise (if it was not started).
      */
     public static boolean stop(boolean cancel) {
-        return stop(null, cancel);
+        return stop(null, cancel, false);
     }
 
     /**
@@ -304,13 +304,20 @@ public class IgnitionEx {
      *      execution. If {@code false}, then jobs currently running will not be
      *      canceled. In either case, grid node will wait for completion of all
      *      jobs running on it before stopping.
+     * @param stopNotStarted If {@code true} and node start did not finish then interrupts starting thread.
      * @return {@code true} if named grid instance was indeed found and stopped,
      *      {@code false} otherwise (the instance with given {@code name} was
      *      not found).
      */
-    public static boolean stop(@Nullable String name, boolean cancel) {
+    public static boolean stop(@Nullable String name, boolean cancel, boolean stopNotStarted) {
         IgniteNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
 
+        if (grid != null && stopNotStarted && grid.startLatch.getCount() != 0) {
+            grid.starterThreadInterrupted = true;
+
+            grid.starterThread.interrupt();
+        }
+
         if (grid != null && grid.state() == STARTED) {
             grid.stop(cancel);
 
@@ -963,7 +970,15 @@ public class IgnitionEx {
         boolean success = false;
 
         try {
-            grid.start(startCtx);
+            try {
+                grid.start(startCtx);
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                if (grid.starterThreadInterrupted)
+                    Thread.interrupted();
+
+                throw e;
+            }
 
             notifyStateChange(name, STARTED);
 
@@ -1413,6 +1428,9 @@ public class IgnitionEx {
         @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
         private Thread starterThread;
 
+        /** */
+        private boolean starterThreadInterrupted;
+
         /**
          * Creates un-started named instance.
          *

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0bd3395/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 865f73f..b80529b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -66,6 +66,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.events.DiscoveryCustomEvent;
 import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.util.GridBoundedLinkedHashSet;
@@ -2165,22 +2166,22 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                     if (ignite != null) {
                         U.error(log, "TcpDiscoverSpi's message worker thread failed abnormally. " +
-                            "Stopping the grid in order to prevent cluster wide instability.", e);
+                            "Stopping the node in order to prevent cluster wide instability.", e);
 
                         new Thread(new Runnable() {
                             @Override public void run() {
                                 try {
-                                    ignite.close();
+                                    IgnitionEx.stop(ignite.name(), true, true);
 
-                                    U.log(log, "Stopped the grid successfully in response to TcpDiscoverySpi's " +
+                                    U.log(log, "Stopped the node successfully in response to TcpDiscoverySpi's " +
                                         "message worker thread abnormal termination.");
                                 }
                                 catch (Throwable e) {
-                                    U.error(log, "Failed to stop the grid in response to TcpDiscoverySpi's " +
+                                    U.error(log, "Failed to stop the node in response to TcpDiscoverySpi's " +
                                         "message worker thread abnormal termination.", e);
                                 }
                             }
-                        }).start();
+                        }, "node-stop-thread").start();
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0bd3395/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
index 8a3d756..27edb0c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
@@ -113,7 +113,13 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
         for (int i = 0; i < gridCount(); i++) {
             GridContinuousProcessor proc = grid(i).context().continuous();
 
-            ConcurrentMap<?, ?> syncMsgFuts = GridTestUtils.getFieldValue(proc, "syncMsgFuts");
+            final ConcurrentMap<?, ?> syncMsgFuts = GridTestUtils.getFieldValue(proc, "syncMsgFuts");
+
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    return syncMsgFuts.size() == 0;
+                }
+            }, 5000);
 
             assertEquals(0, syncMsgFuts.size());
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0bd3395/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java
index 6b20b2a..35aa934 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java
@@ -296,7 +296,6 @@ public class TcpClientDiscoverySpiFailureTimeoutSelfTest extends TcpClientDiscov
         /** {@inheritDoc} */
         @Override protected <T> T readMessage(Socket sock, @Nullable InputStream in, long timeout)
             throws IOException, IgniteCheckedException {
-
             if (readDelay < failureDetectionTimeout()) {
                 try {
                     return super.readMessage(sock, in, timeout);
@@ -313,9 +312,11 @@ public class TcpClientDiscoverySpiFailureTimeoutSelfTest extends TcpClientDiscov
                 if (msg instanceof TcpDiscoveryPingRequest) {
                     try {
                         Thread.sleep(2000);
-                    } catch (InterruptedException e) {
+                    }
+                    catch (InterruptedException e) {
                         // Ignore
                     }
+
                     throw new SocketTimeoutException("Forced timeout");
                 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0bd3395/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
index 42960e7..862e780 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
@@ -1334,7 +1334,7 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
      */
     public void testNodeShutdownOnRingMessageWorkerFailure() throws Exception {
         try {
-            TestMessageWorkerFailureSpi spi0 = new TestMessageWorkerFailureSpi();
+            TestMessageWorkerFailureSpi1 spi0 = new TestMessageWorkerFailureSpi1();
 
             nodeSpi.set(spi0);
 
@@ -1351,9 +1351,9 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
             final UUID failedNodeId = ignite0.cluster().localNode().id();
 
             ignite1.events().localListen(new IgnitePredicate<Event>() {
-                @Override public boolean apply(Event event) {
-                    if (event.type() == EventType.EVT_NODE_FAILED &&
-                        failedNodeId.equals(((DiscoveryEvent)event).eventNode().id()))
+                @Override public boolean apply(Event evt) {
+                    if (evt.type() == EventType.EVT_NODE_FAILED &&
+                        failedNodeId.equals(((DiscoveryEvent)evt).eventNode().id()))
                         disconnected.set(true);
 
                     latch.countDown();
@@ -1382,6 +1382,38 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
             stopAllGrids();
         }
     }
+    /**
+     * @throws Exception If failed
+     */
+    public void testNodeShutdownOnRingMessageWorkerStartNotFinished() throws Exception {
+        try {
+            Ignite ignite0 = startGrid(0);
+
+            TestMessageWorkerFailureSpi2 spi0 = new TestMessageWorkerFailureSpi2();
+
+            nodeSpi.set(spi0);
+
+            try {
+                startGrid(1);
+
+                fail();
+            }
+            catch (Exception e) {
+                log.error("Expected error: " + e, e);
+            }
+
+            Ignite ignite1 = startGrid(1);
+
+            assertEquals(2, ignite1.cluster().nodes().size());
+            assertEquals(4, ignite1.cluster().topologyVersion());
+
+            assertEquals(2, ignite0.cluster().nodes().size());
+            assertEquals(4, ignite0.cluster().topologyVersion());
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
 
 
     /**
@@ -1952,11 +1984,10 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     /**
      *
      */
-    private static class TestMessageWorkerFailureSpi extends TcpDiscoverySpi {
+    private static class TestMessageWorkerFailureSpi1 extends TcpDiscoverySpi {
         /** */
         private volatile boolean stop;
 
-
         /** {@inheritDoc} */
         @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg,
             GridByteArrayOutputStream bout, long timeout) throws IOException, IgniteCheckedException {
@@ -1969,6 +2000,26 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     */
+    private static class TestMessageWorkerFailureSpi2 extends TcpDiscoverySpi {
+        /** */
+        private volatile boolean stop;
+
+        /** {@inheritDoc} */
+        @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg,
+            GridByteArrayOutputStream bout, long timeout) throws IOException, IgniteCheckedException {
+            if (stop)
+                throw new RuntimeException("Failing ring message worker explicitly");
+
+            super.writeToSocket(sock, msg, bout, timeout);
+
+            if (msg instanceof TcpDiscoveryNodeAddedMessage)
+                stop = true;
+        }
+    }
+
+    /**
      * Starts new grid with given index. Method optimize is not invoked.
      *
      * @param idx Index of the grid to start.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0bd3395/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 6f8ca2d..a92451f 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
@@ -76,10 +76,6 @@ 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());


[09/30] ignite git commit: Ignite-1.5 - Renamed incorrectly named method.

Posted by ak...@apache.org.
Ignite-1.5 - Renamed incorrectly named method.


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

Branch: refs/heads/ignite-1.5.1
Commit: b0ff7d716f7e159fdea7b5346ae785fa7d7ef4f6
Parents: f80637f
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Dec 10 12:41:47 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Dec 10 12:41:47 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/configuration/IgniteConfiguration.java  | 2 +-
 .../ignite/internal/portable/GridPortableAffinityKeySelfTest.java  | 2 +-
 .../portable/GridCacheClientNodeBinaryObjectMetadataTest.java      | 2 +-
 .../distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java  | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b0ff7d71/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 7d8f5be..6598dc0 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
@@ -2002,7 +2002,7 @@ public class IgniteConfiguration {
      *
      * @param cacheKeyCfg Cache key configuration.
      */
-    public void setCacheKeyCfg(CacheKeyConfiguration... cacheKeyCfg) {
+    public void setCacheKeyConfiguration(CacheKeyConfiguration... cacheKeyCfg) {
         this.cacheKeyCfg = cacheKeyCfg;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0ff7d71/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
index c243901..c77bd3d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
@@ -73,7 +73,7 @@ public class GridPortableAffinityKeySelfTest extends GridCommonAbstractTest {
         CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(TestObject.class.getName(), "affKey");
         CacheKeyConfiguration keyCfg2 = new CacheKeyConfiguration("TestObject2", "affKey");
 
-        cfg.setCacheKeyCfg(keyCfg, keyCfg2);
+        cfg.setCacheKeyConfiguration(keyCfg, keyCfg2);
 
         cfg.setMarshaller(new BinaryMarshaller());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0ff7d71/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
index f73916d..a108f4c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
@@ -75,7 +75,7 @@ public class GridCacheClientNodeBinaryObjectMetadataTest extends GridCacheAbstra
 
         CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(TestObject1.class.getName(), "val2");
 
-        cfg.setCacheKeyCfg(keyCfg);
+        cfg.setCacheKeyConfiguration(keyCfg);
 
         bCfg.setTypeConfigurations(Arrays.asList(typeCfg));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0ff7d71/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
index a6ff3ff..edd427f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
@@ -39,7 +39,7 @@ public class GridCacheAffinityRoutingPortableSelfTest extends GridCacheAffinityR
 
         CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(AffinityTestKey.class.getName(), "affKey");
 
-        cfg.setCacheKeyCfg(keyCfg);
+        cfg.setCacheKeyConfiguration(keyCfg);
 
         BinaryConfiguration bCfg = new BinaryConfiguration();
 


[20/30] ignite git commit: ignite-1.5 Fixed test.

Posted by ak...@apache.org.
ignite-1.5 Fixed test.


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

Branch: refs/heads/ignite-1.5.1
Commit: 46872d619b14366c40e1905579e7bcd05f7a4321
Parents: d466c7a
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 10 16:45:04 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 10 16:45:04 2015 +0300

----------------------------------------------------------------------
 .../internal/GridEventStorageCheckAllEventsSelfTest.java       | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/46872d61/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageCheckAllEventsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageCheckAllEventsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageCheckAllEventsSelfTest.java
index 00623fc..ad15894 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageCheckAllEventsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageCheckAllEventsSelfTest.java
@@ -167,6 +167,8 @@ public class GridEventStorageCheckAllEventsSelfTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testSuccessTask() throws Exception {
+        generateEvents(null, new GridAllEventsSuccessTestJob()).get();
+
         long tstamp = startTimestamp();
 
         generateEvents(null, new GridAllEventsSuccessTestJob()).get();
@@ -288,7 +290,7 @@ public class GridEventStorageCheckAllEventsSelfTest extends GridCommonAbstractTe
 
         assert U.currentTimeMillis() > tstamp;
 
-        return tstamp;
+        return U.currentTimeMillis();
     }
 
     /**
@@ -475,4 +477,4 @@ public class GridEventStorageCheckAllEventsSelfTest extends GridCommonAbstractTe
             return (Serializable)results;
         }
     }
-}
\ No newline at end of file
+}


[17/30] ignite git commit: ignite-1.5 - removed near cache

Posted by ak...@apache.org.
ignite-1.5 - removed near cache


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

Branch: refs/heads/ignite-1.5.1
Commit: 67f4afa2313af750a933b81b912c9ba504d8ec4e
Parents: 18fef56
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Dec 10 14:01:32 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Dec 10 14:01:32 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/examples/datagrid/CacheTransactionExample.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/67f4afa2/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheTransactionExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheTransactionExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheTransactionExample.java
index 93e51c1..eced5f2 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheTransactionExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheTransactionExample.java
@@ -59,7 +59,7 @@ public class CacheTransactionExample {
 
             cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
 
-            try (IgniteCache<Integer, Account> cache = ignite.getOrCreateCache(cfg, new NearCacheConfiguration<Integer, Account>())) {
+            try (IgniteCache<Integer, Account> cache = ignite.getOrCreateCache(cfg)) {
                 // Initialize.
                 cache.put(1, new Account(1, 100));
                 cache.put(2, new Account(1, 200));
@@ -142,4 +142,4 @@ public class CacheTransactionExample {
             return "Account [id=" + id + ", balance=$" + balance + ']';
         }
     }
-}
\ No newline at end of file
+}


[24/30] ignite git commit: OSGi Karaf features: adjust unit test to install ignite-hibernate.

Posted by ak...@apache.org.
OSGi Karaf features: adjust unit test to install ignite-hibernate.


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

Branch: refs/heads/ignite-1.5.1
Commit: 3f676bc369541c5284ffb9865956296476f0aeb5
Parents: 5c23257
Author: Raul Kripalani <ra...@apache.org>
Authored: Thu Dec 10 16:03:46 2015 +0000
Committer: Raul Kripalani <ra...@apache.org>
Committed: Thu Dec 10 16:03:53 2015 +0000

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/3f676bc3/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 112a607..57fc7db 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
@@ -95,6 +95,6 @@ public class IgniteKarafFeaturesInstallationTest extends AbstractIgniteKarafTest
      * @return Features list.
      */
     @Override protected List<String> featuresToInstall() {
-        return Arrays.asList("ignite-all");
+        return Arrays.asList("ignite-all", "ignite-hibernate");
     }
 }


[03/30] ignite git commit: ignite-1.5 Fixed test.

Posted by ak...@apache.org.
ignite-1.5 Fixed test.


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

Branch: refs/heads/ignite-1.5.1
Commit: 941323243a53ec502b928e5a71ef2289f14245c0
Parents: 6a7913c
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 10 11:02:07 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 10 11:02:07 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/94132324/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 f5b12ea..5dbf75a 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
@@ -693,7 +693,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
         IgniteInternalFuture<Boolean> fut = GridTestUtils.runAsync(new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
                 try {
-                    Ignition.start(getConfiguration(getTestGridName(SRV_CNT)));
+                    Ignition.start(optimize(getConfiguration(getTestGridName(SRV_CNT))));
 
                     fail();
 


[12/30] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5

Posted by ak...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite 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/c12d17e2
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c12d17e2
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c12d17e2

Branch: refs/heads/ignite-1.5.1
Commit: c12d17e2a1eb8a795c890c3df4c03927be97a197
Parents: b0ff7d7 f24749e
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Dec 10 12:45:18 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Dec 10 12:45:18 2015 +0300

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



[10/30] ignite git commit: ignite-1.5 Fixed test.

Posted by ak...@apache.org.
ignite-1.5 Fixed test.


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

Branch: refs/heads/ignite-1.5.1
Commit: f4ad1287044244964e4bf81a19337f71b7aebeb6
Parents: 6d7a6ea
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 10 12:43:47 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 10 12:43:47 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/f4ad1287/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageCheckAllEventsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageCheckAllEventsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageCheckAllEventsSelfTest.java
index 75ec7b7..00623fc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageCheckAllEventsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageCheckAllEventsSelfTest.java
@@ -40,11 +40,13 @@ import org.apache.ignite.events.Event;
 import org.apache.ignite.events.JobEvent;
 import org.apache.ignite.events.TaskEvent;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.resources.TaskSessionResource;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.common.GridCommonTest;
 import org.jetbrains.annotations.Nullable;
@@ -271,13 +273,21 @@ public class GridEventStorageCheckAllEventsSelfTest extends GridCommonAbstractTe
      * to allow pass {@link IgniteUtils#currentTimeMillis()}.
      *
      * @return Call timestamp.
-     * @throws InterruptedException If sleep was interrupted.
+     * @throws Exception If failed.
      */
-    private long startTimestamp() throws InterruptedException {
-        long tstamp = System.currentTimeMillis();
+    private long startTimestamp() throws Exception {
+        final long tstamp = U.currentTimeMillis();
 
         Thread.sleep(20);
 
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return U.currentTimeMillis() > tstamp;
+            }
+        }, 5000);
+
+        assert U.currentTimeMillis() > tstamp;
+
         return tstamp;
     }
 


[16/30] ignite git commit: OSGi Karaf features: ignite-all -= ignite-hibernate to avoid possible licensing issues.

Posted by ak...@apache.org.
OSGi Karaf features: ignite-all -= ignite-hibernate to avoid possible licensing issues.


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

Branch: refs/heads/ignite-1.5.1
Commit: d466c7abc5401dc7744d6d72b328fa61b7f1d05a
Parents: 18fef56
Author: Raul Kripalani <ra...@apache.org>
Authored: Thu Dec 10 10:59:19 2015 +0000
Committer: Raul Kripalani <ra...@apache.org>
Committed: Thu Dec 10 10:59:19 2015 +0000

----------------------------------------------------------------------
 modules/osgi-karaf/src/main/resources/features.xml | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d466c7ab/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
index 4dca39b..983aeed 100644
--- a/modules/osgi-karaf/src/main/resources/features.xml
+++ b/modules/osgi-karaf/src/main/resources/features.xml
@@ -22,10 +22,12 @@
         <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:
+            NOTE #1: 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.]]>
+            You may safely ignore the 'Resource has no uri' exception if you follow this method.
+
+            NOTE #2: This feature *does not* include components that depend on LGPL software (e.g. ignite-hibernate).]]>
         </details>
         <feature>ignite-core</feature>
         <feature>ignite-aop</feature>
@@ -33,7 +35,6 @@
         <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>
@@ -104,7 +105,9 @@
         <details>
             <![CDATA[The Apache Ignite Hibernate module + dependencies. 
             
-            Installing this feature will trigger the installation of the 'hibernate' feature from the Apache Karaf distribution.]]>
+            Installing this feature will trigger the installation of the 'hibernate' feature from the Apache Karaf distribution.
+
+            NOTE: Take into account that certain parts of Hibernate are offered under the LGPL license.]]>
         </details>
         <feature>hibernate</feature>
         <bundle start="true">mvn:org.apache.ignite/ignite-hibernate/${project.version}</bundle>


[14/30] ignite git commit: ignite-1.5 Added more information for warning message.

Posted by ak...@apache.org.
ignite-1.5 Added more information for warning message.


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

Branch: refs/heads/ignite-1.5.1
Commit: 09c9a0bcc9c608628ee9909eed0854c0a4be79aa
Parents: f24749e
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 10 13:23:55 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 10 13:23:55 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java       |  5 ++++-
 .../processors/cache/GridCacheProcessor.java       | 17 +++++++++++++++++
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/09c9a0bc/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 9a391e5..0aa8b1b 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
@@ -192,7 +192,10 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             }
             else {
                 U.warn(log, "Received message without registered handler (will ignore) [msg=" + cacheMsg +
-                    ", nodeId=" + nodeId + ']');
+                    ", nodeId=" + nodeId +
+                    ", locTopVer=" + cctx.exchange().readyAffinityVersion() +
+                    ", msgTopVer=" + cacheMsg.topologyVersion() +
+                    ", cacheDesc=" + cctx.cache().cacheDescriptor(cacheMsg.cacheId()) + ']');
             }
 
             return;

http://git-wip-us.apache.org/repos/asf/ignite/blob/09c9a0bc/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 3bd9466..84b9960 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
@@ -3053,6 +3053,23 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param cacheId Cache ID.
+     * @return Cache descriptor.
+     */
+    @Nullable public DynamicCacheDescriptor cacheDescriptor(int cacheId) {
+        for (DynamicCacheDescriptor cacheDesc : registeredCaches.values()) {
+            CacheConfiguration ccfg = cacheDesc.cacheConfiguration();
+
+            assert ccfg != null : cacheDesc;
+
+            if (CU.cacheId(ccfg.getName()) == cacheId)
+                return cacheDesc;
+        }
+
+        return null;
+    }
+
+    /**
      * @param cacheCfg Cache configuration template.
      * @throws IgniteCheckedException If failed.
      */


[13/30] ignite git commit: IGNITE-2098 - Added test for java Proxy - Fixes #301.

Posted by ak...@apache.org.
IGNITE-2098 - Added test for java Proxy - Fixes #301.

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/d2008c53
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d2008c53
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d2008c53

Branch: refs/heads/ignite-1.5.1
Commit: d2008c53d2d3ab25ba19b018081f88798140a274
Parents: c12d17e
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Dec 10 12:51:16 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Dec 10 12:51:16 2015 +0300

----------------------------------------------------------------------
 .../portable/PortableClassDescriptor.java       |  14 +-
 .../portable/BinaryMarshallerSelfTest.java      | 198 +++++++++++++++++++
 2 files changed, 207 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d2008c53/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index 2dda9eb..9a65da1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@ -42,10 +42,12 @@ import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinarySerializer;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
+import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
 
 import static java.lang.reflect.Modifier.isStatic;
 import static java.lang.reflect.Modifier.isTransient;
@@ -55,6 +57,9 @@ import static java.lang.reflect.Modifier.isTransient;
  */
 public class PortableClassDescriptor {
     /** */
+    public static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** */
     private final PortableContext ctx;
 
     /** */
@@ -228,7 +233,8 @@ public class PortableClassDescriptor {
                 break;
 
             case OBJECT:
-                ctor = constructor(cls);
+                // Must not use constructor to honor transient fields semantics.
+                ctor = null;
                 ArrayList<BinaryFieldAccessor> fields0 = new ArrayList<>();
                 stableFieldsMeta = metaDataEnabled ? new HashMap<String, Integer>() : null;
 
@@ -748,10 +754,8 @@ public class PortableClassDescriptor {
      * @throws BinaryObjectException In case of error.
      */
     private Object newInstance() throws BinaryObjectException {
-        assert ctor != null;
-
         try {
-            return ctor.newInstance();
+            return ctor != null ? ctor.newInstance() : UNSAFE.allocateInstance(cls);
         }
         catch (InstantiationException | InvocationTargetException | IllegalAccessException e) {
             throw new BinaryObjectException("Failed to instantiate instance: " + cls, e);
@@ -799,7 +803,7 @@ public class PortableClassDescriptor {
                     writeObj.getReturnType() == void.class && readObj.getReturnType() == void.class)
                     return true;
             }
-            catch (NoSuchMethodException e) {
+            catch (NoSuchMethodException ignored) {
                 // No-op.
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d2008c53/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 037adf9..9277a77 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
@@ -23,8 +23,10 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.io.Serializable;
 import java.lang.reflect.Field;
+import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.net.InetSocketAddress;
@@ -75,9 +77,11 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import sun.misc.Unsafe;
 
@@ -2329,6 +2333,200 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testProxy() throws Exception {
+        BinaryMarshaller marsh = binaryMarshaller();
+
+        SomeItf inItf = (SomeItf)Proxy.newProxyInstance(
+            BinaryMarshallerSelfTest.class.getClassLoader(), new Class[] {SomeItf.class},
+            new InvocationHandler() {
+                private NonSerializable obj = new NonSerializable(null);
+
+                @Override public Object invoke(Object proxy, Method mtd, Object[] args) throws Throwable {
+                    if ("hashCode".equals(mtd.getName()))
+                        return obj.hashCode();
+
+                    obj.checkAfterUnmarshalled();
+
+                    return 17;
+                }
+            }
+        );
+
+        SomeItf outItf = marsh.unmarshal(marsh.marshal(inItf), null);
+
+        assertEquals(outItf.checkAfterUnmarshalled(), 17);
+    }
+
+    /**
+     *
+     */
+    private static interface SomeItf {
+        /**
+         * @return Check result.
+         */
+        int checkAfterUnmarshalled();
+    }
+
+    /**
+     * Some non-serializable class.
+     */
+    @SuppressWarnings( {"PublicField","TransientFieldInNonSerializableClass","FieldMayBeStatic"})
+    private static class NonSerializableA {
+        /** */
+        private final long longVal = 0x33445566778899AAL;
+
+        /** */
+        protected Short shortVal = (short)0xAABB;
+
+        /** */
+        public String[] strArr = {"AA","BB"};
+
+        /** */
+        public boolean flag1 = true;
+
+        /** */
+        public boolean flag2;
+
+        /** */
+        public Boolean flag3;
+
+        /** */
+        public Boolean flag4 = true;
+
+        /** */
+        public Boolean flag5 = false;
+
+        /** */
+        private transient int intVal = 0xAABBCCDD;
+
+        /**
+         * @param strArr Array.
+         * @param shortVal Short value.
+         */
+        @SuppressWarnings( {"UnusedDeclaration"})
+        private NonSerializableA(@Nullable String[] strArr, @Nullable Short shortVal) {
+            // No-op.
+        }
+
+        /**
+         * Checks correctness of the state after unmarshalling.
+         */
+        void checkAfterUnmarshalled() {
+            assertEquals(longVal, 0x33445566778899AAL);
+
+            assertEquals(shortVal.shortValue(), (short)0xAABB);
+
+            assertTrue(Arrays.equals(strArr, new String[] {"AA","BB"}));
+
+            assertEquals(0, intVal);
+
+            assertTrue(flag1);
+            assertFalse(flag2);
+            assertNull(flag3);
+            assertTrue(flag4);
+            assertFalse(flag5);
+        }
+    }
+
+    /**
+     * Some non-serializable class.
+     */
+    @SuppressWarnings( {"PublicField","TransientFieldInNonSerializableClass","PackageVisibleInnerClass"})
+    static class NonSerializableB extends NonSerializableA {
+        /** */
+        public Short shortValue = 0x1122;
+
+        /** */
+        public long longValue = 0x8877665544332211L;
+
+        /** */
+        private transient NonSerializableA[] aArr = {
+            new NonSerializableA(null, null),
+            new NonSerializableA(null, null),
+            new NonSerializableA(null, null)
+        };
+
+        /** */
+        protected Double doubleVal = 123.456;
+
+        /**
+         * Just to eliminate the default constructor.
+         */
+        private NonSerializableB() {
+            super(null, null);
+        }
+
+        /**
+         * Checks correctness of the state after unmarshalling.
+         */
+        @Override void checkAfterUnmarshalled() {
+            super.checkAfterUnmarshalled();
+
+            assertEquals(shortValue.shortValue(), 0x1122);
+
+            assertEquals(longValue, 0x8877665544332211L);
+
+            assertNull(aArr);
+
+            assertEquals(doubleVal, 123.456);
+        }
+    }
+
+    /**
+     * Some non-serializable class.
+     */
+    @SuppressWarnings( {"TransientFieldInNonSerializableClass","PublicField"})
+    private static class NonSerializable extends NonSerializableB {
+        /** */
+        private int idVal = -17;
+
+        /** */
+        private final NonSerializableA aVal = new NonSerializableB();
+
+        /** */
+        private transient NonSerializableB bVal = new NonSerializableB();
+
+        /** */
+        private NonSerializableA[] bArr = new NonSerializableA[] {
+            new NonSerializableB(),
+            new NonSerializableA(null, null)
+        };
+
+        /** */
+        public float floatVal = 567.89F;
+
+        /**
+         * Just to eliminate the default constructor.
+         *
+         * @param aVal Unused.
+         */
+        @SuppressWarnings( {"UnusedDeclaration"})
+        private NonSerializable(NonSerializableA aVal) {
+        }
+
+        /**
+         * Checks correctness of the state after unmarshalling.
+         */
+        @Override void checkAfterUnmarshalled() {
+            super.checkAfterUnmarshalled();
+
+            assertEquals(idVal, -17);
+
+            aVal.checkAfterUnmarshalled();
+
+            assertNull(bVal);
+
+            for (NonSerializableA a : bArr) {
+                a.checkAfterUnmarshalled();
+            }
+
+            assertEquals(floatVal, 567.89F);
+        }
+    }
+
+    /**
      * Object with class fields.
      */
     private static class ObjectWithClassFields {


[02/30] ignite git commit: ignite-1.5 Added more debug info in the test.

Posted by ak...@apache.org.
ignite-1.5 Added more debug info in the test.


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

Branch: refs/heads/ignite-1.5.1
Commit: 6a7913c7614487993dd9ba97a2216952654726d8
Parents: e0bd339
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 10 10:22:55 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 10 10:22:55 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/IgniteClientReconnectAbstractTest.java   | 1 +
 .../ignite/internal/IgniteClientReconnectCacheTest.java      | 8 +++++++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6a7913c7/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 180047a..0c005e9 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
@@ -105,6 +105,7 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
     }
 
     /**
+     * @param log Logger.
      * @param latch Latch.
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6a7913c7/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 05da0b2..f5b12ea 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
@@ -724,7 +724,13 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
         TestTcpDiscoverySpi srvSpi = spi(srv);
 
         try {
-            assertTrue(joinLatch.await(5000, MILLISECONDS));
+            if (!joinLatch.await(10_000, MILLISECONDS)) {
+                log.error("Failed to wait for join event, will dump threads.");
+
+                U.dumpThreads(log);
+
+                fail("Failed to wait for join event.");
+            }
 
             U.sleep(1000);
 


[06/30] ignite git commit: ignite-2023 Fixed test to do not use hard-coded keys

Posted by ak...@apache.org.
ignite-2023 Fixed test to do not use hard-coded keys


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

Branch: refs/heads/ignite-1.5.1
Commit: 6d7a6eaaf6ab8be9aa10043841817a3944193b41
Parents: 6cdd580
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 10 12:17:50 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 10 12:17:50 2015 +0300

----------------------------------------------------------------------
 ...niteCacheClientNodeChangingTopologyTest.java | 125 +++++++++++++++++--
 1 file changed, 115 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6d7a6eaa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index 1e3382d..e7657a6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.distributed;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
@@ -43,16 +44,20 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.affinity.GridAffinityFunctionContextImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
@@ -68,12 +73,14 @@ import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 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;
@@ -116,6 +123,8 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        cfg.setConsistentId(gridName);
+
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
 
         cfg.setClientMode(client);
@@ -681,6 +690,93 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
     }
 
     /**
+     * Tries to find keys for two partitions: for one partition assignment should not change after node join,
+     * for another primary node should change.
+     *
+     * @param ignite Ignite.
+     * @param nodes Current nodes.
+     * @return Found keys.
+     */
+    private IgniteBiTuple<Integer, Integer> findKeys(Ignite ignite, ClusterNode...nodes) {
+        ClusterNode newNode = new TcpDiscoveryNode();
+
+        GridTestUtils.setFieldValue(newNode, "consistentId", getTestGridName(4));
+        GridTestUtils.setFieldValue(newNode, "id", UUID.randomUUID());
+
+        List<ClusterNode> topNodes = new ArrayList<>();
+
+        Collections.addAll(topNodes, nodes);
+
+        topNodes.add(newNode);
+
+        DiscoveryEvent discoEvt = new DiscoveryEvent(newNode, "", EventType.EVT_NODE_JOINED, newNode);
+
+        final long topVer = ignite.cluster().topologyVersion();
+
+        GridAffinityFunctionContextImpl ctx = new GridAffinityFunctionContextImpl(topNodes,
+            null,
+            discoEvt,
+            new AffinityTopologyVersion(topVer + 1),
+            1);
+
+        AffinityFunction affFunc = ignite.cache(null).getConfiguration(CacheConfiguration.class).getAffinity();
+
+        List<List<ClusterNode>> newAff = affFunc.assignPartitions(ctx);
+
+        List<List<ClusterNode>> curAff = ((IgniteKernal)ignite).context().cache().internalCache(null).context().
+            affinity().assignments(new AffinityTopologyVersion(topVer));
+
+        Integer key1 = null;
+        Integer key2 = null;
+
+        Affinity<Integer> aff = ignite.affinity(null);
+
+        for (int i = 0; i < curAff.size(); i++) {
+            if (key1 == null) {
+                List<ClusterNode> oldNodes = curAff.get(i);
+                List<ClusterNode> newNodes = newAff.get(i);
+
+                if (oldNodes.equals(newNodes))
+                    key1 = findKey(aff, i);
+            }
+
+            if (key2 == null) {
+                ClusterNode oldPrimary = F.first(curAff.get(i));
+                ClusterNode newPrimary = F.first(newAff.get(i));
+
+                if (!oldPrimary.equals(newPrimary))
+                    key2 = findKey(aff, i);
+            }
+
+            if (key1 != null && key2 != null)
+                break;
+        }
+
+        if (key1 == null || key2 == null)
+            fail("Failed to find nodes required for test.");
+
+        return new IgniteBiTuple<>(key1, key2);
+    }
+
+    /**
+     * @param aff Affinity.
+     * @param part Required key partition.
+     * @return Key.
+     */
+    private Integer findKey(Affinity<Integer> aff, int part) {
+        for (int i = 0; i < 10_000; i++) {
+            Integer key = i;
+
+            if (aff.partition(key) == part)
+                return key;
+        }
+
+        fail();
+
+        return null;
+    }
+
+    /**
      * Tests specific scenario when mapping for first locked keys does not change, but changes for second one.
      *
      * @throws Exception If failed.
@@ -710,8 +806,11 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
 
         TestCommunicationSpi spi = (TestCommunicationSpi)ignite3.configuration().getCommunicationSpi();
 
-        final Integer key1 = 0;
-        final Integer key2 = 7;
+        IgniteBiTuple<Integer, Integer> keys =
+            findKeys(ignite0, ignite0.localNode(), ignite1.localNode(), ignite2.localNode());
+
+        final Integer key1 = keys.get1();
+        final Integer key2 = keys.get2();
 
         spi.blockMessages(GridNearLockRequest.class, ignite0.localNode().id());
         spi.blockMessages(GridNearLockRequest.class, ignite1.localNode().id());
@@ -1168,22 +1267,28 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
 
         IgniteCache<Integer, Integer> cache = ignite3.cache(null);
 
+        Affinity<Integer> aff = ignite0.affinity(null);
+
         try (Transaction tx = ignite3.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-            cache.put(1, 1);
-            cache.put(2, 2);
-            cache.put(3, 3);
+            Integer key1 = findKey(aff, 1);
+            Integer key2 = findKey(aff, 2);
+            Integer key3 = findKey(aff, 3);
+
+            cache.put(key1, 1);
+            cache.put(key2, 2);
+            cache.put(key3, 3);
 
             tx.commit();
         }
 
         checkClientLockMessages(spi.recordedMessages(), 3);
 
-        Map<Integer, Integer> map = new HashMap<>();
+        Map<Integer, Integer> map = new LinkedHashMap<>();
 
-        map.put(4, 4);
-        map.put(5, 5);
-        map.put(6, 6);
-        map.put(7, 7);
+        map.put(primaryKey(ignite0.cache(null)), 4);
+        map.put(primaryKey(ignite1.cache(null)), 5);
+        map.put(primaryKey(ignite2.cache(null)), 6);
+        map.put(primaryKeys(ignite0.cache(null), 1, 10_000).get(0), 7);
 
         try (Transaction tx = ignite3.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
             cache.putAll(map);


[05/30] ignite git commit: IGNITE-2099: Fixed serialization of custom collections.

Posted by ak...@apache.org.
IGNITE-2099: Fixed serialization of custom collections.


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

Branch: refs/heads/ignite-1.5.1
Commit: 6cdd58042358049afacc10e169192a009a346846
Parents: be17c9e
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Dec 10 11:41:05 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Dec 10 11:41:05 2015 +0300

----------------------------------------------------------------------
 .../ignite/binary/BinaryCollectionFactory.java  |  33 +++++
 .../apache/ignite/binary/BinaryMapFactory.java  |  33 +++++
 .../apache/ignite/binary/BinaryRawReader.java   |   8 +-
 .../org/apache/ignite/binary/BinaryReader.java  |   9 +-
 .../internal/portable/BinaryFieldAccessor.java  |  19 +--
 .../internal/portable/BinaryReaderExImpl.java   | 107 +++++++-------
 .../internal/portable/BinaryWriteMode.java      |   3 -
 .../internal/portable/BinaryWriterExImpl.java   |  45 ++----
 .../portable/GridPortableMarshaller.java        |  21 ---
 .../portable/PortableClassDescriptor.java       |   7 +-
 .../internal/portable/PortableContext.java      |  71 ++++-----
 .../ignite/internal/portable/PortableUtils.java | 144 +++++--------------
 .../portable/builder/PortableBuilderReader.java |  12 --
 .../builder/PortableBuilderSerializer.java      |  17 +--
 .../portable/builder/PortableLazyMapEntry.java  |  68 ---------
 .../portable/BinaryMarshallerSelfTest.java      | 127 +++++++++++++++-
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 101 ++++++-------
 .../mutabletest/GridPortableTestClasses.java    |  77 +++++++---
 .../src/binary_reader_writer_raw_test.cpp       |  10 +-
 .../core-test/src/binary_reader_writer_test.cpp |  10 +-
 .../core/include/ignite/binary/binary_consts.h  |  29 +---
 .../Impl/Binary/BinaryObjectBuilder.cs          |   8 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |  27 +---
 .../Impl/Binary/BinaryUtils.cs                  |  93 ++----------
 24 files changed, 465 insertions(+), 614 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/binary/BinaryCollectionFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryCollectionFactory.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryCollectionFactory.java
new file mode 100644
index 0000000..0e5bf83
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryCollectionFactory.java
@@ -0,0 +1,33 @@
+/*
+ * 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.binary;
+
+import java.util.Collection;
+
+/**
+ * Collection factory.
+ */
+public interface BinaryCollectionFactory<K> {
+    /**
+     * Create collection.
+     *
+     * @param size Amount of elements in collection.
+     * @return Collection.
+     */
+    public Collection<K> create(int size);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/binary/BinaryMapFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryMapFactory.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryMapFactory.java
new file mode 100644
index 0000000..d514bf9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryMapFactory.java
@@ -0,0 +1,33 @@
+/*
+ * 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.binary;
+
+import java.util.Map;
+
+/**
+ * Map factory.
+ */
+public interface BinaryMapFactory<K, V> {
+    /**
+     * Create collection.
+     *
+     * @param size Amount of elements in collection.
+     * @return Collection.
+     */
+    public Map<K, V> create(int size);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawReader.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawReader.java
index 7ff515a..ce059d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawReader.java
@@ -206,11 +206,11 @@ public interface BinaryRawReader {
     @Nullable public <T> Collection<T> readCollection() throws BinaryObjectException;
 
     /**
-     * @param colCls Collection class.
+     * @param factory Collection factory.
      * @return Collection.
      * @throws BinaryObjectException In case of error.
      */
-    @Nullable public <T> Collection<T> readCollection(Class<? extends Collection<T>> colCls)
+    @Nullable public <T> Collection<T> readCollection(BinaryCollectionFactory<T> factory)
         throws BinaryObjectException;
 
     /**
@@ -220,11 +220,11 @@ public interface BinaryRawReader {
     @Nullable public <K, V> Map<K, V> readMap() throws BinaryObjectException;
 
     /**
-     * @param mapCls Map class.
+     * @param factory Map factory.
      * @return Map.
      * @throws BinaryObjectException In case of error.
      */
-    @Nullable public <K, V> Map<K, V> readMap(Class<? extends Map<K, V>> mapCls) throws BinaryObjectException;
+    @Nullable public <K, V> Map<K, V> readMap(BinaryMapFactory<K, V> factory) throws BinaryObjectException;
 
     /**
      * @return Value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/binary/BinaryReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryReader.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryReader.java
index b8183a2..be7a156 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryReader.java
@@ -242,11 +242,11 @@ public interface BinaryReader {
 
     /**
      * @param fieldName Field name.
-     * @param colCls Collection class.
+     * @param factory Collection factory.
      * @return Collection.
      * @throws BinaryObjectException In case of error.
      */
-    public <T> Collection<T> readCollection(String fieldName, Class<? extends Collection<T>> colCls)
+    public <T> Collection<T> readCollection(String fieldName, BinaryCollectionFactory<T> factory)
         throws BinaryObjectException;
 
     /**
@@ -258,12 +258,11 @@ public interface BinaryReader {
 
     /**
      * @param fieldName Field name.
-     * @param mapCls Map class.
+     * @param factory Map factory.
      * @return Map.
      * @throws BinaryObjectException In case of error.
      */
-    public <K, V> Map<K, V> readMap(String fieldName, Class<? extends Map<K, V>> mapCls)
-        throws BinaryObjectException;
+    public <K, V> Map<K, V> readMap(String fieldName, BinaryMapFactory<K, V> factory) throws BinaryObjectException;
 
     /**
      * @param fieldName Field name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/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 2f6b73c..7701fb5 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,6 +17,11 @@
 
 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;
@@ -24,10 +29,6 @@ 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.
@@ -607,11 +608,6 @@ public abstract class BinaryFieldAccessor {
 
                     break;
 
-                case MAP_ENTRY:
-                    writer.writeMapEntryField((Map.Entry<?, ?>)val);
-
-                    break;
-
                 case PORTABLE_OBJ:
                     writer.writePortableObjectField((BinaryObjectImpl)val);
 
@@ -813,11 +809,6 @@ public abstract class BinaryFieldAccessor {
 
                     break;
 
-                case MAP_ENTRY:
-                    val = reader.readMapEntry(id);
-
-                    break;
-
                 case PORTABLE_OBJ:
                     val = reader.readPortableObject(id);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/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 a0aa2e5..6a8091f 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
@@ -26,8 +26,11 @@ import java.util.Collection;
 import java.util.Date;
 import java.util.Map;
 import java.util.UUID;
+
+import org.apache.ignite.binary.BinaryCollectionFactory;
 import org.apache.ignite.binary.BinaryIdMapper;
 import org.apache.ignite.binary.BinaryInvalidTypeException;
+import org.apache.ignite.binary.BinaryMapFactory;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryRawReader;
@@ -62,7 +65,6 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT_ARR
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG_ARR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENTRY;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR;
@@ -310,24 +312,6 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
     /**
      * @param fieldId Field ID.
-     * @return Value.
-     * @throws BinaryObjectException On case of error.
-     */
-    @Nullable Map.Entry<?, ?> readMapEntry(int fieldId) throws BinaryObjectException {
-        if (findFieldById(fieldId)) {
-            Flag flag = checkFlag(MAP_ENTRY);
-
-            if (flag == Flag.NORMAL)
-                return PortableUtils.doReadMapEntry(in, ctx, ldr, this, true);
-            else if (flag == Flag.HANDLE)
-                return readHandleField();
-        }
-
-        return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
      * @return Portable object.
      * @throws BinaryObjectException In case of error.
      */
@@ -1236,20 +1220,20 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public <T> Collection<T> readCollection(String fieldName,
-        Class<? extends Collection<T>> colCls) throws BinaryObjectException {
-        return findFieldByName(fieldName) ? readCollection0(colCls) : null;
+    @Nullable @Override public <T> Collection<T> readCollection(String fieldName, BinaryCollectionFactory<T> factory)
+        throws BinaryObjectException {
+        return findFieldByName(fieldName) ? readCollection0(factory) : null;
     }
 
     /**
      * @param fieldId Field ID.
-     * @param colCls Collection class.
+     * @param factory Collection factory.
      * @return Value.
      * @throws BinaryObjectException In case of error.
      */
-    @Nullable <T> Collection<T> readCollection(int fieldId, @Nullable Class<? extends Collection> colCls)
+    @Nullable <T> Collection<T> readCollection(int fieldId, @Nullable BinaryCollectionFactory<T> factory)
         throws BinaryObjectException {
-        return findFieldById(fieldId) ? (Collection<T>)readCollection0(colCls) : null;
+        return findFieldById(fieldId) ? (Collection<T>)readCollection0(factory) : null;
     }
 
     /** {@inheritDoc} */
@@ -1258,26 +1242,41 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public <T> Collection<T> readCollection(Class<? extends Collection<T>> colCls)
+    @Nullable @Override public <T> Collection<T> readCollection(BinaryCollectionFactory<T> factory)
         throws BinaryObjectException {
-        return readCollection0(colCls);
+        return readCollection0(factory);
     }
 
     /**
      * Internal read collection routine.
      *
-     * @param cls Collection class.
+     * @param factory Collection factory.
      * @return Value.
      * @throws BinaryObjectException If failed.
      */
-    private Collection readCollection0(@Nullable Class<? extends Collection> cls)
+    private Collection readCollection0(@Nullable BinaryCollectionFactory factory)
         throws BinaryObjectException {
         switch (checkFlag(COL)) {
             case NORMAL:
-                return (Collection)PortableUtils.doReadCollection(in, ctx, ldr, this, true, cls);
+                return (Collection)PortableUtils.doReadCollection(in, ctx, ldr, this, true, factory);
 
-            case HANDLE:
-                return readHandleField();
+            case HANDLE: {
+                int handlePos = PortableUtils.positionForHandle(in) - in.readInt();
+
+                Object obj = getHandle(handlePos);
+
+                if (obj == null) {
+                    int retPos = in.position();
+
+                    streamPosition(handlePos);
+
+                    obj = readCollection0(factory);
+
+                    streamPosition(retPos);
+                }
+
+                return (Collection)obj;
+            }
 
             default:
                 return null;
@@ -1290,19 +1289,19 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public <K, V> Map<K, V> readMap(String fieldName, Class<? extends Map<K, V>> mapCls)
+    @Nullable @Override public <K, V> Map<K, V> readMap(String fieldName, BinaryMapFactory<K, V> factory)
         throws BinaryObjectException {
-        return findFieldByName(fieldName) ? readMap0(mapCls) : null;
+        return findFieldByName(fieldName) ? readMap0(factory) : null;
     }
 
     /**
      * @param fieldId Field ID.
-     * @param mapCls Map class.
+     * @param factory Factory.
      * @return Value.
      * @throws BinaryObjectException In case of error.
      */
-    @Nullable Map<?, ?> readMap(int fieldId, @Nullable Class<? extends Map> mapCls) throws BinaryObjectException {
-        return findFieldById(fieldId) ? readMap0(mapCls) : null;
+    @Nullable Map<?, ?> readMap(int fieldId, @Nullable BinaryMapFactory factory) throws BinaryObjectException {
+        return findFieldById(fieldId) ? readMap0(factory) : null;
     }
 
     /** {@inheritDoc} */
@@ -1311,25 +1310,40 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public <K, V> Map<K, V> readMap(Class<? extends Map<K, V>> mapCls)
+    @Nullable @Override public <K, V> Map<K, V> readMap(BinaryMapFactory<K, V> factory)
         throws BinaryObjectException {
-        return readMap0(mapCls);
+        return readMap0(factory);
     }
 
     /**
      * Internal read map routine.
      *
-     * @param cls Map class.
+     * @param factory Factory.
      * @return Value.
      * @throws BinaryObjectException If failed.
      */
-    private Map readMap0(@Nullable Class<? extends Map> cls) throws BinaryObjectException {
+    private Map readMap0(@Nullable BinaryMapFactory factory) throws BinaryObjectException {
         switch (checkFlag(MAP)) {
             case NORMAL:
-                return (Map)PortableUtils.doReadMap(in, ctx, ldr, this, true, cls);
+                return (Map)PortableUtils.doReadMap(in, ctx, ldr, this, true, factory);
 
-            case HANDLE:
-                return readHandleField();
+            case HANDLE: {
+                int handlePos = PortableUtils.positionForHandle(in) - in.readInt();
+
+                Object obj = getHandle(handlePos);
+
+                if (obj == null) {
+                    int retPos = in.position();
+
+                    streamPosition(handlePos);
+
+                    obj = readMap0(factory);
+
+                    streamPosition(retPos);
+                }
+
+                return (Map)obj;
+            }
 
             default:
                 return null;
@@ -1584,11 +1598,6 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
                 break;
 
-            case MAP_ENTRY:
-                obj = PortableUtils.doReadMapEntry(in, ctx, ldr, this, true);
-
-                break;
-
             case PORTABLE_OBJ:
                 obj = PortableUtils.doReadPortableObject(in, ctx);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriteMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriteMode.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriteMode.java
index bd73ad0..3e0bf69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriteMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriteMode.java
@@ -133,9 +133,6 @@ public enum BinaryWriteMode {
     MAP(GridPortableMarshaller.MAP),
 
     /** */
-    MAP_ENTRY(GridPortableMarshaller.MAP_ENTRY),
-
-    /** */
     PORTABLE_OBJ(GridPortableMarshaller.OBJ),
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/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 6bb493e..4139b8f 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,6 +17,16 @@
 
 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;
@@ -27,15 +37,6 @@ 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;
@@ -62,7 +63,6 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT_ARR
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG_ARR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENTRY;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR;
@@ -761,23 +761,6 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     }
 
     /**
-     * @param e Map entry.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
-     */
-    void doWriteMapEntry(@Nullable Map.Entry<?, ?> e) throws BinaryObjectException {
-        if (e == null)
-            out.writeByte(NULL);
-        else {
-            if (tryWriteAsHandle(e))
-                return;
-
-            out.writeByte(MAP_ENTRY);
-            doWriteObject(e.getKey());
-            doWriteObject(e.getValue());
-        }
-    }
-
-    /**
      * @param val Value.
      */
     void doWriteEnum(@Nullable Enum<?> val) {
@@ -1218,14 +1201,6 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     }
 
     /**
-     * @param e Map entry.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
-     */
-    void writeMapEntryField(@Nullable Map.Entry<?, ?> e) throws BinaryObjectException {
-        doWriteMapEntry(e);
-    }
-
-    /**
      * @param val Value.
      */
     void writeEnumField(@Nullable Enum<?> val) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
index 5244da8..5063a1e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
@@ -119,9 +119,6 @@ public class GridPortableMarshaller {
     public static final byte MAP = 25;
 
     /** */
-    public static final byte MAP_ENTRY = 26;
-
-    /** */
     public static final byte PORTABLE_OBJ = 27;
 
     /** */
@@ -167,30 +164,12 @@ public class GridPortableMarshaller {
     public static final byte LINKED_HASH_SET = 4;
 
     /** */
-    public static final byte TREE_SET = 5;
-
-    /** */
-    public static final byte CONC_SKIP_LIST_SET = 6;
-
-    /** */
-    public static final byte CONC_LINKED_QUEUE = 7;
-
-    /** */
     public static final byte HASH_MAP = 1;
 
     /** */
     public static final byte LINKED_HASH_MAP = 2;
 
     /** */
-    public static final byte TREE_MAP = 3;
-
-    /** */
-    public static final byte CONC_HASH_MAP = 4;
-
-    /** */
-    public static final byte PROPERTIES_MAP = 5;
-
-    /** */
     public static final int OBJECT_TYPE_ID = -1;
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index 50c8bb5..2dda9eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@ -205,7 +205,6 @@ public class PortableClassDescriptor {
             case OBJECT_ARR:
             case COL:
             case MAP:
-            case MAP_ENTRY:
             case PORTABLE_OBJ:
             case ENUM:
             case PORTABLE_ENUM:
@@ -536,11 +535,6 @@ public class PortableClassDescriptor {
 
                 break;
 
-            case MAP_ENTRY:
-                writer.doWriteMapEntry((Map.Entry<?, ?>)obj);
-
-                break;
-
             case ENUM:
                 writer.doWriteEnum((Enum<?>)obj);
 
@@ -794,6 +788,7 @@ public class PortableClassDescriptor {
      *
      * @return {@code true} if to use, {@code false} otherwise.
      */
+    @SuppressWarnings("unchecked")
     private boolean initUseOptimizedMarshallerFlag() {
         for (Class c = cls; c != null && !c.equals(Object.class); c = c.getSuperclass()) {
             try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/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 2c7e4c3..f02867b 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
@@ -17,6 +17,33 @@
 
 package org.apache.ignite.internal.portable;
 
+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.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;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.marshaller.MarshallerContext;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentHashMap8;
+
 import java.io.Externalizable;
 import java.io.File;
 import java.io.IOException;
@@ -41,43 +68,11 @@ import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
-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.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;
-import org.apache.ignite.internal.util.typedef.T2;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.marshaller.MarshallerContext;
-import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentHashMap8;
 
 /**
  * Portable context.
@@ -164,16 +159,9 @@ public class PortableContext implements Externalizable {
         colTypes.put(LinkedList.class, GridPortableMarshaller.LINKED_LIST);
         colTypes.put(HashSet.class, GridPortableMarshaller.HASH_SET);
         colTypes.put(LinkedHashSet.class, GridPortableMarshaller.LINKED_HASH_SET);
-        colTypes.put(TreeSet.class, GridPortableMarshaller.TREE_SET);
-        colTypes.put(ConcurrentSkipListSet.class, GridPortableMarshaller.CONC_SKIP_LIST_SET);
-        colTypes.put(ConcurrentLinkedQueue.class, GridPortableMarshaller.CONC_LINKED_QUEUE);
 
         mapTypes.put(HashMap.class, GridPortableMarshaller.HASH_MAP);
         mapTypes.put(LinkedHashMap.class, GridPortableMarshaller.LINKED_HASH_MAP);
-        mapTypes.put(TreeMap.class, GridPortableMarshaller.TREE_MAP);
-        mapTypes.put(ConcurrentHashMap.class, GridPortableMarshaller.CONC_HASH_MAP);
-        mapTypes.put(ConcurrentHashMap8.class, GridPortableMarshaller.CONC_HASH_MAP);
-        mapTypes.put(Properties.class, GridPortableMarshaller.PROPERTIES_MAP);
 
         // IDs range from [0..200] is used by Java SDK API and GridGain legacy API
 
@@ -210,14 +198,9 @@ public class PortableContext implements Externalizable {
         registerPredefinedType(LinkedList.class, 0);
         registerPredefinedType(HashSet.class, 0);
         registerPredefinedType(LinkedHashSet.class, 0);
-        registerPredefinedType(TreeSet.class, 0);
-        registerPredefinedType(ConcurrentSkipListSet.class, 0);
 
         registerPredefinedType(HashMap.class, 0);
         registerPredefinedType(LinkedHashMap.class, 0);
-        registerPredefinedType(TreeMap.class, 0);
-        registerPredefinedType(ConcurrentHashMap.class, 0);
-        registerPredefinedType(ConcurrentHashMap8.class, 0);
 
         registerPredefinedType(GridMapEntry.class, 60);
         registerPredefinedType(IgniteBiTuple.class, 61);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
index 5d794ca..125bb25 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
@@ -17,15 +17,15 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.util.concurrent.ConcurrentLinkedQueue;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryCollectionFactory;
 import org.apache.ignite.binary.BinaryInvalidTypeException;
+import org.apache.ignite.binary.BinaryMapFactory;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.internal.portable.builder.PortableLazyValue;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
-import org.apache.ignite.internal.util.lang.GridMapEntry;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -35,8 +35,6 @@ import org.jsr166.ConcurrentHashMap8;
 import java.io.ByteArrayInputStream;
 import java.io.Externalizable;
 import java.lang.reflect.Array;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Timestamp;
@@ -50,7 +48,6 @@ import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
@@ -68,9 +65,6 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR_ARR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLASS;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.COL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CONC_HASH_MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CONC_LINKED_QUEUE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CONC_SKIP_LIST_SET;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE_ARR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL;
@@ -92,14 +86,12 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.LINKED_
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG_ARR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENTRY;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJECT_TYPE_ID;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OPTM_MARSH;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.PORTABLE_OBJ;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.PROPERTIES_MAP;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.PROTO_VER;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT_ARR;
@@ -107,8 +99,6 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING_ARR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TREE_MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TREE_SET;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.USER_COL;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.USER_SET;
@@ -271,7 +261,6 @@ public class PortableUtils {
         FIELD_TYPE_NAMES[PORTABLE_OBJ] = "Object";
         FIELD_TYPE_NAMES[COL] = "Collection";
         FIELD_TYPE_NAMES[MAP] = "Map";
-        FIELD_TYPE_NAMES[MAP_ENTRY] = "Entry";
         FIELD_TYPE_NAMES[CLASS] = "Class";
         FIELD_TYPE_NAMES[BYTE_ARR] = "byte[]";
         FIELD_TYPE_NAMES[SHORT_ARR] = "short[]";
@@ -620,15 +609,12 @@ public class PortableUtils {
         if (cls.isArray())
             return cls.getComponentType().isEnum() || cls.getComponentType() == Enum.class ? ENUM_ARR : OBJ_ARR;
 
-        if (Collection.class.isAssignableFrom(cls))
+        if (isSpecialCollection(cls))
             return COL;
 
-        if (Map.class.isAssignableFrom(cls))
+        if (isSpecialMap(cls))
             return MAP;
 
-        if (Map.Entry.class.isAssignableFrom(cls))
-            return MAP_ENTRY;
-
         return OBJ;
     }
 
@@ -1004,19 +990,16 @@ public class PortableUtils {
         else if (cls == Timestamp[].class)
             return BinaryWriteMode.TIMESTAMP_ARR;
         else if (cls.isArray())
-            return cls.getComponentType().isEnum() ?
-                BinaryWriteMode.ENUM_ARR : BinaryWriteMode.OBJECT_ARR;
+            return cls.getComponentType().isEnum() ? BinaryWriteMode.ENUM_ARR : BinaryWriteMode.OBJECT_ARR;
         else if (cls == BinaryObjectImpl.class)
             return BinaryWriteMode.PORTABLE_OBJ;
         else if (Binarylizable.class.isAssignableFrom(cls))
             return BinaryWriteMode.PORTABLE;
         else if (Externalizable.class.isAssignableFrom(cls))
             return BinaryWriteMode.EXTERNALIZABLE;
-        else if (Map.Entry.class.isAssignableFrom(cls))
-            return BinaryWriteMode.MAP_ENTRY;
-        else if (Collection.class.isAssignableFrom(cls))
+        else if (isSpecialCollection(cls))
             return BinaryWriteMode.COL;
-        else if (Map.class.isAssignableFrom(cls))
+        else if (isSpecialMap(cls))
             return BinaryWriteMode.MAP;
         else if (cls.isEnum())
             return BinaryWriteMode.ENUM;
@@ -1027,6 +1010,27 @@ public class PortableUtils {
     }
 
     /**
+     * Check if class represents a collection which must be treated specially.
+     *
+     * @param cls Class.
+     * @return {@code True} if this is a special collection class.
+     */
+    private static boolean isSpecialCollection(Class cls) {
+        return ArrayList.class.equals(cls) || LinkedList.class.equals(cls) ||
+            HashSet.class.equals(cls) || LinkedHashSet.class.equals(cls);
+    }
+
+    /**
+     * Check if class represents a map which must be treated specially.
+     *
+     * @param cls Class.
+     * @return {@code True} if this is a special map class.
+     */
+    private static boolean isSpecialMap(Class cls) {
+        return HashMap.class.equals(cls) || LinkedHashMap.class.equals(cls);
+    }
+
+    /**
      * @return Value.
      */
     public static byte[] doReadByteArray(PortableInputStream in) {
@@ -1688,9 +1692,6 @@ public class PortableUtils {
             case MAP:
                 return doReadMap(in, ctx, ldr, handles, false, null);
 
-            case MAP_ENTRY:
-                return doReadMapEntry(in, ctx, ldr, handles, false);
-
             case PORTABLE_OBJ:
                 return doReadPortableObject(in, ctx);
 
@@ -1738,13 +1739,13 @@ public class PortableUtils {
 
     /**
      * @param deserialize Deep flag.
-     * @param cls Collection class.
+     * @param factory Collection factory.
      * @return Value.
      * @throws BinaryObjectException In case of error.
      */
     @SuppressWarnings("unchecked")
     public static Collection<?> doReadCollection(PortableInputStream in, PortableContext ctx, ClassLoader ldr,
-        BinaryReaderHandlesHolder handles, boolean deserialize, @Nullable Class<? extends Collection> cls)
+        BinaryReaderHandlesHolder handles, boolean deserialize, BinaryCollectionFactory factory)
         throws BinaryObjectException {
         int hPos = positionForHandle(in);
 
@@ -1756,20 +1757,8 @@ public class PortableUtils {
 
         Collection<Object> col;
 
-        if (cls != null) {
-            try {
-                Constructor<? extends Collection> cons = cls.getConstructor();
-
-                col = cons.newInstance();
-            }
-            catch (NoSuchMethodException ignored) {
-                throw new BinaryObjectException("Collection class doesn't have public default constructor: " +
-                    cls.getName());
-            }
-            catch (InvocationTargetException | InstantiationException | IllegalAccessException e) {
-                throw new BinaryObjectException("Failed to instantiate collection: " + cls.getName(), e);
-            }
-        }
+        if (factory != null)
+            col = factory.create(size);
         else {
             switch (colType) {
                 case ARR_LIST:
@@ -1792,21 +1781,6 @@ public class PortableUtils {
 
                     break;
 
-                case TREE_SET:
-                    col = new TreeSet<>();
-
-                    break;
-
-                case CONC_SKIP_LIST_SET:
-                    col = new ConcurrentSkipListSet<>();
-
-                    break;
-
-                case CONC_LINKED_QUEUE:
-                    col = new ConcurrentLinkedQueue<>();
-
-                    break;
-
                 case USER_SET:
                     col = U.newHashSet(size);
 
@@ -1832,13 +1806,13 @@ public class PortableUtils {
 
     /**
      * @param deserialize Deep flag.
-     * @param cls Map class.
+     * @param factory Map factory.
      * @return Value.
      * @throws BinaryObjectException In case of error.
      */
     @SuppressWarnings("unchecked")
     public static Map<?, ?> doReadMap(PortableInputStream in, PortableContext ctx, ClassLoader ldr,
-        BinaryReaderHandlesHolder handles, boolean deserialize, @Nullable Class<? extends Map> cls)
+        BinaryReaderHandlesHolder handles, boolean deserialize, BinaryMapFactory factory)
         throws BinaryObjectException {
         int hPos = positionForHandle(in);
 
@@ -1850,20 +1824,8 @@ public class PortableUtils {
 
         Map<Object, Object> map;
 
-        if (cls != null) {
-            try {
-                Constructor<? extends Map> cons = cls.getConstructor();
-
-                map = cons.newInstance();
-            }
-            catch (NoSuchMethodException ignored) {
-                throw new BinaryObjectException("Map class doesn't have public default constructor: " +
-                    cls.getName());
-            }
-            catch (InvocationTargetException | InstantiationException | IllegalAccessException e) {
-                throw new BinaryObjectException("Failed to instantiate map: " + cls.getName(), e);
-            }
-        }
+        if (factory != null)
+            map = factory.create(size);
         else {
             switch (mapType) {
                 case HASH_MAP:
@@ -1876,26 +1838,11 @@ public class PortableUtils {
 
                     break;
 
-                case TREE_MAP:
-                    map = new TreeMap<>();
-
-                    break;
-
-                case CONC_HASH_MAP:
-                    map = new ConcurrentHashMap<>(size);
-
-                    break;
-
                 case USER_COL:
                     map = U.newHashMap(size);
 
                     break;
 
-                case PROPERTIES_MAP:
-                    map = new Properties();
-
-                    break;
-
                 default:
                     throw new BinaryObjectException("Invalid map type: " + mapType);
             }
@@ -1914,25 +1861,6 @@ public class PortableUtils {
     }
 
     /**
-     * @param deserialize Deserialize flag flag.
-     * @return Value.
-     * @throws BinaryObjectException In case of error.
-     */
-    public static Map.Entry<?, ?> doReadMapEntry(PortableInputStream in, PortableContext ctx, ClassLoader ldr,
-        BinaryReaderHandlesHolder handles, boolean deserialize) throws BinaryObjectException {
-        int hPos = positionForHandle(in);
-
-        Object val1 = deserializeOrUnmarshal(in, ctx, ldr, handles, deserialize);
-        Object val2 = deserializeOrUnmarshal(in, ctx, ldr, handles, deserialize);
-
-        GridMapEntry entry = new GridMapEntry<>(val1, val2);
-
-        handles.setHandle(entry, hPos);
-
-        return entry;
-    }
-
-    /**
      * Deserialize or unmarshal the object.
      *
      * @param deserialize Deserialize.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
index cf27da4..0b7e0b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -344,12 +344,6 @@ public class PortableBuilderReader implements PortablePositionReadable {
                 return;
             }
 
-            case GridPortableMarshaller.MAP_ENTRY:
-                skipValue();
-                skipValue();
-
-                return;
-
             case GridPortableMarshaller.PORTABLE_OBJ:
                 len = readInt() + 4;
 
@@ -448,7 +442,6 @@ public class PortableBuilderReader implements PortablePositionReadable {
             case GridPortableMarshaller.OBJ_ARR:
             case GridPortableMarshaller.COL:
             case GridPortableMarshaller.MAP:
-            case GridPortableMarshaller.MAP_ENTRY:
                 return new LazyCollection(pos);
 
             case GridPortableMarshaller.ENUM: {
@@ -732,8 +725,6 @@ public class PortableBuilderReader implements PortablePositionReadable {
 
                     case GridPortableMarshaller.HASH_SET:
                     case GridPortableMarshaller.LINKED_HASH_SET:
-                    case GridPortableMarshaller.TREE_SET:
-                    case GridPortableMarshaller.CONC_SKIP_LIST_SET:
                         return new PortableLazySet(this, size);
                 }
 
@@ -749,9 +740,6 @@ public class PortableBuilderReader implements PortablePositionReadable {
             case GridPortableMarshaller.ENUM_ARR:
                 return new PortableEnumArrayLazyValue(this);
 
-            case GridPortableMarshaller.MAP_ENTRY:
-                return new PortableLazyMapEntry(this);
-
             case GridPortableMarshaller.PORTABLE_OBJ: {
                 int size = readInt();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
index 52f84c6..d5923ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
@@ -19,14 +19,15 @@ package org.apache.ignite.internal.portable.builder;
 
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.internal.portable.BinaryMetadata;
-import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.BinaryObjectExImpl;
+import org.apache.ignite.internal.portable.BinaryWriterExImpl;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.internal.portable.PortableUtils;
-import org.apache.ignite.internal.portable.BinaryWriterExImpl;
-import org.apache.ignite.internal.util.*;
 
-import java.util.*;
+import java.util.Collection;
+import java.util.IdentityHashMap;
+import java.util.Map;
 
 /**
  *
@@ -118,13 +119,7 @@ class PortableBuilderSerializer {
             writer.writeByte(GridPortableMarshaller.COL);
             writer.writeInt(c.size());
 
-            byte colType;
-
-            if (c instanceof GridConcurrentSkipListSet)
-                colType = GridPortableMarshaller.CONC_SKIP_LIST_SET;
-            else
-                colType = writer.context().collectionType(c.getClass());
-
+            byte colType = writer.context().collectionType(c.getClass());
 
             writer.writeByte(colType);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMapEntry.java
deleted file mode 100644
index 5ebb223..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMapEntry.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable.builder;
-
-import org.apache.ignite.internal.portable.*;
-
-import java.util.Map;
-
-/**
- *
- */
-class PortableLazyMapEntry implements Map.Entry<Object, Object>, PortableBuilderSerializationAware {
-    /** */
-    private final Object key;
-
-    /** */
-    private Object val;
-
-    /**
-     * @param reader GridMutablePortableReader
-     */
-    PortableLazyMapEntry(PortableBuilderReader reader) {
-        key = reader.parseValue();
-        val = reader.parseValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object getKey() {
-        return PortableUtils.unwrapLazy(key);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object getValue() {
-        return PortableUtils.unwrapLazy(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object setValue(Object val) {
-        Object res = getValue();
-
-        this.val = val;
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(BinaryWriterExImpl writer, PortableBuilderSerializer ctx) {
-        writer.writeByte(GridPortableMarshaller.MAP_ENTRY);
-
-        ctx.writeValue(writer, key);
-        ctx.writeValue(writer, val);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/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 d667e07..037adf9 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
@@ -40,6 +40,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Queue;
@@ -50,7 +51,9 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
 import junit.framework.Assert;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryCollectionFactory;
 import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryMapFactory;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryObjectException;
@@ -383,6 +386,50 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test serialization of custom collections.
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testCustomCollections() throws Exception {
+        CustomCollections cc = new CustomCollections();
+
+        cc.list.add(1);
+        cc.customList.add(2);
+
+        CustomCollections copiedCc = marshalUnmarshal(cc);
+
+        assert copiedCc.customList.getClass().equals(CustomArrayList.class);
+
+        assertEquals(cc.list.size(), copiedCc.list.size());
+        assertEquals(cc.customList.size(), copiedCc.customList.size());
+
+        assertEquals(cc.list.get(0), copiedCc.list.get(0));
+        assertEquals(cc.customList.get(0), copiedCc.customList.get(0));
+    }
+
+    /**
+     * Test custom collections with factories.
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testCustomCollectionsWithFactory() throws Exception {
+        CustomCollectionsWithFactory cc = new CustomCollectionsWithFactory();
+
+        cc.list.add(new DummyHolder(1));
+        cc.map.put(new DummyHolder(2), new DummyHolder(3));
+
+        CustomCollectionsWithFactory copiedCc = marshalUnmarshal(cc);
+
+        assertEquals(cc.list.size(), copiedCc.list.size());
+        assertEquals(cc.map.size(), copiedCc.map.size());
+
+        assertEquals(cc.list.get(0), copiedCc.list.get(0));
+        assertEquals(cc.map.get(new DummyHolder(2)), copiedCc.map.get(new DummyHolder(2)));
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testExternalizableHashCode() throws Exception {
@@ -1153,8 +1200,8 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
         ));
 
         Object[] arr = new Object[] {new Value(1), new Value(2), new Value(3)};
-        Collection<Value> col = Arrays.asList(new Value(4), new Value(5), new Value(6));
-        Map<Key, Value> map = F.asMap(new Key(10), new Value(10), new Key(20), new Value(20), new Key(30), new Value(30));
+        Collection<Value> col = new ArrayList<>(Arrays.asList(new Value(4), new Value(5), new Value(6)));
+        Map<Key, Value> map = new HashMap<>(F.asMap(new Key(10), new Value(10), new Key(20), new Value(20), new Key(30), new Value(30)));
 
         CollectionFieldsObject obj = new CollectionFieldsObject(arr, col, map);
 
@@ -3460,6 +3507,82 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Custom array list.
+     */
+    private static class CustomArrayList extends ArrayList {
+        // No-op.
+    }
+
+    /**
+     * Custom hash map.
+     */
+    private static class CustomHashMap extends HashMap {
+        // No-op.
+    }
+
+    /**
+     * Holder for non-stadard collections.
+     */
+    private static class CustomCollections {
+        public List list = new ArrayList();
+        public List customList = new CustomArrayList();
+    }
+
+    @SuppressWarnings("unchecked")
+    private static class CustomCollectionsWithFactory implements Binarylizable {
+        public List list = new CustomArrayList();
+        public Map map = new CustomHashMap();
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.writeCollection("list", list);
+            writer.writeMap("map", map);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            list = (List)reader.readCollection("list", new BinaryCollectionFactory<Object>() {
+                @Override public Collection<Object> create(int size) {
+                    return new CustomArrayList();
+                }
+            });
+
+            map = reader.readMap("map", new BinaryMapFactory<Object, Object>() {
+                @Override public Map<Object, Object> create(int size) {
+                    return new CustomHashMap();
+                }
+            });
+        }
+    }
+
+    /**
+     * Dummy value holder.
+     */
+    private static class DummyHolder {
+        /** Value. */
+        public int val;
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        public DummyHolder(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            return o != null && o instanceof DummyHolder && ((DummyHolder)o).val == val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val;
+        }
+    }
+
+    /**
      */
     private static class CycleLinkObject {
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
index cfeb714..06535d2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
@@ -21,6 +21,24 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
+import org.apache.ignite.internal.portable.builder.PortableBuilderEnum;
+import org.apache.ignite.internal.portable.mutabletest.GridBinaryMarshalerAwareTestClass;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.cache.portable.IgniteBinaryImpl;
+import org.apache.ignite.internal.util.lang.GridMapEntry;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+
 import java.lang.reflect.Field;
 import java.math.BigDecimal;
 import java.sql.Timestamp;
@@ -35,27 +53,11 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteBinary;
-import org.apache.ignite.configuration.BinaryConfiguration;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.portable.builder.PortableBuilderEnum;
-import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
-import org.apache.ignite.internal.portable.mutabletest.GridBinaryMarshalerAwareTestClass;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
-import org.apache.ignite.internal.processors.cache.portable.IgniteBinaryImpl;
-import org.apache.ignite.internal.util.lang.GridMapEntry;
-import org.apache.ignite.binary.BinaryObjectBuilder;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assert;
 
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Address;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.AddressBook;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Addresses;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Companies;
 import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Company;
 import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
 import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectArrayList;
@@ -148,15 +150,6 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
                     break;
                 }
-
-                case "entry":
-                    assertEquals(((Map.Entry)expVal).getKey(), ((Map.Entry)actVal).getKey());
-                    assertEquals(((Map.Entry)expVal).getValue(), ((Map.Entry)actVal).getValue());
-                    break;
-
-                default:
-                    assertTrue(field.getName(), Objects.deepEquals(expVal, actVal));
-                    break;
             }
         }
     }
@@ -923,27 +916,6 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
     /**
      *
      */
-    public void testMapEntryModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = ImmutableMap.of(1, "a").entrySet().iterator().next();
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        Map.Entry<Object, Object> entry = mutableObj.getField("foo");
-
-        assertEquals(1, entry.getKey());
-        assertEquals("a", entry.getValue());
-
-        entry.setValue("b");
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(new GridMapEntry<>(1, "b"), res.foo);
-    }
-
-    /**
-     *
-     */
     public void testMapEntryOverride() {
         TestObjectContainer obj = new TestObjectContainer();
 
@@ -1116,30 +1088,39 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
      *
      */
     public void testChangeMap() {
-        AddressBook addrBook = new AddressBook();
+        Addresses addrs = new Addresses();
+
+        addrs.addCompany(new Company(1, "Google inc", 100,
+            new Address("Saint-Petersburg", "Torzhkovskya", 1, 53), "occupation"));
+
+        addrs.addCompany(new Company(2, "Apple inc", 100,
+            new Address("Saint-Petersburg", "Torzhkovskya", 1, 54), "occupation"));
+
+        addrs.addCompany(new Company(3, "Microsoft", 100,
+            new Address("Saint-Petersburg", "Torzhkovskya", 1, 55), "occupation"));
+
+        addrs.addCompany(new Company(4, "Oracle", 100,
+            new Address("Saint-Petersburg", "Nevskiy", 1, 1), "occupation"));
 
-        addrBook.addCompany(new Company(1, "Google inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 53), "occupation"));
-        addrBook.addCompany(new Company(2, "Apple inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 54), "occupation"));
-        addrBook.addCompany(new Company(3, "Microsoft", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 55), "occupation"));
-        addrBook.addCompany(new Company(4, "Oracle", 100, new Address("Saint-Petersburg", "Nevskiy", 1, 1), "occupation"));
+        BinaryObjectBuilderImpl binaryAddres = wrap(addrs);
 
-        BinaryObjectBuilderImpl mutableObj = wrap(addrBook);
+        Map<String, BinaryObjectBuilderImpl> map = binaryAddres.getField("companyByStreet");
 
-        Map<String, List<BinaryObjectBuilderImpl>> map = mutableObj.getField("companyByStreet");
+        BinaryObjectBuilderImpl binaryCompanies = map.get("Torzhkovskya");
 
-        List<BinaryObjectBuilderImpl> list = map.get("Torzhkovskya");
+        List<BinaryObjectBuilderImpl> binaryCompaniesList = binaryCompanies.getField("companies");
 
-        BinaryObjectBuilderImpl company = list.get(0);
+        BinaryObjectBuilderImpl company = binaryCompaniesList.get(0);
 
         assert "Google inc".equals(company.<String>getField("name"));
 
-        list.remove(0);
+        binaryCompaniesList.remove(0);
 
-        AddressBook res = mutableObj.build().deserialize();
+        Addresses res = binaryAddres.build().deserialize();
 
         assertEquals(Arrays.asList("Nevskiy", "Torzhkovskya"), new ArrayList<>(res.getCompanyByStreet().keySet()));
 
-        List<Company> torzhkovskyaCompanies = res.getCompanyByStreet().get("Torzhkovskya");
+        Companies torzhkovskyaCompanies = res.getCompanyByStreet().get("Torzhkovskya");
 
         assertEquals(2, torzhkovskyaCompanies.size());
         assertEquals("Apple inc", torzhkovskyaCompanies.get(0).name);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java
index 69687ab..b568cb5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java
@@ -31,6 +31,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.UUID;
+
+import org.apache.ignite.binary.BinaryMapFactory;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
 import org.apache.ignite.binary.BinaryObject;
 
@@ -315,7 +321,10 @@ public class GridPortableTestClasses {
     /**
      *
      */
-    public static class Address {
+    public static class Address implements Serializable {
+        /** SUID. */
+        private static final long serialVersionUID = 0L;
+
         /** City. */
         public String city;
 
@@ -354,7 +363,10 @@ public class GridPortableTestClasses {
     /**
      *
      */
-    public static class Company {
+    public static class Company implements Serializable {
+        /** SUID. */
+        private static final long serialVersionUID = 0L;
+
         /** ID. */
         public int id;
 
@@ -396,28 +408,50 @@ public class GridPortableTestClasses {
     }
 
     /**
-     *
+     * Companies.
      */
-    public static class AddressBook {
-        /** */
-        private Map<String, List<Company>> companyByStreet = new TreeMap<>();
+    public static class Companies {
+        /** Companies. */
+        private List<Company> companies = new ArrayList<>();
 
         /**
-         * @param street Street.
+         * @param idx Index.
          * @return Company.
          */
-        public List<Company> findCompany(String street) {
-            return companyByStreet.get(street);
+        public Company get(int idx) {
+            return companies.get(idx);
+        }
+
+        /**
+         * @param company Company.
+         */
+        public void add(Company company) {
+            companies.add(company);
+        }
+
+        /**
+         * @return Size.
+         */
+        public int size() {
+            return companies.size();
         }
+    }
+
+    /**
+     *
+     */
+    public static class Addresses implements Binarylizable {
+        /** */
+        private Map<String, Companies> companyByStreet = new TreeMap<>();
 
         /**
          * @param company Company.
          */
         public void addCompany(Company company) {
-            List<Company> list = companyByStreet.get(company.address.street);
+            Companies list = companyByStreet.get(company.address.street);
 
             if (list == null) {
-                list = new ArrayList<>();
+                list = new Companies();
 
                 companyByStreet.put(company.address.street, list);
             }
@@ -428,16 +462,23 @@ public class GridPortableTestClasses {
         /**
          * @return map
          */
-        public Map<String, List<Company>> getCompanyByStreet() {
+        public Map<String, Companies> getCompanyByStreet() {
             return companyByStreet;
         }
 
-        /**
-         * @param companyByStreet map
-         */
-        public void setCompanyByStreet(Map<String, List<Company>> companyByStreet) {
-            this.companyByStreet = companyByStreet;
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.writeMap("companyByStreet", companyByStreet);
         }
-    }
 
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            companyByStreet = reader.readMap("companyByStreet", new BinaryMapFactory<String, Companies>() {
+                @Override public Map<String, Companies> create(int size) {
+                    return new TreeMap<>();
+                }
+            });
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/platforms/cpp/core-test/src/binary_reader_writer_raw_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/binary_reader_writer_raw_test.cpp b/modules/platforms/cpp/core-test/src/binary_reader_writer_raw_test.cpp
index 8d47c24..4e7e2df 100644
--- a/modules/platforms/cpp/core-test/src/binary_reader_writer_raw_test.cpp
+++ b/modules/platforms/cpp/core-test/src/binary_reader_writer_raw_test.cpp
@@ -1496,7 +1496,7 @@ BOOST_AUTO_TEST_CASE(TestCollectionEmpty)
 
 BOOST_AUTO_TEST_CASE(TestCollectionEmptyTyped)
 {
-    CollectionType typ = IGNITE_COLLECTION_CONCURRENT_SKIP_LIST_SET;
+    CollectionType typ = IGNITE_COLLECTION_LINKED_HASH_SET;
 
     CheckRawCollectionEmpty(&typ);
 }
@@ -1508,7 +1508,7 @@ BOOST_AUTO_TEST_CASE(TestCollection)
 
 BOOST_AUTO_TEST_CASE(TestCollectionTyped)
 {
-    CollectionType typ = IGNITE_COLLECTION_CONCURRENT_SKIP_LIST_SET;
+    CollectionType typ = IGNITE_COLLECTION_LINKED_HASH_SET;
 
     CheckRawCollection(&typ);
 }
@@ -1520,7 +1520,7 @@ BOOST_AUTO_TEST_CASE(TestCollectionIterators)
 
 BOOST_AUTO_TEST_CASE(TestCollectionIteratorsTyped)
 {
-    CollectionType typ = IGNITE_COLLECTION_CONCURRENT_SKIP_LIST_SET;
+    CollectionType typ = IGNITE_COLLECTION_LINKED_HASH_SET;
 
     CheckRawCollectionIterators(&typ);
 }
@@ -1573,7 +1573,7 @@ BOOST_AUTO_TEST_CASE(TestMapEmpty)
 
 BOOST_AUTO_TEST_CASE(TestMapEmptyTyped)
 {
-    MapType typ = IGNITE_MAP_CONCURRENT_HASH_MAP;
+    MapType typ = IGNITE_MAP_LINKED_HASH_MAP;
 
     CheckRawMapEmpty(&typ);
 }
@@ -1585,7 +1585,7 @@ BOOST_AUTO_TEST_CASE(TestMap)
 
 BOOST_AUTO_TEST_CASE(TestMapTyped)
 {
-    MapType typ = IGNITE_MAP_CONCURRENT_HASH_MAP;
+    MapType typ = IGNITE_MAP_LINKED_HASH_MAP;
 
     CheckRawMap(&typ);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp b/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
index b2551ec..71177b8 100644
--- a/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
+++ b/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
@@ -1891,7 +1891,7 @@ BOOST_AUTO_TEST_CASE(TestCollectionEmpty)
 
 BOOST_AUTO_TEST_CASE(TestCollectionEmptyTyped)
 {
-    CollectionType typ = IGNITE_COLLECTION_CONCURRENT_SKIP_LIST_SET;
+    CollectionType typ = IGNITE_COLLECTION_LINKED_HASH_SET;
 
     CheckCollectionEmpty(&typ);
 }
@@ -1903,7 +1903,7 @@ BOOST_AUTO_TEST_CASE(TestCollection)
 
 BOOST_AUTO_TEST_CASE(testCollectionTyped)
 {
-    CollectionType typ = IGNITE_COLLECTION_CONCURRENT_SKIP_LIST_SET;
+    CollectionType typ = IGNITE_COLLECTION_LINKED_HASH_SET;
 
     CheckCollection(&typ);
 }
@@ -1915,7 +1915,7 @@ BOOST_AUTO_TEST_CASE(TestCollectionIterators)
 
 BOOST_AUTO_TEST_CASE(TestCollectionIteratorsTyped)
 {
-    CollectionType typ = IGNITE_COLLECTION_CONCURRENT_SKIP_LIST_SET;
+    CollectionType typ = IGNITE_COLLECTION_LINKED_HASH_SET;
 
     CheckCollectionIterators(&typ);
 }
@@ -1980,7 +1980,7 @@ BOOST_AUTO_TEST_CASE(TestMapEmpty)
 
 BOOST_AUTO_TEST_CASE(TestMapEmptyTyped)
 {
-    MapType typ = IGNITE_MAP_CONCURRENT_HASH_MAP;
+    MapType typ = IGNITE_MAP_LINKED_HASH_MAP;
 
     CheckMapEmpty(&typ);
 }
@@ -1992,7 +1992,7 @@ BOOST_AUTO_TEST_CASE(TestMap)
 
 BOOST_AUTO_TEST_CASE(TestMapTyped)
 {
-    MapType typ = IGNITE_MAP_CONCURRENT_HASH_MAP;
+    MapType typ = IGNITE_MAP_LINKED_HASH_MAP;
 
     CheckMap(&typ);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/platforms/cpp/core/include/ignite/binary/binary_consts.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/binary/binary_consts.h b/modules/platforms/cpp/core/include/ignite/binary/binary_consts.h
index 68ed9f9..7f0fd1e 100644
--- a/modules/platforms/cpp/core/include/ignite/binary/binary_consts.h
+++ b/modules/platforms/cpp/core/include/ignite/binary/binary_consts.h
@@ -52,17 +52,7 @@ namespace ignite
             /**
              * Linked hash set. Maps to LinkedHashSet in Java.
              */
-            IGNITE_COLLECTION_LINKED_HASH_SET = 4,
-
-            /**
-             * Tree set. Maps to TreeSet in Java.
-             */
-            IGNITE_COLLECTION_TREE_SET = 5,
-
-            /**
-             * Concurrent skip list set. Maps to ConcurrentSkipListSet in Java.
-             */
-            IGNITE_COLLECTION_CONCURRENT_SKIP_LIST_SET = 6
+            IGNITE_COLLECTION_LINKED_HASH_SET = 4
         };
 
         /**
@@ -83,22 +73,7 @@ namespace ignite
             /**
              * Linked hash map. Maps to LinkedHashMap in Java.
              */
-            IGNITE_MAP_LINKED_HASH_MAP = 2,
-
-            /**
-             * Tree map. Maps to TreeMap in Java.
-             */
-            IGNITE_MAP_TREE_MAP = 3,
-            
-            /**
-             * Concurrent hash map. Maps to ConcurrentHashMap in Java.
-             */
-            IGNITE_MAP_CONCURRENT_HASH_MAP = 4,
-            
-            /**
-             * Properties map. Maps to Properties in Java.
-             */
-            IGNITE_MAP_PROPERTIES_MAP = 5
+            IGNITE_MAP_LINKED_HASH_MAP = 2
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
index 7ef6259..f41514f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
@@ -972,13 +972,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                     }
 
                     break;
-
-                case BinaryUtils.TypeMapEntry:
-                    Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);
-                    Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);
-
-                    break;
-
+                    
                 case BinaryUtils.TypeBinary:
                     TransferArray(inStream, outStream, 1); // Data array.
                     TransferBytes(inStream, outStream, 4); // Offset in array.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
index 506b56d..7596992 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
@@ -69,7 +69,6 @@ namespace Apache.Ignite.Core.Impl.Binary
             {typeof (Guid?), BinaryUtils.TypeGuid},
             {typeof (ArrayList), BinaryUtils.TypeCollection},
             {typeof (Hashtable), BinaryUtils.TypeDictionary},
-            {typeof (DictionaryEntry), BinaryUtils.TypeMapEntry},
             {typeof (bool[]), BinaryUtils.TypeArrayBool},
             {typeof (byte[]), BinaryUtils.TypeArrayByte},
             {typeof (sbyte[]), BinaryUtils.TypeArrayByte},
@@ -162,11 +161,8 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             // 13. Arbitrary dictionary.
             ReadHandlers[BinaryUtils.TypeDictionary] = new BinarySystemReader(ReadDictionary);
-
-            // 15. Map entry.
-            ReadHandlers[BinaryUtils.TypeMapEntry] = new BinarySystemReader(ReadMapEntry);
             
-            // 16. Enum.
+            // 14. Enum.
             ReadHandlers[BinaryUtils.TypeArrayEnum] = new BinarySystemReader(ReadEnumArray);
         }
 
@@ -218,8 +214,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 return WriteArrayList;
             if (type == typeof(Hashtable))
                 return WriteHashtable;
-            if (type == typeof(DictionaryEntry))
-                return WriteMapEntry;
+
             if (type.IsArray)
             {
                 // We know how to write any array type.
@@ -612,16 +607,6 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /**
-         * <summary>Write map entry.</summary>
-         */
-        private static void WriteMapEntry(BinaryWriter ctx, object obj)
-        {
-            ctx.Stream.WriteByte(BinaryUtils.TypeMapEntry);
-
-            BinaryUtils.WriteMapEntry(ctx, (DictionaryEntry)obj);
-        }
-
-        /**
          * <summary>Write binary object.</summary>
          */
         private static void WriteBinary(BinaryWriter ctx, object obj)
@@ -697,14 +682,6 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /**
-         * <summary>Read map entry.</summary>
-         */
-        private static object ReadMapEntry(BinaryReader ctx, Type type)
-        {
-            return BinaryUtils.ReadMapEntry(ctx);
-        }
-        
-        /**
          * <summary>Read delegate.</summary>
          * <param name="ctx">Read context.</param>
          * <param name="type">Type.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/6cdd5804/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
index 4c54dea..1ae5722 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
@@ -19,7 +19,6 @@ namespace Apache.Ignite.Core.Impl.Binary
 {
     using System;
     using System.Collections;
-    using System.Collections.Concurrent;
     using System.Collections.Generic;
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
@@ -133,10 +132,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
         /** Type: map. */
         public const byte TypeDictionary = 25;
-
-        /** Type: map entry. */
-        public const byte TypeMapEntry = 26;
-
+        
         /** Type: binary object. */
         public const byte TypeBinary = 27;
 
@@ -199,34 +195,13 @@ namespace Apache.Ignite.Core.Impl.Binary
 
         /** Collection: linked list. */
         public const byte CollectionLinkedList = 2;
-
-        /** Collection: hash set. */
-        public const byte CollectionHashSet = 3;
-
-        /** Collection: hash set. */
-        public const byte CollectionLinkedHashSet = 4;
-
-        /** Collection: sorted set. */
-        public const byte CollectionSortedSet = 5;
-
-        /** Collection: concurrent bag. */
-        public const byte CollectionConcurrentBag = 6;
-
+        
         /** Map: custom. */
         public const byte MapCustom = 0;
 
         /** Map: hash map. */
         public const byte MapHashMap = 1;
-
-        /** Map: linked hash map. */
-        public const byte MapLinkedHashMap = 2;
-
-        /** Map: sorted map. */
-        public const byte MapSortedMap = 3;
-
-        /** Map: concurrent hash map. */
-        public const byte MapConcurrentHashMap = 4;
-
+        
         /** Byte "0". */
         public const byte ByteZero = 0;
 
@@ -1096,10 +1071,6 @@ namespace Apache.Ignite.Core.Impl.Binary
                     colType = CollectionArrayList;
                 else if (genType == typeof (LinkedList<>))
                     colType = CollectionLinkedList;
-                else if (genType == typeof (SortedSet<>))
-                    colType = CollectionSortedSet;
-                else if (genType == typeof (ConcurrentBag<>))
-                    colType = CollectionConcurrentBag;
                 else
                     colType = CollectionCustom;
             }
@@ -1147,10 +1118,6 @@ namespace Apache.Ignite.Core.Impl.Binary
             {
                 if (colType == CollectionLinkedList)
                     res = new LinkedList<object>();
-                else if (colType == CollectionSortedSet)
-                    res = new SortedSet<object>();
-                else if (colType == CollectionConcurrentBag)
-                    res = new ConcurrentBag<object>();
                 else
                     res = new ArrayList(len);
             }
@@ -1181,14 +1148,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             {
                 var genType = valType.GetGenericTypeDefinition();
 
-                if (genType == typeof (Dictionary<,>))
-                    dictType = MapHashMap;
-                else if (genType == typeof (SortedDictionary<,>))
-                    dictType = MapSortedMap;
-                else if (genType == typeof (ConcurrentDictionary<,>))
-                    dictType = MapConcurrentHashMap;
-                else
-                    dictType = MapCustom;
+                dictType = genType == typeof (Dictionary<,>) ? MapHashMap : MapCustom;
             }
             else
                 dictType = valType == typeof (Hashtable) ? MapHashMap : MapCustom;
@@ -1221,29 +1181,16 @@ namespace Apache.Ignite.Core.Impl.Binary
          * <param name="factory">Factory delegate.</param>
          * <returns>Dictionary.</returns>
          */
-        public static IDictionary ReadDictionary(BinaryReader ctx,
-            DictionaryFactory factory)
+        public static IDictionary ReadDictionary(BinaryReader ctx, DictionaryFactory factory)
         {
             IBinaryStream stream = ctx.Stream;
 
             int len = stream.ReadInt();
 
-            byte colType = ctx.Stream.ReadByte();
-
-            IDictionary res;
-
-            if (factory == null)
-            {
-                if (colType == MapSortedMap)
-                    res = new SortedDictionary<object, object>();
-                else if (colType == MapConcurrentHashMap)
-                    res = new ConcurrentDictionary<object, object>(Environment.ProcessorCount, len);
-                else
-                    res = new Hashtable(len);
-            }
-            else
-                res = factory.Invoke(len);
+            // Skip dictionary type as we can do nothing with it here.
+            ctx.Stream.ReadByte();
 
+            var res = factory == null ? new Hashtable(len) : factory.Invoke(len);
 
             for (int i = 0; i < len; i++)
             {
@@ -1257,30 +1204,6 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /**
-         * <summary>Write map entry.</summary>
-         * <param name="ctx">Write context.</param>
-         * <param name="val">Value.</param>
-         */
-        public static void WriteMapEntry(BinaryWriter ctx, DictionaryEntry val)
-        {
-            ctx.Write(val.Key);
-            ctx.Write(val.Value);
-        }
-
-        /**
-         * <summary>Read map entry.</summary>
-         * <param name="ctx">Context.</param>
-         * <returns>Map entry.</returns>
-         */
-        public static DictionaryEntry ReadMapEntry(BinaryReader ctx)
-        {
-            object key = ctx.Deserialize<object>();
-            object val = ctx.Deserialize<object>();
-
-            return new DictionaryEntry(key, val);
-        }
-
-        /**
          * <summary>Write binary object.</summary>
          * <param name="stream">Stream.</param>
          * <param name="val">Value.</param>


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

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

Branch: refs/heads/ignite-1.5.1
Commit: f80637fdbfa2027722d5dea7bac1ce7475a92ff7
Parents: 959c4d0 6d7a6ea
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 10 12:34:56 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 10 12:34:56 2015 +0300

----------------------------------------------------------------------
 .../ignite/binary/BinaryCollectionFactory.java  |  33 +++++
 .../apache/ignite/binary/BinaryMapFactory.java  |  33 +++++
 .../apache/ignite/binary/BinaryRawReader.java   |   8 +-
 .../org/apache/ignite/binary/BinaryReader.java  |   9 +-
 .../internal/portable/BinaryFieldAccessor.java  |  19 +--
 .../internal/portable/BinaryReaderExImpl.java   | 107 +++++++-------
 .../internal/portable/BinaryWriteMode.java      |   3 -
 .../internal/portable/BinaryWriterExImpl.java   |  45 ++----
 .../portable/GridPortableMarshaller.java        |  21 ---
 .../portable/PortableClassDescriptor.java       |   7 +-
 .../internal/portable/PortableContext.java      |  71 ++++-----
 .../ignite/internal/portable/PortableUtils.java | 144 +++++--------------
 .../portable/builder/PortableBuilderReader.java |  12 --
 .../builder/PortableBuilderSerializer.java      |  17 +--
 .../portable/builder/PortableLazyMapEntry.java  |  68 ---------
 .../portable/BinaryMarshallerSelfTest.java      | 127 +++++++++++++++-
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 101 ++++++-------
 .../mutabletest/GridPortableTestClasses.java    |  77 +++++++---
 ...niteCacheClientNodeChangingTopologyTest.java | 125 ++++++++++++++--
 .../src/binary_reader_writer_raw_test.cpp       |  10 +-
 .../core-test/src/binary_reader_writer_test.cpp |  10 +-
 .../core/include/ignite/binary/binary_consts.h  |  29 +---
 .../Impl/Binary/BinaryObjectBuilder.cs          |   8 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |  27 +---
 .../Impl/Binary/BinaryUtils.cs                  |  93 ++----------
 25 files changed, 580 insertions(+), 624 deletions(-)
----------------------------------------------------------------------



[19/30] ignite git commit: fixed https://issues.apache.org/jira/browse/IGNITE-2119

Posted by ak...@apache.org.
fixed https://issues.apache.org/jira/browse/IGNITE-2119


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

Branch: refs/heads/ignite-1.5.1
Commit: 2e68a87869bd9d854c3efbbb20a72aef7b44fd2a
Parents: 9012b39
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Dec 10 16:00:35 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Dec 10 16:00:35 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/util/nio/GridNioServer.java   | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2e68a878/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index c0f5cfc..53cec84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -876,24 +876,23 @@ public class GridNioServer<T> {
 
                 return;
             }
-            else if (cnt == 0 && !readBuf.hasRemaining())
-                return;
 
             if (log.isTraceEnabled())
                 log.trace("Bytes received [sockCh=" + sockCh + ", cnt=" + cnt + ']');
 
+            if (cnt == 0)
+                return;
+
             if (metricsLsnr != null)
                 metricsLsnr.onBytesReceived(cnt);
 
             ses.bytesReceived(cnt);
 
-            // Sets limit to current position and
-            // resets position to 0.
             readBuf.flip();
 
-            try {
-                assert readBuf.hasRemaining();
+            assert readBuf.hasRemaining();
 
+            try {
                 filterChain.onMessageReceived(ses, readBuf);
 
                 if (readBuf.hasRemaining())


[25/30] ignite git commit: #Ignite-1.5 - Minor corrections.

Posted by ak...@apache.org.
#Ignite-1.5 - Minor corrections.


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

Branch: refs/heads/ignite-1.5.1
Commit: dfbca79ecc976116557fdf1e58a2c4b2ee6962d7
Parents: 3f676bc
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Dec 10 19:20:34 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Dec 10 19:20:34 2015 +0300

----------------------------------------------------------------------
 .../examples/binary/datagrid/CacheClientBinaryQueryExample.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dfbca79e/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
index c8beaf4..dc468ef 100644
--- a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
@@ -137,7 +137,7 @@ public class CacheClientBinaryQueryExample {
 
         fields.put("name", String.class.getName());
         fields.put("salary", Long.class.getName());
-        fields.put("address.zip", Integer.class.getName());
+        fields.put("addr.zip", Integer.class.getName());
         fields.put("organizationId", Integer.class.getName());
         fields.put("address.street", Integer.class.getName());
 


[15/30] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

Posted by ak...@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/18fef56c
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/18fef56c
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/18fef56c

Branch: refs/heads/ignite-1.5.1
Commit: 18fef56c2036b1d6fba8792522ea79d827e65aac
Parents: 09c9a0b d2008c5
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 10 13:24:30 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 10 13:24:30 2015 +0300

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |   2 +-
 .../portable/PortableClassDescriptor.java       |  14 +-
 .../portable/BinaryMarshallerSelfTest.java      | 198 +++++++++++++++++++
 .../GridPortableAffinityKeySelfTest.java        |   2 +-
 ...CacheClientNodeBinaryObjectMetadataTest.java |   2 +-
 ...ridCacheAffinityRoutingPortableSelfTest.java |   2 +-
 6 files changed, 211 insertions(+), 9 deletions(-)
----------------------------------------------------------------------



[22/30] ignite git commit: fixed https://issues.apache.org/jira/browse/IGNITE-2117

Posted by ak...@apache.org.
fixed https://issues.apache.org/jira/browse/IGNITE-2117


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

Branch: refs/heads/ignite-1.5.1
Commit: dfd7a67ca079efff06522c007f924814f1b57941
Parents: 388c173
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Dec 10 18:46:15 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Dec 10 18:46:15 2015 +0300

----------------------------------------------------------------------
 .../ignite/osgi/classloaders/package-info.java  | 21 ++++++++++++++++++++
 .../org/apache/ignite/osgi/package-info.java    | 21 ++++++++++++++++++++
 2 files changed, 42 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dfd7a67c/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/package-info.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/package-info.java b/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/package-info.java
new file mode 100644
index 0000000..1eec8b4
--- /dev/null
+++ b/modules/osgi/src/main/java/org/apache/ignite/osgi/classloaders/package-info.java
@@ -0,0 +1,21 @@
+/*
+ *  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.
+ */
+
+/**
+ * Contains implementation of OSGi class loaders.
+ */
+package org.apache.ignite.osgi.classloaders;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfd7a67c/modules/osgi/src/main/java/org/apache/ignite/osgi/package-info.java
----------------------------------------------------------------------
diff --git a/modules/osgi/src/main/java/org/apache/ignite/osgi/package-info.java b/modules/osgi/src/main/java/org/apache/ignite/osgi/package-info.java
new file mode 100644
index 0000000..c2f46a2
--- /dev/null
+++ b/modules/osgi/src/main/java/org/apache/ignite/osgi/package-info.java
@@ -0,0 +1,21 @@
+/*
+ *  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.
+ */
+
+/**
+ * Contains OSGi related classes.
+ */
+package org.apache.ignite.osgi;


[26/30] ignite git commit: #Ignite-1.5 - Minor corrections.

Posted by ak...@apache.org.
#Ignite-1.5 - Minor corrections.


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

Branch: refs/heads/ignite-1.5.1
Commit: 5ba664c48d94f60358bb06d5cfc8d62e7ec1b586
Parents: dfbca79
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Dec 10 19:24:22 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Dec 10 19:24:22 2015 +0300

----------------------------------------------------------------------
 .../examples/binary/datagrid/CacheClientBinaryQueryExample.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5ba664c4/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
index dc468ef..7cc95b5 100644
--- a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
@@ -139,7 +139,7 @@ public class CacheClientBinaryQueryExample {
         fields.put("salary", Long.class.getName());
         fields.put("addr.zip", Integer.class.getName());
         fields.put("organizationId", Integer.class.getName());
-        fields.put("address.street", Integer.class.getName());
+        fields.put("addr.street", Integer.class.getName());
 
         employeeEntity.setFields(fields);
 


[27/30] ignite git commit: fixed https://issues.apache.org/jira/browse/IGNITE-2078

Posted by ak...@apache.org.
fixed https://issues.apache.org/jira/browse/IGNITE-2078


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

Branch: refs/heads/ignite-1.5.1
Commit: 1bc137394e7530096f10cf5f38f4982d6c1115f3
Parents: 5ba664c
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Dec 10 20:07:16 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Dec 10 20:07:16 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/examples/model/Organization.java  | 16 ++++------------
 .../org/apache/ignite/examples/model/Person.java    | 16 ++++------------
 2 files changed, 8 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1bc13739/examples/src/main/java/org/apache/ignite/examples/model/Organization.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/model/Organization.java b/examples/src/main/java/org/apache/ignite/examples/model/Organization.java
index 3dcaa44..2500584 100644
--- a/examples/src/main/java/org/apache/ignite/examples/model/Organization.java
+++ b/examples/src/main/java/org/apache/ignite/examples/model/Organization.java
@@ -17,17 +17,16 @@
 
 package org.apache.ignite.examples.model;
 
-import org.apache.ignite.cache.query.annotations.QuerySqlField;
-
 import java.io.Serializable;
-import java.util.Random;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
 
 /**
  * Organization class.
  */
 public class Organization implements Serializable {
     /** */
-    private static final Random RND = new Random();
+    private static final AtomicLong ID_GEN = new AtomicLong();
 
     /** Organization ID (indexed). */
     @QuerySqlField(index = true)
@@ -38,19 +37,12 @@ public class Organization implements Serializable {
     public String name;
 
     /**
-     * Default empty constructor.
-     */
-    public Organization() {
-        // No-op.
-    }
-
-    /**
      * Create organization.
      *
      * @param name Organization name.
      */
     public Organization(String name) {
-        id = RND.nextLong();
+        id = ID_GEN.incrementAndGet();
 
         this.name = name;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1bc13739/examples/src/main/java/org/apache/ignite/examples/model/Person.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/model/Person.java b/examples/src/main/java/org/apache/ignite/examples/model/Person.java
index 5ccda51..5f7deae 100644
--- a/examples/src/main/java/org/apache/ignite/examples/model/Person.java
+++ b/examples/src/main/java/org/apache/ignite/examples/model/Person.java
@@ -17,19 +17,18 @@
 
 package org.apache.ignite.examples.model;
 
+import java.io.Serializable;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.cache.affinity.AffinityKey;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.cache.query.annotations.QueryTextField;
 
-import java.io.Serializable;
-import java.util.Random;
-
 /**
  * Person class.
  */
 public class Person implements Serializable {
     /** */
-    private static final Random RND = new Random();
+    private static final AtomicLong ID_GEN = new AtomicLong();
 
     /** Person ID (indexed). */
     @QuerySqlField(index = true)
@@ -59,13 +58,6 @@ public class Person implements Serializable {
     private transient AffinityKey<Long> key;
 
     /**
-     * Default empty constructor.
-     */
-    public Person() {
-        // No-op.
-    }
-
-    /**
      * Constructs person record.
      *
      * @param org       Organization.
@@ -76,7 +68,7 @@ public class Person implements Serializable {
      */
     public Person(Organization org, String firstName, String lastName, double salary, String resume) {
         // Generate unique ID for this person.
-        id = RND.nextLong();
+        id = ID_GEN.incrementAndGet();
 
         orgId = org.id;
 


[11/30] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

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

Branch: refs/heads/ignite-1.5.1
Commit: f24749e2cc2880715765096b755806a6c48a8356
Parents: f4ad128 f80637f
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 10 12:44:11 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 10 12:44:11 2015 +0300

----------------------------------------------------------------------
 ...idCacheReplicatedUnswapAdvancedSelfTest.java | 42 +++++--------------
 .../GridCacheReplicatedPreloadSelfTest.java     | 44 +++++---------------
 .../tests/p2p/CacheDeploymentTestValue3.java    | 41 ++++++++++++++++++
 3 files changed, 61 insertions(+), 66 deletions(-)
----------------------------------------------------------------------



[29/30] ignite git commit: IGNITE-2118 Updated cache store examples to produce stable output.

Posted by ak...@apache.org.
IGNITE-2118 Updated cache store examples to produce stable output.


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

Branch: refs/heads/ignite-1.5.1
Commit: 478657b4d08d38583ad6698390308f11ab8d29ff
Parents: 7fc6d81
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Dec 11 11:21:07 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 11 11:21:07 2015 +0700

----------------------------------------------------------------------
 .../store/auto/CacheBinaryAutoStoreExample.java | 38 ++++++++++++--------
 .../store/auto/CacheAutoStoreExample.java       | 32 +++++++++++------
 .../apache/ignite/examples/model/Person.java    | 36 ++++++++++++++++---
 .../ignite/examples/util/DbH2ServerStartup.java | 31 ++++++++++++----
 4 files changed, 101 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/478657b4/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 63d947c..aa228f1 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
@@ -56,7 +56,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
  */
 public class CacheBinaryAutoStoreExample {
     /** Global person ID to use across entire example. */
-    private static final Long id = Math.abs(UUID.randomUUID().getLeastSignificantBits());
+    private static final Long id = 25121642L;
 
     /** Cache name. */
     public static final String CACHE_NAME = CacheBinaryAutoStoreExample.class.getSimpleName();
@@ -81,9 +81,9 @@ public class CacheBinaryAutoStoreExample {
 
         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")
+            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);
@@ -108,32 +108,42 @@ public class CacheBinaryAutoStoreExample {
      * Executes example.
      *
      * @param args Command line arguments, none required.
-     * @throws IgniteException If example execution failed.
+     * @throws Exception If example execution failed.
      */
-    public static void main(String[] args) throws IgniteException {
+    public static void main(String[] args) throws Exception {
         // 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.");
+            System.out.println(">>> Populate database with data...");
+            DbH2ServerStartup.populateDatabase();
+
+            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);
+                    System.out.println(">>> Read value: " + val);
 
-                    val = cache.getAndPut(id, new Person(id, "Isaac", "Newton"));
+                    val = cache.getAndPut(id, new Person(id, 1L, "Isaac", "Newton", 100.10, "English physicist and mathematician"));
 
-                    System.out.println("Overwrote old value: " + val);
+                    System.out.println(">>> Overwrote old value: " + val);
 
                     val = cache.get(id);
 
-                    System.out.println("Read value: " + val);
+                    System.out.println(">>> Read value: " + val);
+
+                    System.out.println(">>> Update salary in transaction...");
+
+                    val.salary *= 2;
+
+                    cache.put(id, val);
 
                     tx.commit();
                 }
 
-                System.out.println("Read value after commit: " + cache.get(id));
+                System.out.println(">>> Read value after commit: " + cache.get(id));
 
                 cache.clear();
 
@@ -143,7 +153,7 @@ public class CacheBinaryAutoStoreExample {
                 // 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());
+                System.out.println(">>> Loaded cache entries: " + cache.size());
 
                 cache.clear();
 
@@ -151,7 +161,7 @@ public class CacheBinaryAutoStoreExample {
                 System.out.println(">>> Load ALL data to cache from DB...");
                 cache.loadCache(null);
 
-                System.out.println("Loaded cache entries: " + cache.size());
+                System.out.println(">>> Loaded cache entries: " + cache.size());
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/478657b4/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 7d21fce..a61752d 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
@@ -54,7 +54,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
  */
 public class CacheAutoStoreExample {
     /** Global person ID to use across entire example. */
-    private static final Long id = Math.abs(UUID.randomUUID().getLeastSignificantBits());
+    private static final Long id = 25121642L;
 
     /** Cache name. */
     public static final String CACHE_NAME = CacheAutoStoreExample.class.getSimpleName();
@@ -114,32 +114,42 @@ public class CacheAutoStoreExample {
      * Executes example.
      *
      * @param args Command line arguments, none required.
-     * @throws IgniteException If example execution failed.
+     * @throws Exception If example execution failed.
      */
-    public static void main(String[] args) throws IgniteException {
+    public static void main(String[] args) throws Exception {
         // 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.");
+            System.out.println(">>> Populate database with data...");
+            DbH2ServerStartup.populateDatabase();
+
+            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);
+                    System.out.println(">>> Read value: " + val);
 
-                    val = cache.getAndPut(id, new Person(id, "Isaac", "Newton"));
+                    val = cache.getAndPut(id, new Person(id, 1L, "Isaac", "Newton", 100.10, "English physicist and mathematician"));
 
-                    System.out.println("Overwrote old value: " + val);
+                    System.out.println(">>> Overwrote old value: " + val);
 
                     val = cache.get(id);
 
-                    System.out.println("Read value: " + val);
+                    System.out.println(">>> Read value: " + val);
+
+                    System.out.println(">>> Update salary in transaction...");
+
+                    val.salary *= 2;
+
+                    cache.put(id, val);
 
                     tx.commit();
                 }
 
-                System.out.println("Read value after commit: " + cache.get(id));
+                System.out.println(">>> Read value after commit: " + cache.get(id));
 
                 cache.clear();
 
@@ -149,7 +159,7 @@ public class CacheAutoStoreExample {
                 // 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());
+                System.out.println(">>> Loaded cache entries: " + cache.size());
 
                 cache.clear();
 
@@ -157,7 +167,7 @@ public class CacheAutoStoreExample {
                 System.out.println(">>> Load ALL data to cache from DB...");
                 cache.loadCache(null);
 
-                System.out.println("Loaded cache entries: " + cache.size());
+                System.out.println(">>> Loaded cache entries: " + cache.size());
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/478657b4/examples/src/main/java/org/apache/ignite/examples/model/Person.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/model/Person.java b/examples/src/main/java/org/apache/ignite/examples/model/Person.java
index 5f7deae..618fa5c 100644
--- a/examples/src/main/java/org/apache/ignite/examples/model/Person.java
+++ b/examples/src/main/java/org/apache/ignite/examples/model/Person.java
@@ -58,6 +58,13 @@ public class Person implements Serializable {
     private transient AffinityKey<Long> key;
 
     /**
+     * Default constructor.
+     */
+    public Person() {
+        // No-op.
+    }
+
+    /**
      * Constructs person record.
      *
      * @param org       Organization.
@@ -74,8 +81,27 @@ public class Person implements Serializable {
 
         this.firstName = firstName;
         this.lastName = lastName;
+        this.salary = salary;
         this.resume = resume;
+    }
+
+    /**
+     * Constructs person record.
+     *
+     * @param id Person ID.
+     * @param orgId Organization ID.
+     * @param firstName First name.
+     * @param lastName Last name.
+     * @param salary    Salary.
+     * @param resume    Resume text.
+     */
+    public Person(Long id, Long orgId, String firstName, String lastName, double salary, String resume) {
+        this.id = id;
+        this.orgId = orgId;
+        this.firstName = firstName;
+        this.lastName = lastName;
         this.salary = salary;
+        this.resume = resume;
     }
 
     /**
@@ -109,11 +135,11 @@ public class Person implements Serializable {
      * {@inheritDoc}
      */
     @Override public String toString() {
-        return "Person [firstName=" + firstName +
-                ", lastName=" + lastName +
-                ", id=" + id +
+        return "Person [id=" + id +
                 ", orgId=" + orgId +
-                ", resume=" + resume +
-                ", salary=" + salary + ']';
+                ", lastName=" + lastName +
+                ", firstName=" + firstName +
+                ", salary=" + salary +
+                ", resume=" + resume + ']';
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/478657b4/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
index 01717d0..f3da07d 100644
--- a/examples/src/main/java/org/apache/ignite/examples/util/DbH2ServerStartup.java
+++ b/examples/src/main/java/org/apache/ignite/examples/util/DbH2ServerStartup.java
@@ -30,17 +30,34 @@ import org.h2.tools.Server;
  */
 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));";
+    private static final String CREATE_PERSON_TABLE =
+        "create table if not exists 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 =
+    private static final String POPULATE_PERSON_TABLE =
+        "delete from PERSON;\n" +
         "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');";
+        "insert into PERSON(id, first_name, last_name) values(6, 'Wilhelm', 'Leibniz');";
+
+    /**
+     * Populate sample database.
+     *
+     * @throws SQLException if
+     */
+    public static void populateDatabase() throws SQLException {
+        // 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_PERSON_TABLE));
+
+        // Populates Person table with sample data in database.
+        RunScript.execute(dataSrc.getConnection(), new StringReader(POPULATE_PERSON_TABLE));
+    }
 
     /**
      * Start H2 database TCP server.
@@ -53,14 +70,16 @@ public class DbH2ServerStartup {
             // Start H2 database TCP server in order to access sample in-memory database from other processes.
             Server.createTcpServer("-tcpDaemon").start();
 
+            populateDatabase();
+
             // 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));
+            RunScript.execute(dataSrc.getConnection(), new StringReader(CREATE_PERSON_TABLE));
 
             // Populates Person table with sample data in database.
-            RunScript.execute(dataSrc.getConnection(), new StringReader(POPULATE_TABLE));
+            RunScript.execute(dataSrc.getConnection(), new StringReader(POPULATE_PERSON_TABLE));
         }
         catch (SQLException e) {
             throw new IgniteException("Failed to start database TCP server", e);


[23/30] ignite git commit: #Ignite-1.5 - Minor corrections.

Posted by ak...@apache.org.
#Ignite-1.5 - Minor corrections.


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

Branch: refs/heads/ignite-1.5.1
Commit: 5c2325765fe8370118ef135231632776443ea454
Parents: dfd7a67
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Dec 10 18:56:36 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Dec 10 18:56:36 2015 +0300

----------------------------------------------------------------------
 .../examples/binary/datagrid/CacheClientBinaryPutGetExample.java | 2 +-
 .../examples/binary/datagrid/CacheClientBinaryQueryExample.java  | 4 ++--
 .../org/apache/ignite/marshaller/optimized/package-info.java     | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5c232576/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryPutGetExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryPutGetExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryPutGetExample.java
index c416501..26f8d71 100644
--- a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryPutGetExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryPutGetExample.java
@@ -68,7 +68,7 @@ public class CacheClientBinaryPutGetExample {
             cfg.setName(CACHE_NAME);
             cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
 
-            try (IgniteCache<Integer, Organization> cache = ignite.createCache(cfg)) {
+            try (IgniteCache<Integer, Organization> cache = ignite.getOrCreateCache(cfg)) {
                 if (ignite.cluster().forDataNodes(cache.getName()).nodes().isEmpty()) {
                     System.out.println();
                     System.out.println(">>> This example requires remote cache node nodes to be started.");

http://git-wip-us.apache.org/repos/asf/ignite/blob/5c232576/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
index 34452e4..c8beaf4 100644
--- a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
@@ -81,8 +81,8 @@ public class CacheClientBinaryQueryExample {
 
             employeeCacheCfg.setQueryEntities(Arrays.asList(createEmployeeQueryEntity()));
 
-            try (IgniteCache<Integer, Organization> orgCache = ignite.createCache(orgCacheCfg);
-                 IgniteCache<EmployeeKey, Employee> employeeCache = ignite.createCache(employeeCacheCfg)
+            try (IgniteCache<Integer, Organization> orgCache = ignite.getOrCreateCache(orgCacheCfg);
+                 IgniteCache<EmployeeKey, Employee> employeeCache = ignite.getOrCreateCache(employeeCacheCfg)
             ) {
                 if (ignite.cluster().forDataNodes(orgCache.getName()).nodes().isEmpty()) {
                     System.out.println();

http://git-wip-us.apache.org/repos/asf/ignite/blob/5c232576/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/package-info.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/package-info.java
index de1fad0..bb61add 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/package-info.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/package-info.java
@@ -17,6 +17,6 @@
 
 /**
  * <!-- Package description. -->
- * Contains <b>default</b> Optimized marshaller.
+ * Contains Optimized marshaller.
  */
 package org.apache.ignite.marshaller.optimized;
\ No newline at end of file


[18/30] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5_

Posted by ak...@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/9012b39d
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/9012b39d
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/9012b39d

Branch: refs/heads/ignite-1.5.1
Commit: 9012b39d2f06cd124ff3830167af8b0953df7f64
Parents: 67f4afa d466c7a
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Dec 10 14:01:41 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Dec 10 14:01:41 2015 +0300

----------------------------------------------------------------------
 modules/osgi-karaf/src/main/resources/features.xml | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[21/30] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

Posted by ak...@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/388c1736
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/388c1736
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/388c1736

Branch: refs/heads/ignite-1.5.1
Commit: 388c173651781232205e545453cfdaf636e11a31
Parents: 46872d6 2e68a87
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 10 16:45:34 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 10 16:45:34 2015 +0300

----------------------------------------------------------------------
 .../examples/datagrid/CacheTransactionExample.java       |  4 ++--
 .../apache/ignite/internal/util/nio/GridNioServer.java   | 11 +++++------
 2 files changed, 7 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[07/30] ignite git commit: IGNITE-2064 Fixed more tests.

Posted by ak...@apache.org.
IGNITE-2064 Fixed more tests.


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

Branch: refs/heads/ignite-1.5.1
Commit: 959c4d00220b9eb4f516d2b858b7b23c7fcf51b5
Parents: be17c9e
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 10 12:34:36 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 10 12:34:36 2015 +0300

----------------------------------------------------------------------
 ...idCacheReplicatedUnswapAdvancedSelfTest.java | 42 +++++--------------
 .../GridCacheReplicatedPreloadSelfTest.java     | 44 +++++---------------
 .../tests/p2p/CacheDeploymentTestValue3.java    | 41 ++++++++++++++++++
 3 files changed, 61 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/959c4d00/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
index e574418..8f3f3f8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.replicated;
 
-import java.io.Serializable;
 import java.util.Collections;
 import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.Ignite;
@@ -26,13 +25,12 @@ import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
-import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.lang.IgnitePredicate;
 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.spi.swapspace.file.FileSwapSpaceSpi;
-import org.apache.ignite.testframework.GridTestClassLoader;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
@@ -50,9 +48,6 @@ public class GridCacheReplicatedUnswapAdvancedSelfTest extends GridCommonAbstrac
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setPeerClassLoadingLocalClassPathExclude(GridCacheReplicatedUnswapAdvancedSelfTest.class.getName(),
-            TestClass.class.getName());
-
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
         discoSpi.setIpFinder(IP_FINDER);
@@ -68,6 +63,9 @@ public class GridCacheReplicatedUnswapAdvancedSelfTest extends GridCommonAbstrac
 
         cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
+        if (getTestGridName(1).equals(gridName) || cfg.getMarshaller() instanceof BinaryMarshaller)
+            cfg.setClassLoader(getExternalClassLoader());
+
         return cfg;
     }
 
@@ -84,11 +82,9 @@ public class GridCacheReplicatedUnswapAdvancedSelfTest extends GridCommonAbstrac
         IgniteCache<Object, Object> cache2 = g2.cache(null);
 
         try {
-            ClassLoader ldr = new GridTestClassLoader(
-                GridCacheReplicatedUnswapAdvancedSelfTest.class.getName(),
-                TestClass.class.getName());
+            ClassLoader ldr = grid(1).configuration().getClassLoader();
 
-            Object v = ldr.loadClass(TestClass.class.getName()).newInstance();
+            Object v = ldr.loadClass("org.apache.ignite.tests.p2p.CacheDeploymentTestValue3").newInstance();
 
             info("v loader: " + v.getClass().getClassLoader());
 
@@ -132,7 +128,8 @@ public class GridCacheReplicatedUnswapAdvancedSelfTest extends GridCommonAbstrac
             assert v2 != null;
             assert v2.toString().equals(v.toString());
             assert !v2.getClass().getClassLoader().equals(getClass().getClassLoader());
-            assert v2.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader");
+            assert v2.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader")||
+                grid(2).configuration().getMarshaller() instanceof BinaryMarshaller;
 
             // To swap storage.
             cache2.localEvict(Collections.<Object>singleton(key));
@@ -145,31 +142,12 @@ public class GridCacheReplicatedUnswapAdvancedSelfTest extends GridCommonAbstrac
 
             assert v2 != null;
 
-            assert v2.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader");
+            assert v2.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader")|
+                grid(2).configuration().getMarshaller() instanceof BinaryMarshaller;
         }
         finally {
             stopGrid(1);
             stopGrid(2);
         }
     }
-    /**
-     * Test class.
-     */
-    @SuppressWarnings("PublicInnerClass")
-    public static class TestClass implements Serializable {
-        /** String value. */
-        private String s = "Test string";
-
-        /**
-         * @return String value.
-         */
-        public String getStr() {
-            return s;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(TestClass.class, this);
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/959c4d00/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
index 64f1495..5ee27c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.replicated.preloader;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -29,7 +28,6 @@ import java.util.Random;
 import java.util.UUID;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.affinity.AffinityFunction;
@@ -39,16 +37,15 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 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.spi.eventstorage.memory.MemoryEventStorageSpi;
-import org.apache.ignite.testframework.GridTestClassLoader;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -93,8 +90,6 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
         cfg.setDiscoverySpi(disco);
 
         cfg.setCacheConfiguration(cacheConfiguration(gridName));
-        cfg.setPeerClassLoadingLocalClassPathExclude(GridCacheReplicatedPreloadSelfTest.class.getName(),
-            TestValue.class.getName());
 
         cfg.setDeploymentMode(CONTINUOUS);
 
@@ -106,6 +101,9 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
 
         cfg.setEventStorageSpi(spi);
 
+        if (getTestGridName(1).equals(gridName) || cfg.getMarshaller() instanceof BinaryMarshaller)
+            cfg.setClassLoader(getExternalClassLoader());
+
         return cfg;
     }
 
@@ -217,12 +215,9 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
             IgniteCache<Integer, Object> cache1 = g1.cache(null);
             IgniteCache<Integer, Object> cache2 = g2.cache(null);
 
-            ClassLoader ldr = new GridTestClassLoader(
-                GridCacheReplicatedPreloadSelfTest.class.getName(),
-                TestValue.class.getName(),
-                TestAffinityFunction.class.getName());
+            ClassLoader ldr = grid(1).configuration().getClassLoader();
 
-            Object v1 = ldr.loadClass(TestValue.class.getName()).newInstance();
+            Object v1 = ldr.loadClass("org.apache.ignite.tests.p2p.CacheDeploymentTestValue3").newInstance();
 
             cache1.put(1, v1);
 
@@ -235,7 +230,8 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
             assert v2 != null;
             assert v2.toString().equals(v1.toString());
             assert !v2.getClass().getClassLoader().equals(getClass().getClassLoader());
-            assert v2.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader");
+            assert v2.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader") ||
+                grid(2).configuration().getMarshaller() instanceof BinaryMarshaller;
 
             stopGrid(1);
 
@@ -252,7 +248,8 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
             assert v3 != null;
             assert v3.toString().equals(v1.toString());
             assert !v3.getClass().getClassLoader().equals(getClass().getClassLoader());
-            assert v3.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader");
+            assert v3.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader")||
+                grid(3).configuration().getMarshaller() instanceof BinaryMarshaller;
         }
         finally {
             stopAllGrids();
@@ -504,27 +501,6 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     *
-     */
-    @SuppressWarnings({"PublicInnerClass"})
-    public static class TestValue implements Serializable {
-        /** */
-        private String val = "test-" + System.currentTimeMillis();
-
-        /**
-         * @return Value
-         */
-        public String getValue() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(TestValue.class, this);
-        }
-    }
-
-    /**
      * Test affinity.
      */
     @SuppressWarnings({"PublicInnerClass"})

http://git-wip-us.apache.org/repos/asf/ignite/blob/959c4d00/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestValue3.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestValue3.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestValue3.java
new file mode 100644
index 0000000..643b0fc
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestValue3.java
@@ -0,0 +1,41 @@
+/*
+ *  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.tests.p2p;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Third test value for deployment.
+ */
+public class CacheDeploymentTestValue3 implements Serializable {
+    /** */
+    private String val = "test-" + System.currentTimeMillis();
+
+    /**
+     * @return Value
+     */
+    public String getValue() {
+        return val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheDeploymentTestValue3.class, this);
+    }
+}
\ No newline at end of file


[04/30] ignite git commit: IGNITE-2064 Fixed deserialize. Added more tests.

Posted by ak...@apache.org.
IGNITE-2064 Fixed deserialize. Added more tests.


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

Branch: refs/heads/ignite-1.5.1
Commit: be17c9eef8a1afa647214089e3d7689e97bb5bea
Parents: 9413232
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Dec 9 17:33:59 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 10 11:17:25 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryObjectImpl.java     |  28 ++--
 .../internal/portable/PortableContext.java      |   7 +
 .../processors/cache/CacheLazyEntry.java        |  12 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   2 +-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |  69 ++++++----
 ...yAtomicEntryProcessorDeploymentSelfTest.java | 129 +++++++++++++++++++
 ...ctionalEntryProcessorDeploymentSelfTest.java |  31 +++++
 .../IgniteBinaryObjectsCacheTestSuite3.java     |   9 +-
 ...cheDeploymentBinaryObjectEntryProcessor.java |  36 ++++++
 9 files changed, 284 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
index cf5a659..c8528d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
@@ -17,6 +17,16 @@
 
 package org.apache.ignite.internal.portable;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
@@ -33,17 +43,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.Nullable;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.sql.Timestamp;
-import java.util.Date;
-import java.util.UUID;
-
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;
@@ -543,6 +542,8 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     /**
      * Runs value deserialization regardless of whether obj already has the deserialized value.
      * Will set obj if descriptor is configured to keep deserialized values.
+     * @param coCtx CacheObjectContext.
+     * @return Object.
      */
     private Object deserializeValue(@Nullable CacheObjectContext coCtx) {
         // TODO: IGNITE-1272 - Deserialize with proper class loader.
@@ -575,6 +576,9 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
      * @return Reader.
      */
     private BinaryReaderExImpl reader(@Nullable BinaryReaderHandles rCtx) {
-        return new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, start), null, rCtx);
+        return new BinaryReaderExImpl(ctx,
+            PortableHeapInputStream.create(arr, start),
+            ctx.configuration().getClassLoader(),
+            rCtx);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/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 8bad737..2c7e4c3 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
@@ -234,6 +234,13 @@ public class PortableContext implements Externalizable {
     }
 
     /**
+     * @return Ignite configuration.
+     */
+    public IgniteConfiguration configuration(){
+        return igniteCfg;
+    }
+
+    /**
      * @param marsh Portable marshaller.
      * @param cfg Configuration.
      * @throws BinaryObjectException In case of error.

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
index 2b9efa3..713bb3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
@@ -77,7 +77,7 @@ public class CacheLazyEntry<K, V> implements Cache.Entry<K, V> {
      * @param valObj Cache object
      * @param val Cache value.
      */
-    public CacheLazyEntry(GridCacheContext<K, V> ctx, 
+    public CacheLazyEntry(GridCacheContext<K, V> ctx,
         KeyCacheObject keyObj,
         K key,
         CacheObject valObj,
@@ -102,6 +102,16 @@ public class CacheLazyEntry<K, V> implements Cache.Entry<K, V> {
 
     /** {@inheritDoc} */
     @Override public V getValue() {
+        return getValue(keepPortable);
+    }
+
+    /**
+     * Returns the value stored in the cache when this entry was created.
+     *
+     * @param keepPortable Flag to keep portable if needed.
+     * @return the value corresponding to this entry
+     */
+    public V getValue(boolean keepPortable) {
         if (val == null)
             val = (V)cctx.unwrapPortableIfNeeded(valObj, keepPortable, false);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/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 3829e28..40399b4 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
@@ -382,7 +382,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
                                     procRes = processor.process(invokeEntry, t.get2());
 
-                                    val = cacheCtx.toCacheObject(invokeEntry.getValue());
+                                    val = cacheCtx.toCacheObject(invokeEntry.getValue(true));
                                 }
                                 catch (Exception e) {
                                     err = e;

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/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 35ee6cc..137e90f 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
@@ -45,27 +45,24 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
-    /** Entry processor */
-    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";
 
     /** */
-    private DeploymentMode depMode;
+    protected DeploymentMode depMode;
 
     /** */
-    private boolean cliendMode;
+    protected boolean clientMode;
+
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        if (cliendMode)
-            cfg.setClientMode(cliendMode);
+        cfg.setClientMode(clientMode);
+
+        if (clientMode)
+            cfg.setClassLoader(getExternalClassLoader());
 
         cfg.setDeploymentMode(depMode);
 
@@ -99,6 +96,22 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
         return cfg;
     }
 
+    /**
+     * @return Cache.
+     */
+    protected IgniteCache getCache(){
+        return grid(1).cache(null);
+    }
+
+    /**
+     * @return Entry Processor.
+     */
+    protected String getEntryProcessor(){
+       return GridTestProperties.getProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME) != null ?
+            GridTestProperties.getProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME) :
+            "org.apache.ignite.tests.p2p.CacheDeploymentEntryProcessor";
+    }
+
     protected CacheAtomicityMode atomicityMode() {
         return ATOMIC;
     }
@@ -144,26 +157,28 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
      */
     private void doTestInvoke() throws Exception {
         try {
-            cliendMode = false;
+            clientMode = false;
             startGrid(0);
 
-            cliendMode = true;
+            clientMode = true;
             startGrid(1);
 
-            ClassLoader ldr = getExternalClassLoader();
-
-            Class procCls = ldr.loadClass(TEST_ENT_PROCESSOR);
-            Class valCls = ldr.loadClass(TEST_VALUE);
+            Class procCls = grid(1).configuration().getClassLoader().loadClass(getEntryProcessor());
+            Class valCls = grid(1).configuration().getClassLoader().loadClass(TEST_VALUE);
 
             assertTrue(grid(1).configuration().isClientMode());
+            assertFalse(grid(0).configuration().isClientMode());
 
-            IgniteCache cache = grid(1).cache(null);
+            IgniteCache cache = getCache();
 
             cache.put("key", valCls.newInstance());
 
             Boolean res = (Boolean)cache.invoke("key", (CacheEntryProcessor)procCls.newInstance());
 
             assertTrue(res);
+
+            // Checks that get produces no exceptions.
+            cache.get("key");
         }
         finally {
             stopAllGrids();
@@ -175,20 +190,19 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
      */
     private void doTestInvokeAll() throws Exception {
         try {
-            cliendMode = false;
+            clientMode = false;
             startGrid(0);
 
-            cliendMode = true;
+            clientMode = true;
             startGrid(1);
 
-            ClassLoader ldr = getExternalClassLoader();
-
-            Class procCls = ldr.loadClass(TEST_ENT_PROCESSOR);
-            Class valCls = ldr.loadClass(TEST_VALUE);
+            Class procCls = grid(1).configuration().getClassLoader().loadClass(getEntryProcessor());
+            Class valCls = grid(1).configuration().getClassLoader().loadClass(TEST_VALUE);
 
             assertTrue(grid(1).configuration().isClientMode());
+            assertFalse(grid(0).configuration().isClientMode());
 
-            IgniteCache cache = grid(1).cache(null);
+            IgniteCache cache = getCache();
 
             HashSet keys = new HashSet();
 
@@ -207,6 +221,13 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
 
             for (EntryProcessorResult result : res.values())
                 assertTrue((Boolean)result.get());
+
+            // Checks that get produces no exceptions.
+            for (int i = 0; i < 3; i++) {
+                String key = "key" + i;
+
+                cache.get(key);
+            }
         }
         finally {
             stopAllGrids();

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest.java
new file mode 100644
index 0000000..1004e7a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest.java
@@ -0,0 +1,129 @@
+/*
+ *  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.portable;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.configuration.DeploymentMode;
+import org.apache.ignite.internal.processors.cache.GridCacheAtomicEntryProcessorDeploymentSelfTest;
+
+/**
+ * Cache EntryProcessor + Deployment.
+ */
+public class GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest
+    extends GridCacheAtomicEntryProcessorDeploymentSelfTest {
+    /** {@inheritDoc} */
+    protected IgniteCache getCache() {
+        return grid(1).cache(null).withKeepBinary();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String getEntryProcessor() {
+        return "org.apache.ignite.tests.p2p.CacheDeploymentBinaryObjectEntryProcessor";
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetDeployment() throws Exception {
+        depMode = DeploymentMode.CONTINUOUS;
+
+        doTestGet(false);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetDeployment2() throws Exception {
+        depMode = DeploymentMode.SHARED;
+
+        doTestGet(false);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetDeploymentWithKeepBinary() throws Exception {
+        depMode = DeploymentMode.CONTINUOUS;
+
+        doTestGet(true);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetDeployment2WithKeepBinary() throws Exception {
+        depMode = DeploymentMode.SHARED;
+
+        doTestGet(true);
+    }
+
+    /**
+     * @throws Exception Exception.
+     */
+    private void doTestGet(boolean withKeepBinary) throws Exception {
+        try {
+            clientMode = false;
+            startGrid(0);
+
+            clientMode = true;
+            startGrid(1);
+
+            Class valCls = grid(1).configuration().getClassLoader().loadClass(TEST_VALUE);
+
+            assertTrue(grid(1).configuration().isClientMode());
+            assertFalse(grid(0).configuration().isClientMode());
+
+            IgniteCache cache1 = grid(1).cache(null);
+            IgniteCache cache0 = grid(0).cache(null);
+
+            if (withKeepBinary) {
+                cache1 = cache1.withKeepBinary();
+                cache0 = cache0.withKeepBinary();
+            }
+
+            cache1.put("key", valCls.newInstance());
+
+            if (withKeepBinary) {
+                BinaryObject obj = (BinaryObject)(cache0.get("key"));
+
+                try {
+                    obj.deserialize();
+
+                    fail("Exception did not happened.");
+                }
+                catch (BinaryInvalidTypeException e) {
+                    // No-op.
+                }
+            }
+            else
+                try {
+                    cache0.get("key");
+
+                    fail("Exception did not happened.");
+                }
+                catch (BinaryInvalidTypeException e) {
+                    // No-op.
+                }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest.java
new file mode 100644
index 0000000..c115754
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest.java
@@ -0,0 +1,31 @@
+/*
+ *  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.portable;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+
+/**
+ * Cache EntryProcessor + Deployment for transactional cache.
+ */
+public class GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest extends
+    GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/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 3d25645..3aecda3 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
@@ -19,6 +19,8 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.portable.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest;
 import org.apache.ignite.testframework.config.GridTestProperties;
 
 /**
@@ -34,6 +36,11 @@ public class IgniteBinaryObjectsCacheTestSuite3 {
         GridTestProperties.setProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME,
             "org.apache.ignite.tests.p2p.CacheDeploymentPortableEntryProcessor");
 
-        return IgniteCacheTestSuite3.suite();
+        TestSuite suite = IgniteCacheTestSuite3.suite();
+
+        suite.addTestSuite(GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest.class);
+        suite.addTestSuite(GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest.class);
+
+        return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentBinaryObjectEntryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentBinaryObjectEntryProcessor.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentBinaryObjectEntryProcessor.java
new file mode 100644
index 0000000..7c6895c
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentBinaryObjectEntryProcessor.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.tests.p2p;
+
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CacheEntryProcessor;
+
+/**
+ * Entry processor for {@code GridCacheEntryProcessorDeploymentSelfTest}.
+ */
+public class CacheDeploymentBinaryObjectEntryProcessor implements CacheEntryProcessor<String, BinaryObject, Boolean> {
+    /** {@inheritDoc} */
+    @Override public Boolean process(MutableEntry<String, BinaryObject> entry,
+        Object... arguments) throws EntryProcessorException {
+        BinaryObject val = entry.getValue();
+
+        return true;
+    }
+}


[28/30] ignite git commit: fixed https://issues.apache.org/jira/browse/IGNITE-2035

Posted by ak...@apache.org.
fixed https://issues.apache.org/jira/browse/IGNITE-2035


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

Branch: refs/heads/ignite-1.5.1
Commit: 7fc6d812f4f94d987f0d2d92182ff85cbb7807f6
Parents: 1bc1373
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Dec 10 20:14:28 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Dec 10 20:14:28 2015 +0300

----------------------------------------------------------------------
 .../computegrid/ComputeClientBinaryTaskExecutionExample.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7fc6d812/examples/src/main/java/org/apache/ignite/examples/binary/computegrid/ComputeClientBinaryTaskExecutionExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/computegrid/ComputeClientBinaryTaskExecutionExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/computegrid/ComputeClientBinaryTaskExecutionExample.java
index 49f254a..d839c96 100644
--- a/examples/src/main/java/org/apache/ignite/examples/binary/computegrid/ComputeClientBinaryTaskExecutionExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/computegrid/ComputeClientBinaryTaskExecutionExample.java
@@ -34,8 +34,8 @@ import org.apache.ignite.binary.BinaryObject;
  * Since these objects are never deserialized on remote nodes, classes are not required on classpath
  * of these nodes.
  * <p>
- * Remote nodes should always be started with special configuration file which
- * enables the binary marshaller: {@code 'ignite.{sh|bat} examples/config/binary/example-ignite-binary.xml'}.
+ * Remote nodes should always be started with the following command:
+ * {@code 'ignite.{sh|bat} examples/config/example-ignite.xml'}.
  * <p>
  * Alternatively you can run {@link org.apache.ignite.examples.ExampleNodeStartup} in another JVM which will
  * start a node with {@code examples/config/example-ignite.xml} configuration.


[30/30] ignite git commit: Merge branches 'ignite-1.5' and 'ignite-1.5.1' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5.1

Posted by ak...@apache.org.
Merge branches 'ignite-1.5' and 'ignite-1.5.1' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5.1


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

Branch: refs/heads/ignite-1.5.1
Commit: 09b394e1740a37e8833c91600be27a7ea0021366
Parents: c9b7149 478657b
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Dec 11 11:22:24 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 11 11:22:24 2015 +0700

----------------------------------------------------------------------
 ...ComputeClientBinaryTaskExecutionExample.java |   4 +-
 .../CacheClientBinaryPutGetExample.java         |   2 +-
 .../datagrid/CacheClientBinaryQueryExample.java |   8 +-
 .../store/auto/CacheBinaryAutoStoreExample.java |  38 ++-
 .../datagrid/CacheTransactionExample.java       |   4 +-
 .../store/auto/CacheAutoStoreExample.java       |  32 +-
 .../ignite/examples/model/Organization.java     |  16 +-
 .../apache/ignite/examples/model/Person.java    |  40 ++-
 .../ignite/examples/util/DbH2ServerStartup.java |  31 +-
 .../main/java/org/apache/ignite/Ignition.java   |   2 +-
 .../ignite/binary/BinaryCollectionFactory.java  |  33 ++
 .../apache/ignite/binary/BinaryMapFactory.java  |  33 ++
 .../apache/ignite/binary/BinaryRawReader.java   |   8 +-
 .../org/apache/ignite/binary/BinaryReader.java  |   9 +-
 .../configuration/IgniteConfiguration.java      |   2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  24 +-
 .../internal/portable/BinaryFieldAccessor.java  |  19 +-
 .../internal/portable/BinaryObjectImpl.java     |  28 +-
 .../internal/portable/BinaryReaderExImpl.java   | 107 +++---
 .../internal/portable/BinaryWriteMode.java      |   3 -
 .../internal/portable/BinaryWriterExImpl.java   |  45 +--
 .../portable/GridPortableMarshaller.java        |  21 --
 .../portable/PortableClassDescriptor.java       |  21 +-
 .../internal/portable/PortableContext.java      |  78 ++---
 .../ignite/internal/portable/PortableUtils.java | 144 ++------
 .../portable/builder/PortableBuilderReader.java |  12 -
 .../builder/PortableBuilderSerializer.java      |  17 +-
 .../portable/builder/PortableLazyMapEntry.java  |  68 ----
 .../processors/cache/CacheLazyEntry.java        |  12 +-
 .../processors/cache/GridCacheIoManager.java    |   5 +-
 .../processors/cache/GridCacheProcessor.java    |  17 +
 .../distributed/dht/GridDhtTxPrepareFuture.java |   2 +-
 .../ignite/internal/util/nio/GridNioServer.java |  11 +-
 .../marshaller/optimized/package-info.java      |   2 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  11 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |  22 +-
 .../IgniteClientReconnectAbstractTest.java      |   1 +
 .../IgniteClientReconnectCacheTest.java         |  10 +-
 .../portable/BinaryMarshallerSelfTest.java      | 325 ++++++++++++++++++-
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 101 +++---
 .../GridPortableAffinityKeySelfTest.java        |   2 +-
 .../mutabletest/GridPortableTestClasses.java    |  77 ++++-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |  69 ++--
 .../IgniteCacheEntryListenerAbstractTest.java   |   8 +-
 ...niteCacheClientNodeChangingTopologyTest.java | 125 ++++++-
 ...idCacheReplicatedUnswapAdvancedSelfTest.java |  42 +--
 .../GridCacheReplicatedPreloadSelfTest.java     |  44 +--
 ...yAtomicEntryProcessorDeploymentSelfTest.java | 129 ++++++++
 ...ctionalEntryProcessorDeploymentSelfTest.java |  31 ++
 ...CacheClientNodeBinaryObjectMetadataTest.java |   2 +-
 ...ridCacheAffinityRoutingPortableSelfTest.java |   2 +-
 ...lientDiscoverySpiFailureTimeoutSelfTest.java |   5 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  63 +++-
 .../IgniteBinaryObjectsCacheTestSuite3.java     |   9 +-
 ...cheDeploymentBinaryObjectEntryProcessor.java |  36 ++
 .../tests/p2p/CacheDeploymentTestValue3.java    |  41 +++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   4 -
 .../osgi-karaf/src/main/resources/features.xml  |  11 +-
 .../ignite/osgi/classloaders/package-info.java  |  21 ++
 .../org/apache/ignite/osgi/package-info.java    |  21 ++
 .../IgniteKarafFeaturesInstallationTest.java    |   2 +-
 .../src/binary_reader_writer_raw_test.cpp       |  10 +-
 .../core-test/src/binary_reader_writer_test.cpp |  10 +-
 .../core/include/ignite/binary/binary_consts.h  |  29 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |   8 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |  27 +-
 .../Impl/Binary/BinaryUtils.cs                  |  93 +-----
 67 files changed, 1446 insertions(+), 843 deletions(-)
----------------------------------------------------------------------